From 23ed0dfd6ed0bf86f8a06f2f14af817b3caca11e Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Tue, 7 Jan 2025 16:44:19 +0800 Subject: [PATCH 01/27] statistics: add recover to protect background task (#58739) close pingcap/tidb#58738 --- pkg/metrics/metrics.go | 1 + pkg/statistics/handle/usage/session_stats_collect.go | 2 ++ pkg/statistics/handle/util/BUILD.bazel | 1 + pkg/statistics/handle/util/util.go | 2 ++ 4 files changed, 6 insertions(+) diff --git a/pkg/metrics/metrics.go b/pkg/metrics/metrics.go index 5b2e26ef23b97..17670041c2820 100644 --- a/pkg/metrics/metrics.go +++ b/pkg/metrics/metrics.go @@ -45,6 +45,7 @@ const ( LabelGCWorker = "gcworker" LabelAnalyze = "analyze" LabelWorkerPool = "worker-pool" + LabelStats = "stats" LabelBatchRecvLoop = "batch-recv-loop" LabelBatchSendLoop = "batch-send-loop" diff --git a/pkg/statistics/handle/usage/session_stats_collect.go b/pkg/statistics/handle/usage/session_stats_collect.go index 805a85a02f4cf..fcabb207b1d5d 100644 --- a/pkg/statistics/handle/usage/session_stats_collect.go +++ b/pkg/statistics/handle/usage/session_stats_collect.go @@ -85,6 +85,7 @@ func (s *statsUsageImpl) needDumpStatsDelta(is infoschema.InfoSchema, dumpAll bo // DumpStatsDeltaToKV sweeps the whole list and updates the global map, then we dumps every table that held in map to KV. // If the mode is `DumpDelta`, it will only dump that delta info that `Modify Count / Table Count` greater than a ratio. func (s *statsUsageImpl) DumpStatsDeltaToKV(dumpAll bool) error { + defer util.Recover(metrics.LabelStats, "DumpStatsDeltaToKV", nil, false) start := time.Now() defer func() { dur := time.Since(start) @@ -217,6 +218,7 @@ func (s *statsUsageImpl) dumpTableStatCountToKV(is infoschema.InfoSchema, physic // DumpColStatsUsageToKV sweeps the whole list, updates the column stats usage map and dumps it to KV. func (s *statsUsageImpl) DumpColStatsUsageToKV() error { + defer util.Recover(metrics.LabelStats, "DumpColStatsUsageToKV", nil, false) s.SweepSessionStatsList() colMap := s.SessionStatsUsage().GetUsageAndReset() defer func() { diff --git a/pkg/statistics/handle/util/BUILD.bazel b/pkg/statistics/handle/util/BUILD.bazel index a40c702ffb6b6..dd5915b072e24 100644 --- a/pkg/statistics/handle/util/BUILD.bazel +++ b/pkg/statistics/handle/util/BUILD.bazel @@ -15,6 +15,7 @@ go_library( "//pkg/infoschema", "//pkg/kv", "//pkg/meta/model", + "//pkg/metrics", "//pkg/parser/terror", "//pkg/planner/core/resolve", "//pkg/sessionctx", diff --git a/pkg/statistics/handle/util/util.go b/pkg/statistics/handle/util/util.go index e01ed16ca52a1..bed79fc1a5998 100644 --- a/pkg/statistics/handle/util/util.go +++ b/pkg/statistics/handle/util/util.go @@ -23,6 +23,7 @@ import ( "github.com/pingcap/failpoint" "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/terror" "github.com/pingcap/tidb/pkg/planner/core/resolve" "github.com/pingcap/tidb/pkg/sessionctx" @@ -75,6 +76,7 @@ var ( // CallWithSCtx allocates a sctx from the pool and call the f(). func CallWithSCtx(pool util.SessionPool, f func(sctx sessionctx.Context) error, flags ...int) (err error) { + defer util.Recover(metrics.LabelStats, "CallWithSCtx", nil, false) se, err := pool.Get() if err != nil { return err From b2f2faaa95bff642920560c24e3714957bcb4c74 Mon Sep 17 00:00:00 2001 From: D3Hunter Date: Tue, 7 Jan 2025 18:20:04 +0800 Subject: [PATCH 02/27] parser: move 'model' to 'ast' pkg (#58704) ref pingcap/tidb#54436 --- br/pkg/checkpoint/BUILD.bazel | 4 +- br/pkg/checkpoint/checkpoint_test.go | 16 +- br/pkg/checkpoint/log_restore.go | 20 +- br/pkg/checkpoint/restore.go | 4 +- br/pkg/checkpoint/storage.go | 6 +- br/pkg/checksum/BUILD.bazel | 2 +- br/pkg/checksum/executor_test.go | 6 +- br/pkg/glue/BUILD.bazel | 2 +- br/pkg/glue/glue.go | 4 +- br/pkg/gluetidb/BUILD.bazel | 4 +- br/pkg/gluetidb/glue.go | 4 +- br/pkg/gluetidb/glue_test.go | 24 +- br/pkg/gluetidb/mock/BUILD.bazel | 2 +- br/pkg/gluetidb/mock/mock.go | 4 +- br/pkg/metautil/BUILD.bazel | 2 +- br/pkg/metautil/load_test.go | 14 +- br/pkg/restore/BUILD.bazel | 4 +- br/pkg/restore/ingestrec/BUILD.bazel | 4 +- br/pkg/restore/ingestrec/ingest_recorder.go | 6 +- .../restore/ingestrec/ingest_recorder_test.go | 66 +-- .../restore/internal/prealloc_db/BUILD.bazel | 4 +- br/pkg/restore/internal/prealloc_db/db.go | 6 +- .../restore/internal/prealloc_db/db_test.go | 46 +- br/pkg/restore/misc.go | 6 +- br/pkg/restore/misc_test.go | 8 +- br/pkg/restore/snap_client/BUILD.bazel | 4 +- br/pkg/restore/snap_client/client_test.go | 16 +- .../placement_rule_manager_test.go | 10 +- .../restore/snap_client/systable_restore.go | 12 +- .../snap_client/systable_restore_test.go | 16 +- .../restore/snap_client/tikv_sender_test.go | 24 +- br/pkg/restore/tiflashrec/BUILD.bazel | 2 +- .../tiflashrec/tiflash_recorder_test.go | 6 +- br/pkg/restore/utils/BUILD.bazel | 2 +- br/pkg/restore/utils/rewrite_rule_test.go | 18 +- br/pkg/stream/BUILD.bazel | 1 - br/pkg/stream/rewrite_meta_rawkv_test.go | 25 +- br/pkg/task/BUILD.bazel | 4 +- br/pkg/task/config_test.go | 6 +- br/pkg/task/restore.go | 4 +- br/pkg/task/restore_test.go | 22 +- br/pkg/utils/BUILD.bazel | 2 +- br/pkg/utils/schema.go | 12 +- cmd/ddltest/BUILD.bazel | 2 +- cmd/ddltest/ddl_test.go | 4 +- dumpling/export/BUILD.bazel | 1 - dumpling/export/sql.go | 8 +- lightning/pkg/importer/BUILD.bazel | 1 - lightning/pkg/importer/get_pre_info_test.go | 8 +- lightning/pkg/importer/mock/BUILD.bazel | 4 +- lightning/pkg/importer/mock/mock.go | 4 +- lightning/pkg/importer/mock/mock_test.go | 8 +- lightning/pkg/importer/table_import_test.go | 45 +- pkg/autoid_service/BUILD.bazel | 2 +- pkg/autoid_service/autoid_test.go | 10 +- pkg/bindinfo/tests/BUILD.bazel | 2 +- pkg/bindinfo/tests/bind_test.go | 4 +- pkg/ddl/BUILD.bazel | 2 - pkg/ddl/add_column.go | 3 +- pkg/ddl/backfilling.go | 4 +- pkg/ddl/backfilling_dist_scheduler_test.go | 8 +- pkg/ddl/bdr_test.go | 3 +- pkg/ddl/bench_test.go | 6 +- pkg/ddl/column.go | 21 +- pkg/ddl/column_modify_test.go | 18 +- pkg/ddl/column_test.go | 4 +- pkg/ddl/constraint.go | 13 +- pkg/ddl/copr/BUILD.bazel | 4 +- pkg/ddl/copr/copr_ctx.go | 4 +- pkg/ddl/copr/copr_ctx_test.go | 14 +- pkg/ddl/create_table.go | 49 +- pkg/ddl/db_integration_test.go | 42 +- pkg/ddl/db_rename_test.go | 54 +- pkg/ddl/db_table_test.go | 84 +-- pkg/ddl/db_test.go | 7 +- pkg/ddl/ddl_test.go | 3 +- pkg/ddl/executor.go | 183 +++--- pkg/ddl/executor_nokit_test.go | 24 +- pkg/ddl/executor_test.go | 20 +- pkg/ddl/foreign_key.go | 15 +- pkg/ddl/foreign_key_test.go | 26 +- pkg/ddl/generated_column.go | 7 +- pkg/ddl/index.go | 35 +- pkg/ddl/index_cop_test.go | 4 +- pkg/ddl/index_modify_test.go | 26 +- pkg/ddl/modify_column.go | 17 +- pkg/ddl/multi_schema_change.go | 9 +- pkg/ddl/notifier/BUILD.bazel | 4 +- pkg/ddl/notifier/events.go | 10 +- pkg/ddl/notifier/events_test.go | 14 +- pkg/ddl/notifier/store_test.go | 22 +- pkg/ddl/notifier/testkit_test.go | 20 +- pkg/ddl/partition.go | 89 ++- pkg/ddl/partition_test.go | 20 +- pkg/ddl/placement/BUILD.bazel | 2 +- pkg/ddl/placement/meta_bundle_test.go | 36 +- pkg/ddl/placement_policy.go | 5 +- pkg/ddl/placement_policy_ddl_test.go | 8 +- pkg/ddl/placement_policy_test.go | 130 ++-- pkg/ddl/placement_sql_test.go | 32 +- pkg/ddl/primary_key_handle_test.go | 34 +- pkg/ddl/resource_group.go | 7 +- pkg/ddl/resourcegroup/BUILD.bazel | 2 +- pkg/ddl/resourcegroup/group.go | 8 +- pkg/ddl/schema_test.go | 8 +- pkg/ddl/schematracker/BUILD.bazel | 2 - pkg/ddl/schematracker/checker.go | 9 +- pkg/ddl/schematracker/dm_tracker.go | 37 +- pkg/ddl/schematracker/dm_tracker_test.go | 5 +- pkg/ddl/schematracker/info_store.go | 26 +- pkg/ddl/schematracker/info_store_test.go | 22 +- pkg/ddl/table.go | 6 +- pkg/ddl/table_lock.go | 14 +- pkg/ddl/table_split_test.go | 8 +- pkg/ddl/table_test.go | 33 +- pkg/ddl/tests/fail/BUILD.bazel | 2 +- pkg/ddl/tests/fail/fail_db_test.go | 10 +- pkg/ddl/tests/fk/BUILD.bazel | 2 +- pkg/ddl/tests/fk/foreign_key_test.go | 210 +++---- pkg/ddl/tests/multivaluedindex/BUILD.bazel | 2 +- .../multi_valued_index_test.go | 4 +- pkg/ddl/tests/partition/BUILD.bazel | 1 - pkg/ddl/tests/partition/db_partition_test.go | 47 +- pkg/ddl/tests/partition/multi_domain_test.go | 42 +- .../tests/partition/reorg_partition_test.go | 26 +- pkg/ddl/tests/serial/BUILD.bazel | 2 +- pkg/ddl/tests/serial/serial_test.go | 16 +- pkg/ddl/tests/tiflash/BUILD.bazel | 2 +- pkg/ddl/tests/tiflash/ddl_tiflash_test.go | 64 +- pkg/ddl/testutil/BUILD.bazel | 2 +- pkg/ddl/testutil/testutil.go | 6 +- pkg/ddl/tiflash_replica_test.go | 10 +- pkg/ddl/ttl.go | 3 +- pkg/ddl/ttl_test.go | 17 +- pkg/disttask/importinto/BUILD.bazel | 1 - pkg/disttask/importinto/planner_test.go | 4 +- .../importinto/scheduler_testkit_test.go | 6 +- .../importinto/task_executor_testkit_test.go | 4 +- pkg/domain/BUILD.bazel | 2 - pkg/domain/domain_test.go | 3 +- pkg/domain/extract.go | 10 +- pkg/domain/infosync/BUILD.bazel | 2 +- pkg/domain/infosync/info_test.go | 6 +- pkg/domain/plan_replayer_dump.go | 17 +- pkg/domain/ru_stats.go | 4 +- pkg/domain/ru_stats_test.go | 8 +- pkg/domain/test_helper.go | 4 +- pkg/executor/BUILD.bazel | 2 - pkg/executor/adapter.go | 3 +- pkg/executor/admin.go | 3 +- pkg/executor/analyze_test.go | 4 +- pkg/executor/batch_point_get.go | 6 +- pkg/executor/benchmark_test.go | 5 +- pkg/executor/brie.go | 3 +- pkg/executor/brie_utils.go | 8 +- pkg/executor/brie_utils_test.go | 37 +- pkg/executor/builder.go | 9 +- pkg/executor/chunk_size_control_test.go | 6 +- pkg/executor/copr_cache_test.go | 4 +- pkg/executor/ddl.go | 9 +- pkg/executor/distsql_test.go | 6 +- pkg/executor/foreign_key.go | 31 +- pkg/executor/grant.go | 9 +- pkg/executor/historical_stats_test.go | 16 +- pkg/executor/importer/BUILD.bazel | 1 - .../importer/chunk_process_testkit_test.go | 4 +- .../importer/importer_testkit_test.go | 15 +- pkg/executor/importer/precheck_test.go | 4 +- .../importer/table_import_testkit_test.go | 9 +- pkg/executor/infoschema_reader.go | 54 +- .../infoschema_reader_internal_test.go | 38 +- .../internal/calibrateresource/BUILD.bazel | 1 - .../calibrateresource/calibrate_resource.go | 7 +- pkg/executor/internal/exec/BUILD.bazel | 2 +- pkg/executor/internal/exec/indexusage_test.go | 12 +- pkg/executor/internal/querywatch/BUILD.bazel | 1 - .../internal/querywatch/query_watch.go | 9 +- pkg/executor/lockstats/BUILD.bazel | 2 - pkg/executor/lockstats/lock_stats_executor.go | 3 +- .../lockstats/lock_stats_executor_test.go | 29 +- pkg/executor/partition_table_test.go | 4 +- pkg/executor/point_get.go | 8 +- pkg/executor/prepared_test.go | 4 +- pkg/executor/show.go | 29 +- pkg/executor/show_placement.go | 7 +- pkg/executor/show_placement_test.go | 4 +- pkg/executor/show_stats_test.go | 6 +- pkg/executor/simple.go | 9 +- pkg/executor/slow_query_sql_test.go | 4 +- pkg/executor/slow_query_test.go | 4 +- pkg/executor/split.go | 6 +- pkg/executor/split_test.go | 28 +- pkg/executor/stmtsummary_test.go | 8 +- pkg/executor/test/admintest/BUILD.bazel | 2 +- pkg/executor/test/admintest/admin_test.go | 94 +-- pkg/executor/test/analyzetest/BUILD.bazel | 2 +- pkg/executor/test/analyzetest/analyze_test.go | 140 ++--- pkg/executor/test/ddl/BUILD.bazel | 2 +- pkg/executor/test/ddl/ddl_test.go | 30 +- pkg/executor/test/executor/BUILD.bazel | 2 +- pkg/executor/test/executor/executor_test.go | 12 +- pkg/executor/test/fktest/BUILD.bazel | 1 - pkg/executor/test/fktest/foreign_key_test.go | 23 +- pkg/executor/test/seqtest/BUILD.bazel | 1 - .../test/seqtest/seq_executor_test.go | 4 +- pkg/executor/test/showtest/BUILD.bazel | 2 +- pkg/executor/test/showtest/show_test.go | 8 +- pkg/executor/test/simpletest/BUILD.bazel | 2 +- pkg/executor/test/simpletest/simple_test.go | 8 +- pkg/executor/test/writetest/BUILD.bazel | 2 +- pkg/executor/test/writetest/write_test.go | 8 +- pkg/executor/update.go | 4 +- pkg/expression/BUILD.bazel | 2 - pkg/expression/builtin_cast.go | 3 +- pkg/expression/builtin_convert_charset.go | 5 +- pkg/expression/builtin_other.go | 3 +- pkg/expression/builtin_test.go | 3 +- pkg/expression/column_test.go | 7 +- pkg/expression/distsql_builtin.go | 3 +- pkg/expression/evaluator_test.go | 9 +- pkg/expression/expression.go | 17 +- pkg/expression/expression_test.go | 5 +- pkg/expression/helper_test.go | 7 +- pkg/expression/integration_test/BUILD.bazel | 2 +- .../integration_test/integration_test.go | 16 +- pkg/expression/scalar_function.go | 5 +- pkg/expression/sessionexpr/BUILD.bazel | 2 +- pkg/expression/sessionexpr/sessionctx.go | 4 +- pkg/expression/simple_rewriter_test.go | 69 ++- .../test/multivaluedindex/BUILD.bazel | 2 +- .../multi_valued_index_test.go | 14 +- pkg/expression/util_test.go | 3 +- pkg/expression/vs_helper.go | 3 +- pkg/infoschema/BUILD.bazel | 4 +- pkg/infoschema/builder.go | 4 +- pkg/infoschema/context/BUILD.bazel | 2 +- pkg/infoschema/context/infoschema.go | 24 +- pkg/infoschema/infoschema.go | 60 +- pkg/infoschema/infoschema_nokit_test.go | 6 +- pkg/infoschema/infoschema_test.go | 78 +-- pkg/infoschema/infoschema_v2.go | 38 +- pkg/infoschema/infoschema_v2_test.go | 72 +-- pkg/infoschema/infoschemav2_cache_test.go | 12 +- pkg/infoschema/interface.go | 6 +- pkg/infoschema/internal/BUILD.bazel | 2 +- pkg/infoschema/internal/testkit.go | 14 +- pkg/infoschema/metrics_schema.go | 4 +- pkg/infoschema/tables.go | 12 +- .../test/clustertablestest/BUILD.bazel | 2 +- .../clustertablestest/cluster_tables_test.go | 6 +- .../test/clustertablestest/tables_test.go | 14 +- .../test/infoschemav2test/BUILD.bazel | 2 +- .../test/infoschemav2test/v2_test.go | 24 +- pkg/lightning/backend/kv/BUILD.bazel | 1 - pkg/lightning/backend/kv/base_test.go | 4 +- pkg/lightning/backend/kv/sql2kv_test.go | 15 +- pkg/lightning/backend/local/BUILD.bazel | 1 - pkg/lightning/backend/local/checksum_test.go | 6 +- pkg/lightning/backend/tidb/BUILD.bazel | 4 +- pkg/lightning/backend/tidb/tidb.go | 8 +- pkg/lightning/backend/tidb/tidb_test.go | 26 +- pkg/lightning/checkpoints/BUILD.bazel | 2 +- .../checkpoints/checkpoints_file_test.go | 6 +- .../checkpoints/checkpoints_sql_test.go | 18 +- pkg/lightning/errormanager/BUILD.bazel | 1 - .../errormanager/errormanager_test.go | 30 +- pkg/lightning/mydump/BUILD.bazel | 1 - pkg/lightning/mydump/schema_import.go | 17 +- pkg/lock/BUILD.bazel | 2 +- pkg/lock/context/BUILD.bazel | 2 +- pkg/lock/context/lockcontext.go | 4 +- pkg/lock/lock.go | 18 +- pkg/meta/BUILD.bazel | 3 +- pkg/meta/autoid/BUILD.bazel | 2 +- pkg/meta/autoid/autoid_test.go | 44 +- pkg/meta/autoid/bench_test.go | 10 +- pkg/meta/autoid/seq_autoid_test.go | 10 +- pkg/meta/meta.go | 10 +- pkg/meta/meta_test.go | 77 ++- pkg/meta/model/BUILD.bazel | 2 - pkg/meta/model/column.go | 16 +- pkg/meta/model/column_test.go | 12 +- pkg/meta/model/db.go | 12 +- pkg/meta/model/index.go | 15 +- pkg/meta/model/index_test.go | 26 +- pkg/meta/model/job.go | 22 +- pkg/meta/model/job_args.go | 71 ++- pkg/meta/model/job_args_test.go | 85 ++- pkg/meta/model/job_test.go | 8 +- pkg/meta/model/placement.go | 8 +- pkg/meta/model/placement_test.go | 6 +- pkg/meta/model/resource_group.go | 26 +- pkg/meta/model/table.go | 140 ++--- pkg/meta/model/table_test.go | 40 +- pkg/parser/BUILD.bazel | 2 - pkg/parser/ast/BUILD.bazel | 3 +- pkg/parser/ast/ddl.go | 133 +++-- pkg/parser/ast/dml.go | 35 +- pkg/parser/ast/expressions.go | 7 +- pkg/parser/ast/functions.go | 5 +- pkg/parser/ast/misc.go | 37 +- pkg/parser/{model => ast}/model.go | 2 +- pkg/parser/{model => ast}/model_test.go | 2 +- pkg/parser/ast/stats.go | 17 +- pkg/parser/hintparser.go | 83 ++- pkg/parser/hintparser.y | 83 ++- pkg/parser/hintparser_test.go | 157 +++-- pkg/parser/model/BUILD.bazel | 21 - pkg/parser/parser.go | 543 +++++++++-------- pkg/parser/parser.y | 553 +++++++++--------- pkg/parser/parser_test.go | 39 +- pkg/planner/BUILD.bazel | 1 - pkg/planner/cardinality/BUILD.bazel | 2 +- pkg/planner/cardinality/row_size_test.go | 4 +- pkg/planner/cardinality/selectivity_test.go | 30 +- pkg/planner/cardinality/trace_test.go | 4 +- pkg/planner/cascades/memo/BUILD.bazel | 2 +- pkg/planner/cascades/memo/memo_test.go | 10 +- pkg/planner/cascades/rule/BUILD.bazel | 2 +- pkg/planner/cascades/rule/binder_test.go | 26 +- pkg/planner/core/BUILD.bazel | 2 - pkg/planner/core/casetest/BUILD.bazel | 2 +- pkg/planner/core/casetest/cbotest/BUILD.bazel | 2 +- pkg/planner/core/casetest/cbotest/cbo_test.go | 4 +- .../core/casetest/enforcempp/BUILD.bazel | 2 +- .../casetest/enforcempp/enforce_mpp_test.go | 14 +- pkg/planner/core/casetest/hint/BUILD.bazel | 2 +- pkg/planner/core/casetest/hint/hint_test.go | 6 +- pkg/planner/core/casetest/index/BUILD.bazel | 2 +- pkg/planner/core/casetest/index/index_test.go | 4 +- pkg/planner/core/casetest/integration_test.go | 18 +- pkg/planner/core/casetest/mpp/BUILD.bazel | 2 +- pkg/planner/core/casetest/mpp/mpp_test.go | 16 +- pkg/planner/core/casetest/plan_test.go | 4 +- .../core/casetest/planstats/BUILD.bazel | 2 +- .../casetest/planstats/plan_stats_test.go | 8 +- .../core/casetest/pushdown/BUILD.bazel | 2 +- .../core/casetest/pushdown/push_down_test.go | 10 +- .../core/casetest/vectorsearch/BUILD.bazel | 2 +- .../vectorsearch/vector_index_test.go | 4 +- .../core/collect_column_stats_usage_test.go | 4 +- pkg/planner/core/common_plans.go | 7 +- pkg/planner/core/enforce_mpp_test.go | 4 +- .../core/exhaust_physical_plans_test.go | 55 +- pkg/planner/core/expression_codec_fn.go | 4 +- pkg/planner/core/expression_rewriter.go | 3 +- pkg/planner/core/expression_test.go | 9 +- pkg/planner/core/foreign_key.go | 22 +- pkg/planner/core/fragment.go | 4 +- pkg/planner/core/hint_utils.go | 59 +- pkg/planner/core/indexmerge_path_test.go | 5 +- pkg/planner/core/integration_test.go | 72 +-- pkg/planner/core/logical_plan_builder.go | 97 ++- pkg/planner/core/logical_plans_test.go | 11 +- .../core/memtable_infoschema_extractor.go | 60 +- pkg/planner/core/mock.go | 180 +++--- .../core/operator/logicalop/BUILD.bazel | 1 - .../core/operator/logicalop/logical_cte.go | 6 +- .../operator/logicalop/logical_datasource.go | 9 +- .../operator/logicalop/logical_mem_table.go | 4 +- .../core/operator/logicalop/logical_show.go | 5 +- .../logicalop/logicalop_test/BUILD.bazel | 1 - .../logicalop_test/hash64_equals_test.go | 5 +- pkg/planner/core/optimizer.go | 3 +- pkg/planner/core/partition_prune.go | 12 +- pkg/planner/core/partition_pruning_test.go | 5 +- pkg/planner/core/physical_plans.go | 17 +- pkg/planner/core/plan_cache_utils.go | 5 +- .../core/plan_cacheable_checker_test.go | 27 +- pkg/planner/core/plan_cost_ver1_test.go | 4 +- pkg/planner/core/plan_test.go | 3 +- pkg/planner/core/planbuilder.go | 77 ++- pkg/planner/core/planbuilder_test.go | 35 +- pkg/planner/core/point_get_plan.go | 25 +- pkg/planner/core/preprocess.go | 23 +- pkg/planner/core/resolve/BUILD.bazel | 1 - pkg/planner/core/resolve/result.go | 8 +- .../rule_generate_column_substitute_test.go | 4 +- pkg/planner/core/rule_join_reorder_dp_test.go | 3 +- pkg/planner/core/rule_partition_processor.go | 37 +- .../core/runtime_filter_generator_test.go | 6 +- pkg/planner/core/util_test.go | 117 ++-- pkg/planner/indexadvisor/BUILD.bazel | 1 - pkg/planner/indexadvisor/optimizer.go | 21 +- pkg/planner/optimize.go | 5 +- pkg/planner/util/BUILD.bazel | 1 - pkg/planner/util/misc.go | 7 +- pkg/resourcegroup/runaway/BUILD.bazel | 2 +- pkg/resourcegroup/runaway/record.go | 8 +- pkg/resourcegroup/tests/BUILD.bazel | 2 +- .../tests/resource_group_test.go | 20 +- pkg/server/BUILD.bazel | 2 +- pkg/server/driver_tidb_test.go | 26 +- pkg/server/handler/BUILD.bazel | 2 +- pkg/server/handler/optimizor/BUILD.bazel | 4 +- pkg/server/handler/optimizor/plan_replayer.go | 4 +- .../handler/optimizor/plan_replayer_test.go | 4 +- .../handler/optimizor/statistics_handler.go | 8 +- .../optimizor/statistics_handler_test.go | 4 +- pkg/server/handler/tests/BUILD.bazel | 2 +- .../handler/tests/http_handler_serial_test.go | 4 +- pkg/server/handler/tests/http_handler_test.go | 8 +- pkg/server/handler/tikv_handler.go | 4 +- pkg/server/handler/tikvhandler/BUILD.bazel | 2 +- .../handler/tikvhandler/tikv_handler.go | 18 +- pkg/session/BUILD.bazel | 1 - pkg/session/bootstrap.go | 6 +- pkg/session/nontransactional.go | 15 +- pkg/session/schematest/BUILD.bazel | 2 +- pkg/session/schematest/schema_test.go | 6 +- pkg/session/session.go | 17 +- pkg/session/test/BUILD.bazel | 1 - pkg/session/test/common/BUILD.bazel | 2 +- pkg/session/test/common/common_test.go | 4 +- pkg/session/test/session_test.go | 3 +- pkg/sessionctx/variable/BUILD.bazel | 1 - pkg/sessionctx/variable/sysvar.go | 7 +- pkg/sessiontxn/BUILD.bazel | 1 - pkg/sessiontxn/txn_manager_test.go | 3 +- pkg/statistics/BUILD.bazel | 2 +- pkg/statistics/handle/autoanalyze/BUILD.bazel | 4 +- .../handle/autoanalyze/autoanalyze.go | 4 +- .../handle/autoanalyze/autoanalyze_test.go | 10 +- .../handle/autoanalyze/exec/BUILD.bazel | 2 +- .../handle/autoanalyze/exec/exec_test.go | 4 +- .../autoanalyze/priorityqueue/BUILD.bazel | 2 +- .../analysis_job_factory_test.go | 26 +- ...mic_partitioned_table_analysis_job_test.go | 16 +- ...non_partitioned_table_analysis_job_test.go | 16 +- .../priorityqueue/queue_ddl_handler_test.go | 28 +- .../autoanalyze/priorityqueue/queue_test.go | 30 +- ...tic_partitioned_table_analysis_job_test.go | 16 +- .../handle/autoanalyze/refresher/BUILD.bazel | 2 +- .../autoanalyze/refresher/refresher_test.go | 20 +- pkg/statistics/handle/ddl/BUILD.bazel | 2 +- pkg/statistics/handle/ddl/ddl_test.go | 82 +-- pkg/statistics/handle/globalstats/BUILD.bazel | 2 +- .../handle/globalstats/global_stats_test.go | 6 +- pkg/statistics/handle/handletest/BUILD.bazel | 2 +- .../handle/handletest/analyze/BUILD.bazel | 2 +- .../handle/handletest/analyze/analyze_test.go | 4 +- .../handle/handletest/handle_test.go | 44 +- .../handle/handletest/initstats/BUILD.bazel | 2 +- .../handletest/initstats/load_stats_test.go | 6 +- .../handle/handletest/lockstats/BUILD.bazel | 2 +- .../lockstats/lock_partition_stats_test.go | 4 +- .../lockstats/lock_table_stats_test.go | 8 +- .../handle/handletest/statstest/BUILD.bazel | 2 +- .../handle/handletest/statstest/stats_test.go | 18 +- pkg/statistics/handle/storage/BUILD.bazel | 3 +- pkg/statistics/handle/storage/dump_test.go | 32 +- pkg/statistics/handle/storage/gc_test.go | 4 +- pkg/statistics/handle/storage/read_test.go | 4 +- .../handle/storage/stats_read_writer.go | 4 +- .../handle/storage/stats_read_writer_test.go | 4 +- pkg/statistics/handle/syncload/BUILD.bazel | 2 +- .../handle/syncload/stats_syncload_test.go | 16 +- pkg/statistics/handle/updatetest/BUILD.bazel | 2 +- .../handle/updatetest/update_test.go | 38 +- pkg/statistics/handle/usage/BUILD.bazel | 2 +- .../usage/index_usage_integration_test.go | 4 +- .../handle/usage/predicate_column_test.go | 4 +- pkg/statistics/histogram_test.go | 4 +- pkg/statistics/integration_test.go | 18 +- pkg/store/helper/BUILD.bazel | 2 +- pkg/store/helper/helper_test.go | 6 +- pkg/store/mockstore/mockcopr/BUILD.bazel | 2 +- pkg/store/mockstore/mockcopr/executor_test.go | 4 +- pkg/table/BUILD.bazel | 2 - pkg/table/column_test.go | 3 +- pkg/table/constraint.go | 13 +- pkg/table/table.go | 4 +- pkg/table/tables/BUILD.bazel | 2 - pkg/table/tables/bench_test.go | 8 +- pkg/table/tables/cache_test.go | 6 +- pkg/table/tables/index_test.go | 5 +- pkg/table/tables/mutation_checker_test.go | 10 +- pkg/table/tables/partition.go | 43 +- pkg/table/tables/tables_test.go | 26 +- pkg/table/tables/test/partition/BUILD.bazel | 2 +- .../tables/test/partition/partition_test.go | 30 +- pkg/table/temptable/BUILD.bazel | 3 +- pkg/table/temptable/ddl.go | 11 +- pkg/table/temptable/ddl_test.go | 42 +- pkg/table/temptable/main_test.go | 6 +- pkg/testkit/BUILD.bazel | 1 - pkg/testkit/external/BUILD.bazel | 2 +- pkg/testkit/external/util.go | 6 +- pkg/testkit/mockstore.go | 4 +- pkg/ttl/cache/BUILD.bazel | 2 - pkg/ttl/cache/infoschema.go | 6 +- pkg/ttl/cache/split_test.go | 6 +- pkg/ttl/cache/table.go | 11 +- pkg/ttl/cache/table_test.go | 15 +- pkg/ttl/sqlbuilder/BUILD.bazel | 1 - pkg/ttl/sqlbuilder/sql_test.go | 75 ++- pkg/ttl/ttlworker/BUILD.bazel | 2 - .../ttlworker/job_manager_integration_test.go | 51 +- pkg/ttl/ttlworker/job_manager_test.go | 3 +- pkg/ttl/ttlworker/session_test.go | 21 +- .../task_manager_integration_test.go | 18 +- pkg/ttl/ttlworker/timer_sync.go | 10 +- pkg/ttl/ttlworker/timer_sync_test.go | 8 +- pkg/types/BUILD.bazel | 1 - pkg/types/field_name.go | 11 +- pkg/util/BUILD.bazel | 4 +- pkg/util/admin/BUILD.bazel | 2 +- pkg/util/admin/admin.go | 4 +- pkg/util/dbutil/dbutiltest/BUILD.bazel | 1 - pkg/util/dbutil/dbutiltest/utils.go | 5 +- pkg/util/deadlockhistory/BUILD.bazel | 2 +- .../deadlockhistory/deadlock_history_test.go | 20 +- pkg/util/hint/BUILD.bazel | 1 - pkg/util/hint/hint.go | 35 +- pkg/util/hint/hint_processor.go | 7 +- pkg/util/hint/hint_query_block.go | 17 +- pkg/util/keydecoder/BUILD.bazel | 2 +- pkg/util/keydecoder/keydecoder_test.go | 20 +- pkg/util/misc.go | 10 +- pkg/util/misc_test.go | 6 +- pkg/util/mock/BUILD.bazel | 1 - pkg/util/mock/context.go | 5 +- pkg/util/ranger/BUILD.bazel | 1 - pkg/util/ranger/ranger_test.go | 9 +- pkg/util/rowDecoder/BUILD.bazel | 2 +- pkg/util/rowDecoder/decoder_test.go | 28 +- pkg/util/schemacmp/BUILD.bazel | 2 +- pkg/util/schemacmp/table.go | 6 +- pkg/util/stmtsummary/BUILD.bazel | 2 +- .../stmtsummary/statement_summary_test.go | 4 +- pkg/util/stmtsummary/v2/BUILD.bazel | 2 +- pkg/util/stmtsummary/v2/column_test.go | 28 +- pkg/util/stmtsummary/v2/reader_test.go | 14 +- pkg/util/workloadrepo/BUILD.bazel | 4 +- pkg/util/workloadrepo/const.go | 4 +- pkg/util/workloadrepo/housekeeper.go | 6 +- pkg/util/workloadrepo/table.go | 10 +- pkg/util/workloadrepo/worker_test.go | 8 +- tests/realtikvtest/addindextest3/BUILD.bazel | 2 +- .../addindextest3/functional_test.go | 6 +- .../addindextest3/operator_test.go | 4 +- .../realtikvtest/importintotest2/BUILD.bazel | 2 +- .../write_after_import_test.go | 6 +- .../realtikvtest/pessimistictest/BUILD.bazel | 2 +- .../pessimistictest/pessimistic_test.go | 4 +- tests/realtikvtest/sessiontest/BUILD.bazel | 2 +- .../sessiontest/infoschema_v2_test.go | 14 +- tests/realtikvtest/statisticstest/BUILD.bazel | 2 +- .../statisticstest/statistics_test.go | 4 +- 549 files changed, 4451 insertions(+), 4659 deletions(-) rename pkg/parser/{model => ast}/model.go (99%) rename pkg/parser/{model => ast}/model_test.go (98%) delete mode 100644 pkg/parser/model/BUILD.bazel diff --git a/br/pkg/checkpoint/BUILD.bazel b/br/pkg/checkpoint/BUILD.bazel index dc3471726625b..d31e5e01fe19d 100644 --- a/br/pkg/checkpoint/BUILD.bazel +++ b/br/pkg/checkpoint/BUILD.bazel @@ -25,7 +25,7 @@ go_library( "//pkg/domain", "//pkg/kv", "//pkg/meta/model", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/util", "//pkg/util/sqlexec", "@com_github_google_uuid//:uuid", @@ -53,7 +53,7 @@ go_test( "//br/pkg/storage", "//br/pkg/utiltest", "//pkg/meta/model", - "//pkg/parser/model", + "//pkg/parser/ast", "@com_github_pingcap_failpoint//:failpoint", "@com_github_pingcap_kvproto//pkg/brpb", "@com_github_pingcap_kvproto//pkg/encryptionpb", diff --git a/br/pkg/checkpoint/checkpoint_test.go b/br/pkg/checkpoint/checkpoint_test.go index b70348aaa5fd9..f2534f26c5f52 100644 --- a/br/pkg/checkpoint/checkpoint_test.go +++ b/br/pkg/checkpoint/checkpoint_test.go @@ -31,7 +31,7 @@ import ( "github.com/pingcap/tidb/br/pkg/storage" "github.com/pingcap/tidb/br/pkg/utiltest" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/stretchr/testify/require" "github.com/tikv/client-go/v2/oracle" ) @@ -128,8 +128,8 @@ func TestCheckpointMetaForRestore(t *testing.T) { SQLs: []checkpoint.CheckpointIngestIndexRepairSQL{ { IndexID: 1, - SchemaName: pmodel.NewCIStr("2"), - TableName: pmodel.NewCIStr("3"), + SchemaName: ast.NewCIStr("2"), + TableName: ast.NewCIStr("3"), IndexName: "4", AddSQL: "5", AddArgs: []any{"6", "7", "8"}, @@ -140,8 +140,8 @@ func TestCheckpointMetaForRestore(t *testing.T) { repairSQLs, err := checkpoint.LoadCheckpointIngestIndexRepairSQLs(ctx, se.GetSessionCtx().GetRestrictedSQLExecutor()) require.NoError(t, err) require.Equal(t, repairSQLs.SQLs[0].IndexID, int64(1)) - require.Equal(t, repairSQLs.SQLs[0].SchemaName, pmodel.NewCIStr("2")) - require.Equal(t, repairSQLs.SQLs[0].TableName, pmodel.NewCIStr("3")) + require.Equal(t, repairSQLs.SQLs[0].SchemaName, ast.NewCIStr("2")) + require.Equal(t, repairSQLs.SQLs[0].TableName, ast.NewCIStr("3")) require.Equal(t, repairSQLs.SQLs[0].IndexName, "4") require.Equal(t, repairSQLs.SQLs[0].AddSQL, "5") require.Equal(t, repairSQLs.SQLs[0].AddArgs, []any{"6", "7", "8"}) @@ -360,7 +360,7 @@ func TestCheckpointRestoreRunner(t *testing.T) { exists := checkpoint.ExistsSstRestoreCheckpoint(ctx, s.Mock.Domain, checkpoint.SnapshotRestoreCheckpointDatabaseName) require.False(t, exists) - exists = s.Mock.Domain.InfoSchema().SchemaExists(pmodel.NewCIStr(checkpoint.SnapshotRestoreCheckpointDatabaseName)) + exists = s.Mock.Domain.InfoSchema().SchemaExists(ast.NewCIStr(checkpoint.SnapshotRestoreCheckpointDatabaseName)) require.False(t, exists) } @@ -545,7 +545,7 @@ func TestCheckpointLogRestoreRunner(t *testing.T) { exists := checkpoint.ExistsLogRestoreCheckpointMetadata(ctx, s.Mock.Domain) require.False(t, exists) - exists = s.Mock.Domain.InfoSchema().SchemaExists(pmodel.NewCIStr(checkpoint.LogRestoreCheckpointDatabaseName)) + exists = s.Mock.Domain.InfoSchema().SchemaExists(ast.NewCIStr(checkpoint.LogRestoreCheckpointDatabaseName)) require.False(t, exists) } @@ -640,6 +640,6 @@ func TestCheckpointCompactedRestoreRunner(t *testing.T) { exists = checkpoint.ExistsSstRestoreCheckpoint(ctx, s.Mock.Domain, checkpoint.CustomSSTRestoreCheckpointDatabaseName) require.False(t, exists) - exists = s.Mock.Domain.InfoSchema().SchemaExists(pmodel.NewCIStr(checkpoint.CustomSSTRestoreCheckpointDatabaseName)) + exists = s.Mock.Domain.InfoSchema().SchemaExists(ast.NewCIStr(checkpoint.CustomSSTRestoreCheckpointDatabaseName)) require.False(t, exists) } diff --git a/br/pkg/checkpoint/log_restore.go b/br/pkg/checkpoint/log_restore.go index 0fd046b67ad7c..15f2f995aec5c 100644 --- a/br/pkg/checkpoint/log_restore.go +++ b/br/pkg/checkpoint/log_restore.go @@ -23,7 +23,7 @@ import ( "github.com/pingcap/tidb/br/pkg/glue" "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/util/sqlexec" ) @@ -191,7 +191,7 @@ func ExistsLogRestoreCheckpointMetadata( dom *domain.Domain, ) bool { return dom.InfoSchema(). - TableExists(pmodel.NewCIStr(LogRestoreCheckpointDatabaseName), pmodel.NewCIStr(checkpointMetaTableName)) + TableExists(ast.NewCIStr(LogRestoreCheckpointDatabaseName), ast.NewCIStr(checkpointMetaTableName)) } // A progress type for snapshot + log restore. @@ -251,7 +251,7 @@ func ExistsCheckpointProgress( dom *domain.Domain, ) bool { return dom.InfoSchema(). - TableExists(pmodel.NewCIStr(LogRestoreCheckpointDatabaseName), pmodel.NewCIStr(checkpointProgressTableName)) + TableExists(ast.NewCIStr(LogRestoreCheckpointDatabaseName), ast.NewCIStr(checkpointProgressTableName)) } // CheckpointTaskInfo is unique information within the same cluster id. It represents the last @@ -298,12 +298,12 @@ func TryToGetCheckpointTaskInfo( } type CheckpointIngestIndexRepairSQL struct { - IndexID int64 `json:"index-id"` - SchemaName pmodel.CIStr `json:"schema-name"` - TableName pmodel.CIStr `json:"table-name"` - IndexName string `json:"index-name"` - AddSQL string `json:"add-sql"` - AddArgs []any `json:"add-args"` + IndexID int64 `json:"index-id"` + SchemaName ast.CIStr `json:"schema-name"` + TableName ast.CIStr `json:"table-name"` + IndexName string `json:"index-name"` + AddSQL string `json:"add-sql"` + AddArgs []any `json:"add-args"` } type CheckpointIngestIndexRepairSQLs struct { @@ -321,7 +321,7 @@ func LoadCheckpointIngestIndexRepairSQLs( func ExistsCheckpointIngestIndexRepairSQLs(ctx context.Context, dom *domain.Domain) bool { return dom.InfoSchema(). - TableExists(pmodel.NewCIStr(LogRestoreCheckpointDatabaseName), pmodel.NewCIStr(checkpointIngestTableName)) + TableExists(ast.NewCIStr(LogRestoreCheckpointDatabaseName), ast.NewCIStr(checkpointIngestTableName)) } func SaveCheckpointIngestIndexRepairSQLs( diff --git a/br/pkg/checkpoint/restore.go b/br/pkg/checkpoint/restore.go index 2e55cc3eb81c2..32ee7e02d4392 100644 --- a/br/pkg/checkpoint/restore.go +++ b/br/pkg/checkpoint/restore.go @@ -23,7 +23,7 @@ import ( "github.com/pingcap/tidb/br/pkg/glue" "github.com/pingcap/tidb/br/pkg/pdutil" "github.com/pingcap/tidb/pkg/domain" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/util/sqlexec" ) @@ -173,7 +173,7 @@ func ExistsSstRestoreCheckpoint( // we only check the existence of the checkpoint data table // because the checkpoint metadata is not used for restore return dom.InfoSchema(). - TableExists(pmodel.NewCIStr(dbName), pmodel.NewCIStr(checkpointDataTableName)) + TableExists(ast.NewCIStr(dbName), ast.NewCIStr(checkpointDataTableName)) } func RemoveCheckpointDataForSstRestore(ctx context.Context, dom *domain.Domain, se glue.Session, dbName string) error { diff --git a/br/pkg/checkpoint/storage.go b/br/pkg/checkpoint/storage.go index 4a37a14b0da12..c68d9b41ff506 100644 --- a/br/pkg/checkpoint/storage.go +++ b/br/pkg/checkpoint/storage.go @@ -27,7 +27,7 @@ import ( "github.com/pingcap/tidb/br/pkg/glue" "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/kv" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/util/sqlexec" "go.uber.org/zap" ) @@ -89,7 +89,7 @@ const ( ) // IsCheckpointDB checks whether the dbname is checkpoint database. -func IsCheckpointDB(dbname pmodel.CIStr) bool { +func IsCheckpointDB(dbname ast.CIStr) bool { return dbname.O == LogRestoreCheckpointDatabaseName || dbname.O == SnapshotRestoreCheckpointDatabaseName || dbname.O == CustomSSTRestoreCheckpointDatabaseName @@ -333,7 +333,7 @@ func dropCheckpointTables( } } // check if any user table is created in the checkpoint database - tables, err := dom.InfoSchema().SchemaTableInfos(ctx, pmodel.NewCIStr(dbName)) + tables, err := dom.InfoSchema().SchemaTableInfos(ctx, ast.NewCIStr(dbName)) if err != nil { return errors.Trace(err) } diff --git a/br/pkg/checksum/BUILD.bazel b/br/pkg/checksum/BUILD.bazel index 3835e6ad0c8c7..035819c5ab724 100644 --- a/br/pkg/checksum/BUILD.bazel +++ b/br/pkg/checksum/BUILD.bazel @@ -45,7 +45,7 @@ go_test( "//pkg/distsql", "//pkg/kv", "//pkg/meta/model", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/sessionctx/variable", "//pkg/testkit", "//pkg/testkit/testsetup", diff --git a/br/pkg/checksum/executor_test.go b/br/pkg/checksum/executor_test.go index 06d4d1f3ac551..93483664ea8fa 100644 --- a/br/pkg/checksum/executor_test.go +++ b/br/pkg/checksum/executor_test.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" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/testkit" "github.com/stretchr/testify/require" @@ -23,8 +23,8 @@ import ( func getTableInfo(t *testing.T, mock *mock.Cluster, db, table string) *model.TableInfo { info, err := mock.Domain.GetSnapshotInfoSchema(math.MaxUint64) require.NoError(t, err) - cDBName := pmodel.NewCIStr(db) - cTableName := pmodel.NewCIStr(table) + cDBName := ast.NewCIStr(db) + cTableName := ast.NewCIStr(table) tableInfo, err := info.TableByName(context.Background(), cDBName, cTableName) require.NoError(t, err) return tableInfo.Meta() diff --git a/br/pkg/glue/BUILD.bazel b/br/pkg/glue/BUILD.bazel index 819414ff70eac..8bcb4e9db0c76 100644 --- a/br/pkg/glue/BUILD.bazel +++ b/br/pkg/glue/BUILD.bazel @@ -15,7 +15,7 @@ go_library( "//pkg/domain", "//pkg/kv", "//pkg/meta/model", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/sessionctx", "@com_github_fatih_color//:color", "@com_github_pingcap_log//:log", diff --git a/br/pkg/glue/glue.go b/br/pkg/glue/glue.go index d691d638169ab..751a2acb10164 100644 --- a/br/pkg/glue/glue.go +++ b/br/pkg/glue/glue.go @@ -9,7 +9,7 @@ import ( "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/sessionctx" pd "github.com/tikv/pd/client" ) @@ -54,7 +54,7 @@ type Session interface { Execute(ctx context.Context, sql string) error ExecuteInternal(ctx context.Context, sql string, args ...any) error CreateDatabase(ctx context.Context, schema *model.DBInfo) error - CreateTable(ctx context.Context, dbName pmodel.CIStr, table *model.TableInfo, + CreateTable(ctx context.Context, dbName ast.CIStr, table *model.TableInfo, cs ...ddl.CreateTableOption) error CreatePlacementPolicy(ctx context.Context, policy *model.PolicyInfo) error Close() diff --git a/br/pkg/gluetidb/BUILD.bazel b/br/pkg/gluetidb/BUILD.bazel index 66498201712b7..a90cd03d814db 100644 --- a/br/pkg/gluetidb/BUILD.bazel +++ b/br/pkg/gluetidb/BUILD.bazel @@ -15,7 +15,7 @@ go_library( "//pkg/executor", "//pkg/kv", "//pkg/meta/model", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/session", "//pkg/session/types", "//pkg/sessionctx", @@ -35,7 +35,7 @@ go_test( deps = [ "//br/pkg/glue", "//pkg/meta/model", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/session", "//pkg/testkit", "//pkg/types", diff --git a/br/pkg/gluetidb/glue.go b/br/pkg/gluetidb/glue.go index 9514ae4f5f7a1..e81752c0c9079 100644 --- a/br/pkg/gluetidb/glue.go +++ b/br/pkg/gluetidb/glue.go @@ -18,7 +18,7 @@ import ( "github.com/pingcap/tidb/pkg/executor" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/session" sessiontypes "github.com/pingcap/tidb/pkg/session/types" "github.com/pingcap/tidb/pkg/sessionctx" @@ -250,7 +250,7 @@ func (gs *tidbSession) CreateTables(_ context.Context, } // CreateTable implements glue.Session. -func (gs *tidbSession) CreateTable(_ context.Context, dbName pmodel.CIStr, +func (gs *tidbSession) CreateTable(_ context.Context, dbName ast.CIStr, table *model.TableInfo, cs ...ddl.CreateTableOption) error { return errors.Trace(executor.BRIECreateTable(gs.se, dbName, table, brComment, cs...)) } diff --git a/br/pkg/gluetidb/glue_test.go b/br/pkg/gluetidb/glue_test.go index ceb43566f503b..2959a2497beff 100644 --- a/br/pkg/gluetidb/glue_test.go +++ b/br/pkg/gluetidb/glue_test.go @@ -20,7 +20,7 @@ import ( "github.com/pingcap/tidb/br/pkg/glue" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/session" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/types" @@ -45,7 +45,7 @@ func TestTheSessionIsoation(t *testing.T) { }) require.NoError(t, glueSe.CreateDatabase(ctx, &model.DBInfo{ - Name: pmodel.NewCIStr("test_db"), + Name: ast.NewCIStr("test_db"), })) tk := testkit.NewTestKit(t, store) tk.MustExec("use test_db") @@ -54,27 +54,27 @@ func TestTheSessionIsoation(t *testing.T) { req.NoError(glueSe.ExecuteInternal(ctx, "use test;")) infos := []*model.TableInfo{} infos = append(infos, &model.TableInfo{ - Name: pmodel.NewCIStr("tables_1"), + Name: ast.NewCIStr("tables_1"), Columns: []*model.ColumnInfo{ - {Name: pmodel.NewCIStr("foo"), FieldType: *types.NewFieldType(types.KindBinaryLiteral), State: model.StatePublic}, + {Name: ast.NewCIStr("foo"), FieldType: *types.NewFieldType(types.KindBinaryLiteral), State: model.StatePublic}, }, }) infos = append(infos, &model.TableInfo{ - Name: pmodel.NewCIStr("tables_2"), + Name: ast.NewCIStr("tables_2"), PlacementPolicyRef: &model.PolicyRefInfo{ - Name: pmodel.NewCIStr("threereplication"), + Name: ast.NewCIStr("threereplication"), }, Columns: []*model.ColumnInfo{ - {Name: pmodel.NewCIStr("foo"), FieldType: *types.NewFieldType(types.KindBinaryLiteral), State: model.StatePublic}, + {Name: ast.NewCIStr("foo"), FieldType: *types.NewFieldType(types.KindBinaryLiteral), State: model.StatePublic}, }, }) infos = append(infos, &model.TableInfo{ - Name: pmodel.NewCIStr("tables_3"), + Name: ast.NewCIStr("tables_3"), PlacementPolicyRef: &model.PolicyRefInfo{ - Name: pmodel.NewCIStr("fivereplication"), + Name: ast.NewCIStr("fivereplication"), }, Columns: []*model.ColumnInfo{ - {Name: pmodel.NewCIStr("foo"), FieldType: *types.NewFieldType(types.KindBinaryLiteral), State: model.StatePublic}, + {Name: ast.NewCIStr("foo"), FieldType: *types.NewFieldType(types.KindBinaryLiteral), State: model.StatePublic}, }, }) polices := []*model.PolicyInfo{ @@ -82,13 +82,13 @@ func TestTheSessionIsoation(t *testing.T) { PlacementSettings: &model.PlacementSettings{ Followers: 4, }, - Name: pmodel.NewCIStr("fivereplication"), + Name: ast.NewCIStr("fivereplication"), }, { PlacementSettings: &model.PlacementSettings{ Followers: 2, }, - Name: pmodel.NewCIStr("threereplication"), + Name: ast.NewCIStr("threereplication"), }, } for _, pinfo := range polices { diff --git a/br/pkg/gluetidb/mock/BUILD.bazel b/br/pkg/gluetidb/mock/BUILD.bazel index e8a640a5839b1..553d6d07d7f3b 100644 --- a/br/pkg/gluetidb/mock/BUILD.bazel +++ b/br/pkg/gluetidb/mock/BUILD.bazel @@ -11,7 +11,7 @@ go_library( "//pkg/domain", "//pkg/kv", "//pkg/meta/model", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/session/types", "//pkg/sessionctx", "@com_github_tikv_pd_client//:client", diff --git a/br/pkg/gluetidb/mock/mock.go b/br/pkg/gluetidb/mock/mock.go index 6b4c792d69710..6eba84fb6e153 100644 --- a/br/pkg/gluetidb/mock/mock.go +++ b/br/pkg/gluetidb/mock/mock.go @@ -23,7 +23,7 @@ import ( "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" sessiontypes "github.com/pingcap/tidb/pkg/session/types" "github.com/pingcap/tidb/pkg/sessionctx" pd "github.com/tikv/pd/client" @@ -86,7 +86,7 @@ func (*mockSession) CreateTables(_ context.Context, _ map[string][]*model.TableI } // CreateTable implements glue.Session. -func (*mockSession) CreateTable(_ context.Context, _ pmodel.CIStr, +func (*mockSession) CreateTable(_ context.Context, _ ast.CIStr, _ *model.TableInfo, _ ...ddl.CreateTableOption) error { log.Fatal("unimplemented CreateDatabase for mock session") return nil diff --git a/br/pkg/metautil/BUILD.bazel b/br/pkg/metautil/BUILD.bazel index 9aecac3fc6b75..ca54a2ba4cfbf 100644 --- a/br/pkg/metautil/BUILD.bazel +++ b/br/pkg/metautil/BUILD.bazel @@ -52,7 +52,7 @@ go_test( "//br/pkg/storage", "//br/pkg/utils", "//pkg/meta/model", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/statistics/handle/types", "//pkg/statistics/util", "//pkg/tablecodec", diff --git a/br/pkg/metautil/load_test.go b/br/pkg/metautil/load_test.go index d78ad6c3b4cc3..003d77711844d 100644 --- a/br/pkg/metautil/load_test.go +++ b/br/pkg/metautil/load_test.go @@ -24,7 +24,7 @@ import ( "github.com/pingcap/kvproto/pkg/encryptionpb" "github.com/pingcap/tidb/br/pkg/storage" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/statistics/util" "github.com/pingcap/tidb/pkg/tablecodec" "github.com/stretchr/testify/require" @@ -42,8 +42,8 @@ func TestLoadBackupMeta(t *testing.T) { store, err := storage.NewLocalStorage(testDir) require.NoError(t, err) - tblName := pmodel.NewCIStr("t1") - dbName := pmodel.NewCIStr("test") + tblName := ast.NewCIStr("t1") + dbName := ast.NewCIStr("test") tblID := int64(123) mockTbl := &model.TableInfo{ ID: tblID, @@ -119,8 +119,8 @@ func TestLoadBackupMetaPartionTable(t *testing.T) { store, err := storage.NewLocalStorage(testDir) require.NoError(t, err) - tblName := pmodel.NewCIStr("t1") - dbName := pmodel.NewCIStr("test") + tblName := ast.NewCIStr("t1") + dbName := ast.NewCIStr("test") tblID := int64(123) partID1 := int64(124) partID2 := int64(125) @@ -222,7 +222,7 @@ func TestLoadBackupMetaPartionTable(t *testing.T) { } func buildTableAndFiles(name string, tableID, fileCount int) (*model.TableInfo, []*backuppb.File) { - tblName := pmodel.NewCIStr(name) + tblName := ast.NewCIStr(name) tblID := int64(tableID) mockTbl := &model.TableInfo{ ID: tblID, @@ -249,7 +249,7 @@ func buildBenchmarkBackupmeta(b *testing.B, dbName string, tableCount, fileCount mockDB := model.DBInfo{ ID: 1, - Name: pmodel.NewCIStr(dbName), + Name: ast.NewCIStr(dbName), } mockDB.Deprecated.Tables = []*model.TableInfo{ mockTbl, diff --git a/br/pkg/restore/BUILD.bazel b/br/pkg/restore/BUILD.bazel index 584f73869b956..e15ba71b6d25e 100644 --- a/br/pkg/restore/BUILD.bazel +++ b/br/pkg/restore/BUILD.bazel @@ -25,7 +25,7 @@ go_library( "//pkg/kv", "//pkg/meta", "//pkg/meta/model", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/util", "@com_github_go_sql_driver_mysql//:mysql", "@com_github_opentracing_opentracing_go//:opentracing-go", @@ -65,7 +65,7 @@ go_test( "//br/pkg/restore/utils", "//br/pkg/utils/iter", "//pkg/kv", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/session", "//pkg/tablecodec", "//pkg/util", diff --git a/br/pkg/restore/ingestrec/BUILD.bazel b/br/pkg/restore/ingestrec/BUILD.bazel index e09029c3a8154..a7466b1ffa919 100644 --- a/br/pkg/restore/ingestrec/BUILD.bazel +++ b/br/pkg/restore/ingestrec/BUILD.bazel @@ -8,7 +8,7 @@ go_library( deps = [ "//pkg/infoschema", "//pkg/meta/model", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/types", "@com_github_pingcap_errors//:errors", "@com_github_pingcap_log//:log", @@ -27,7 +27,7 @@ go_test( "//pkg/kv", "//pkg/meta", "//pkg/meta/model", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/session", "//pkg/store/mockstore", "@com_github_pingcap_errors//:errors", diff --git a/br/pkg/restore/ingestrec/ingest_recorder.go b/br/pkg/restore/ingestrec/ingest_recorder.go index 9c0226a746796..a1adc0d82a48d 100644 --- a/br/pkg/restore/ingestrec/ingest_recorder.go +++ b/br/pkg/restore/ingestrec/ingest_recorder.go @@ -23,15 +23,15 @@ import ( "github.com/pingcap/log" "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/types" "go.uber.org/zap" ) // IngestIndexInfo records the information used to generate index drop/re-add SQL. type IngestIndexInfo struct { - SchemaName pmodel.CIStr - TableName pmodel.CIStr + SchemaName ast.CIStr + TableName ast.CIStr ColumnList string ColumnArgs []any IsPrimary bool diff --git a/br/pkg/restore/ingestrec/ingest_recorder_test.go b/br/pkg/restore/ingestrec/ingest_recorder_test.go index 8be133914580a..2df3a4c4ab17b 100644 --- a/br/pkg/restore/ingestrec/ingest_recorder_test.go +++ b/br/pkg/restore/ingestrec/ingest_recorder_test.go @@ -24,7 +24,7 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/session" "github.com/pingcap/tidb/pkg/store/mockstore" "github.com/stretchr/testify/require" @@ -62,7 +62,7 @@ func getIndex(id int64, columnsName []string) *model.IndexInfo { columns := make([]*model.IndexColumn, 0, len(columnsName)) for _, columnName := range columnsName { columns = append(columns, &model.IndexColumn{ - Name: pmodel.CIStr{ + Name: ast.CIStr{ O: columnName, L: columnName, }, @@ -70,7 +70,7 @@ func getIndex(id int64, columnsName []string) *model.IndexInfo { } return &model.IndexInfo{ ID: id, - Name: pmodel.CIStr{ + Name: ast.CIStr{ O: columnsName[0], L: columnsName[0], // noused }, @@ -120,22 +120,22 @@ func TestAddIngestRecorder(t *testing.T) { createMeta(t, store, func(m *meta.Mutator) { dbInfo := &model.DBInfo{ ID: 1, - Name: pmodel.NewCIStr(SchemaName), + Name: ast.NewCIStr(SchemaName), State: model.StatePublic, } err := m.CreateDatabase(dbInfo) require.NoError(t, err) tblInfo := &model.TableInfo{ ID: TableID, - Name: pmodel.NewCIStr(TableName), + Name: ast.NewCIStr(TableName), Columns: []*model.ColumnInfo{ { - Name: pmodel.NewCIStr("x"), + Name: ast.NewCIStr("x"), Hidden: false, State: model.StatePublic, }, { - Name: pmodel.NewCIStr("y"), + Name: ast.NewCIStr("y"), Hidden: false, State: model.StatePublic, }, @@ -143,22 +143,22 @@ func TestAddIngestRecorder(t *testing.T) { Indices: []*model.IndexInfo{ { ID: 1, - Name: pmodel.NewCIStr("x"), - Table: pmodel.NewCIStr(TableName), + Name: ast.NewCIStr("x"), + Table: ast.NewCIStr(TableName), Columns: []*model.IndexColumn{ { - Name: pmodel.NewCIStr("x"), + Name: ast.NewCIStr("x"), Offset: 0, Length: -1, }, { - Name: pmodel.NewCIStr("y"), + Name: ast.NewCIStr("y"), Offset: 1, Length: -1, }, }, Comment: "123", - Tp: pmodel.IndexTypeBtree, + Tp: ast.IndexTypeBtree, State: model.StatePublic, }, }, @@ -303,28 +303,28 @@ func TestIndexesKind(t *testing.T) { createMeta(t, store, func(m *meta.Mutator) { dbInfo := &model.DBInfo{ ID: 1, - Name: pmodel.NewCIStr(SchemaName), + Name: ast.NewCIStr(SchemaName), State: model.StatePublic, } err := m.CreateDatabase(dbInfo) require.NoError(t, err) tblInfo := &model.TableInfo{ ID: TableID, - Name: pmodel.NewCIStr(TableName), + Name: ast.NewCIStr(TableName), Columns: []*model.ColumnInfo{ { - Name: pmodel.NewCIStr("x"), + Name: ast.NewCIStr("x"), Hidden: false, State: model.StatePublic, }, { - Name: pmodel.NewCIStr("_V$_x_0"), + Name: ast.NewCIStr("_V$_x_0"), Hidden: true, GeneratedExprString: "`x` * 2", State: model.StatePublic, }, { - Name: pmodel.NewCIStr("z"), + Name: ast.NewCIStr("z"), Hidden: false, State: model.StatePublic, }, @@ -332,27 +332,27 @@ func TestIndexesKind(t *testing.T) { Indices: []*model.IndexInfo{ { ID: 1, - Name: pmodel.NewCIStr("x"), - Table: pmodel.NewCIStr(TableName), + Name: ast.NewCIStr("x"), + Table: ast.NewCIStr(TableName), Columns: []*model.IndexColumn{ { - Name: pmodel.NewCIStr("x"), + Name: ast.NewCIStr("x"), Offset: 0, Length: -1, }, { - Name: pmodel.NewCIStr("_V$_x_0"), + Name: ast.NewCIStr("_V$_x_0"), Offset: 1, Length: -1, }, { - Name: pmodel.NewCIStr("z"), + Name: ast.NewCIStr("z"), Offset: 2, Length: 4, }, }, Comment: "123", - Tp: pmodel.IndexTypeHash, + Tp: ast.IndexTypeHash, Invisible: true, State: model.StatePublic, }, @@ -396,7 +396,7 @@ func TestIndexesKind(t *testing.T) { require.Equal(t, 1, count) require.Equal(t, TableID, tableID) require.Equal(t, int64(1), indexID) - require.Equal(t, pmodel.NewCIStr(SchemaName), info.SchemaName) + require.Equal(t, ast.NewCIStr(SchemaName), info.SchemaName) require.Equal(t, "%n,(`x` * 2),%n(4)", info.ColumnList) require.Equal(t, []any{"x", "z"}, info.ColumnArgs) require.Equal(t, TableName, info.IndexInfo.Table.O) @@ -412,22 +412,22 @@ func TestRewriteTableID(t *testing.T) { createMeta(t, store, func(m *meta.Mutator) { dbInfo := &model.DBInfo{ ID: 1, - Name: pmodel.NewCIStr(SchemaName), + Name: ast.NewCIStr(SchemaName), State: model.StatePublic, } err := m.CreateDatabase(dbInfo) require.NoError(t, err) tblInfo := &model.TableInfo{ ID: TableID, - Name: pmodel.NewCIStr(TableName), + Name: ast.NewCIStr(TableName), Columns: []*model.ColumnInfo{ { - Name: pmodel.NewCIStr("x"), + Name: ast.NewCIStr("x"), Hidden: false, State: model.StatePublic, }, { - Name: pmodel.NewCIStr("y"), + Name: ast.NewCIStr("y"), Hidden: false, State: model.StatePublic, }, @@ -435,22 +435,22 @@ func TestRewriteTableID(t *testing.T) { Indices: []*model.IndexInfo{ { ID: 1, - Name: pmodel.NewCIStr("x"), - Table: pmodel.NewCIStr(TableName), + Name: ast.NewCIStr("x"), + Table: ast.NewCIStr(TableName), Columns: []*model.IndexColumn{ { - Name: pmodel.NewCIStr("x"), + Name: ast.NewCIStr("x"), Offset: 0, Length: -1, }, { - Name: pmodel.NewCIStr("y"), + Name: ast.NewCIStr("y"), Offset: 1, Length: -1, }, }, Comment: "123", - Tp: pmodel.IndexTypeBtree, + Tp: ast.IndexTypeBtree, State: model.StatePublic, }, }, diff --git a/br/pkg/restore/internal/prealloc_db/BUILD.bazel b/br/pkg/restore/internal/prealloc_db/BUILD.bazel index c4fb560ec9010..5f04d898af821 100644 --- a/br/pkg/restore/internal/prealloc_db/BUILD.bazel +++ b/br/pkg/restore/internal/prealloc_db/BUILD.bazel @@ -14,7 +14,7 @@ go_library( "//pkg/ddl", "//pkg/kv", "//pkg/meta/model", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/parser/mysql", "//pkg/sessionctx/variable", "@com_github_pingcap_errors//:errors", @@ -41,7 +41,7 @@ go_test( "//pkg/meta", "//pkg/meta/autoid", "//pkg/meta/model", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/parser/mysql", "//pkg/parser/types", "//pkg/testkit", diff --git a/br/pkg/restore/internal/prealloc_db/db.go b/br/pkg/restore/internal/prealloc_db/db.go index 35d4758abf868..af5cbb30ac402 100644 --- a/br/pkg/restore/internal/prealloc_db/db.go +++ b/br/pkg/restore/internal/prealloc_db/db.go @@ -17,7 +17,7 @@ import ( "github.com/pingcap/tidb/pkg/ddl" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/sessionctx/variable" "go.uber.org/zap" @@ -86,7 +86,7 @@ func (db *DB) ExecDDL(ctx context.Context, ddlJob *model.Job) error { } return errors.Trace(err) case model.ActionCreateTable: - err = db.se.CreateTable(ctx, pmodel.NewCIStr(ddlJob.SchemaName), tableInfo) + err = db.se.CreateTable(ctx, ast.NewCIStr(ddlJob.SchemaName), tableInfo) if err != nil { log.Error("create table failed", zap.Stringer("db", dbInfo.Name), @@ -379,7 +379,7 @@ func (db *DB) Close() { db.se.Close() } -func (db *DB) ensurePlacementPolicy(ctx context.Context, policyName pmodel.CIStr, policies *sync.Map) error { +func (db *DB) ensurePlacementPolicy(ctx context.Context, policyName ast.CIStr, policies *sync.Map) error { if policies == nil { return nil } diff --git a/br/pkg/restore/internal/prealloc_db/db_test.go b/br/pkg/restore/internal/prealloc_db/db_test.go index bfd05ace51b84..40fc9fe3c5490 100644 --- a/br/pkg/restore/internal/prealloc_db/db_test.go +++ b/br/pkg/restore/internal/prealloc_db/db_test.go @@ -21,7 +21,7 @@ import ( "github.com/pingcap/tidb/pkg/meta" "github.com/pingcap/tidb/pkg/meta/autoid" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/types" "github.com/pingcap/tidb/pkg/testkit" @@ -47,9 +47,9 @@ func TestRestoreAutoIncID(t *testing.T) { // Get schemas of db and table info, err := s.Mock.Domain.GetSnapshotInfoSchema(math.MaxUint64) require.NoErrorf(t, err, "Error get snapshot info schema: %s", err) - dbInfo, exists := info.SchemaByName(pmodel.NewCIStr("test")) + dbInfo, exists := info.SchemaByName(ast.NewCIStr("test")) require.Truef(t, exists, "Error get db info") - tableInfo, err := info.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("\"t\"")) + tableInfo, err := info.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("\"t\"")) require.NoErrorf(t, err, "Error get table info: %s", err) table := metautil.Table{ Info: tableInfo.Meta(), @@ -182,12 +182,12 @@ func prepareAllocTables( info, err := dom.GetSnapshotInfoSchema(math.MaxUint64) require.NoError(t, err) - dbInfo, exists := info.SchemaByName(pmodel.NewCIStr("test")) + dbInfo, exists := info.SchemaByName(ast.NewCIStr("test")) require.True(t, exists) tableInfos = make([]*metautil.Table, 0, 4) for i := 1; i <= len(createTableSQLs); i += 1 { tableName := fmt.Sprintf("t%d", i) - tableInfo, err := info.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr(tableName)) + tableInfo, err := info.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr(tableName)) require.NoError(t, err) tableInfos = append(tableInfos, &metautil.Table{ DB: dbInfo.Clone(), @@ -221,7 +221,7 @@ func cloneTableInfos( for i := int64(0); i < int64(len(createTableSQLs)); i += 1 { newTableInfo := originTableInfos[i].Info.Clone() newTableInfo.ID = id + i + 1 - newTableInfo.Name = pmodel.NewCIStr(fmt.Sprintf("%s%d", prefix, i+1)) + newTableInfo.Name = ast.NewCIStr(fmt.Sprintf("%s%d", prefix, i+1)) tableInfos = append(tableInfos, &metautil.Table{ DB: originTableInfos[i].DB.Clone(), Info: newTableInfo, @@ -240,7 +240,7 @@ func fakePolicyInfo(ident byte) *model.PolicyInfo { id := int64(ident) uid := uint64(ident) str := string(ident) - cistr := pmodel.NewCIStr(str) + cistr := ast.NewCIStr(str) return &model.PolicyInfo{ PlacementSettings: &model.PlacementSettings{ Followers: uid, @@ -318,7 +318,7 @@ func TestPolicyMode(t *testing.T) { policyMap.Store(fakepolicy1.Name.L, fakepolicy1) err = db.CreateDatabase(ctx, &model.DBInfo{ ID: 20000, - Name: pmodel.NewCIStr("test_db"), + Name: ast.NewCIStr("test_db"), Charset: "utf8mb4", Collate: "utf8mb4_bin", State: model.StatePublic, @@ -349,7 +349,7 @@ func TestUpdateMetaVersion(t *testing.T) { db.Session().Execute(ctx, "insert into test.t values (1),(2),(3);") info, err := s.Mock.Domain.GetSnapshotInfoSchema(math.MaxUint64) require.NoError(t, err) - tableInfo, err := info.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tableInfo, err := info.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) restoreTS := uint64(0) ctx = kv.WithInternalSourceType(ctx, kv.InternalTxnBR) @@ -375,7 +375,7 @@ func TestCreateTablesInDb(t *testing.T) { info, err := s.Mock.Domain.GetSnapshotInfoSchema(math.MaxUint64) require.NoErrorf(t, err, "Error get snapshot info schema: %s", err) - dbSchema, isExist := info.SchemaByName(pmodel.NewCIStr("test")) + dbSchema, isExist := info.SchemaByName(ast.NewCIStr("test")) require.True(t, isExist) tables := make([]*metautil.Table, 4) @@ -387,10 +387,10 @@ func TestCreateTablesInDb(t *testing.T) { DB: dbSchema, Info: &model.TableInfo{ ID: int64(i), - Name: pmodel.NewCIStr("test" + strconv.Itoa(i)), + Name: ast.NewCIStr("test" + strconv.Itoa(i)), Columns: []*model.ColumnInfo{{ ID: 1, - Name: pmodel.NewCIStr("id"), + Name: ast.NewCIStr("id"), FieldType: *intField, State: model.StatePublic, }}, @@ -427,17 +427,17 @@ func TestDDLJobMap(t *testing.T) { info, err := s.Mock.Domain.GetSnapshotInfoSchema(math.MaxUint64) require.NoError(t, err) - dbInfo, exists := info.SchemaByName(pmodel.NewCIStr("test")) + dbInfo, exists := info.SchemaByName(ast.NewCIStr("test")) require.True(t, exists) - tableInfo1, err := info.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t1")) + tableInfo1, err := info.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t1")) require.NoError(t, err) - tableInfo2, err := info.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t2")) + tableInfo2, err := info.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t2")) require.NoError(t, err) - tableInfo3, err := info.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t3")) + tableInfo3, err := info.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t3")) require.NoError(t, err) - tableInfo4, err := info.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t4")) + tableInfo4, err := info.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t4")) require.NoError(t, err) - tableInfo5, err := info.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t5")) + tableInfo5, err := info.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t5")) require.NoError(t, err) toBeCorrectedTables := map[restore.UniqueTableName]bool{ @@ -501,7 +501,7 @@ func TestDB_ExecDDL2(t *testing.T) { BinlogInfo: &model.HistoryInfo{ DBInfo: &model.DBInfo{ ID: 20000, - Name: pmodel.NewCIStr("test_db"), + Name: ast.NewCIStr("test_db"), Charset: "utf8mb4", Collate: "utf8mb4_bin", State: model.StatePublic, @@ -515,13 +515,13 @@ func TestDB_ExecDDL2(t *testing.T) { BinlogInfo: &model.HistoryInfo{ TableInfo: &model.TableInfo{ ID: 20000, - Name: pmodel.NewCIStr("t1"), + Name: ast.NewCIStr("t1"), Charset: "utf8mb4", Collate: "utf8mb4_bin", Columns: []*model.ColumnInfo{ { ID: 1, - Name: pmodel.NewCIStr("id"), + Name: ast.NewCIStr("id"), FieldType: *fieldType, State: model.StatePublic, Version: 2, @@ -564,9 +564,9 @@ func TestCreateTableConsistent(t *testing.T) { getTableInfo := func(name string) (*model.DBInfo, *model.TableInfo) { info, err := s.Mock.Domain.GetSnapshotInfoSchema(math.MaxUint64) require.NoError(t, err) - dbInfo, exists := info.SchemaByName(pmodel.NewCIStr("test")) + dbInfo, exists := info.SchemaByName(ast.NewCIStr("test")) require.True(t, exists) - tableInfo, err := info.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr(name)) + tableInfo, err := info.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr(name)) require.NoError(t, err) return dbInfo, tableInfo.Meta() } diff --git a/br/pkg/restore/misc.go b/br/pkg/restore/misc.go index 7bf0564e787eb..80802f359c369 100644 --- a/br/pkg/restore/misc.go +++ b/br/pkg/restore/misc.go @@ -29,7 +29,7 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" tidbutil "github.com/pingcap/tidb/pkg/util" "github.com/tikv/client-go/v2/oracle" pd "github.com/tikv/pd/client" @@ -59,8 +59,8 @@ func TransferBoolToValue(enable bool) string { // GetTableSchema returns the schema of a table from TiDB. func GetTableSchema( dom *domain.Domain, - dbName pmodel.CIStr, - tableName pmodel.CIStr, + dbName ast.CIStr, + tableName ast.CIStr, ) (*model.TableInfo, error) { info := dom.InfoSchema() table, err := info.TableByName(context.Background(), dbName, tableName) diff --git a/br/pkg/restore/misc_test.go b/br/pkg/restore/misc_test.go index 37fe2c4544859..1b0cb8f84d5be 100644 --- a/br/pkg/restore/misc_test.go +++ b/br/pkg/restore/misc_test.go @@ -25,7 +25,7 @@ import ( "github.com/pingcap/tidb/br/pkg/restore" "github.com/pingcap/tidb/br/pkg/restore/split" "github.com/pingcap/tidb/pkg/kv" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/session" "github.com/stretchr/testify/require" ) @@ -41,11 +41,11 @@ func TestGetTableSchema(t *testing.T) { defer m.Stop() dom := m.Domain - _, err = restore.GetTableSchema(dom, model.NewCIStr("test"), model.NewCIStr("tidb")) + _, err = restore.GetTableSchema(dom, ast.NewCIStr("test"), ast.NewCIStr("tidb")) require.Error(t, err) - tableInfo, err := restore.GetTableSchema(dom, model.NewCIStr("mysql"), model.NewCIStr("tidb")) + tableInfo, err := restore.GetTableSchema(dom, ast.NewCIStr("mysql"), ast.NewCIStr("tidb")) require.NoError(t, err) - require.Equal(t, model.NewCIStr("tidb"), tableInfo.Name) + require.Equal(t, ast.NewCIStr("tidb"), tableInfo.Name) } func TestAssertUserDBsEmpty(t *testing.T) { diff --git a/br/pkg/restore/snap_client/BUILD.bazel b/br/pkg/restore/snap_client/BUILD.bazel index 0a1a31cbd01c0..2df9df140d94f 100644 --- a/br/pkg/restore/snap_client/BUILD.bazel +++ b/br/pkg/restore/snap_client/BUILD.bazel @@ -38,7 +38,7 @@ go_library( "//pkg/kv", "//pkg/meta", "//pkg/meta/model", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/parser/mysql", "//pkg/tablecodec", "//pkg/util", @@ -97,7 +97,7 @@ go_test( "//pkg/domain", "//pkg/kv", "//pkg/meta/model", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/parser/mysql", "//pkg/session", "//pkg/tablecodec", diff --git a/br/pkg/restore/snap_client/client_test.go b/br/pkg/restore/snap_client/client_test.go index 4b96877949e23..2a53bb0b38bd9 100644 --- a/br/pkg/restore/snap_client/client_test.go +++ b/br/pkg/restore/snap_client/client_test.go @@ -36,7 +36,7 @@ import ( snapclient "github.com/pingcap/tidb/br/pkg/restore/snap_client" "github.com/pingcap/tidb/br/pkg/restore/split" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/tablecodec" "github.com/pingcap/tidb/pkg/types" @@ -54,7 +54,7 @@ func TestCreateTables(t *testing.T) { info, err := m.Domain.GetSnapshotInfoSchema(math.MaxUint64) require.NoError(t, err) - dbSchema, isExist := info.SchemaByName(pmodel.NewCIStr("test")) + dbSchema, isExist := info.SchemaByName(ast.NewCIStr("test")) require.True(t, isExist) client.SetBatchDdlSize(1) @@ -66,10 +66,10 @@ func TestCreateTables(t *testing.T) { DB: dbSchema, Info: &model.TableInfo{ ID: int64(i), - Name: pmodel.NewCIStr("test" + strconv.Itoa(i)), + Name: ast.NewCIStr("test" + strconv.Itoa(i)), Columns: []*model.ColumnInfo{{ ID: 1, - Name: pmodel.NewCIStr("id"), + Name: ast.NewCIStr("id"), FieldType: *intField, State: model.StatePublic, }}, @@ -156,7 +156,7 @@ func TestCheckTargetClusterFresh(t *testing.T) { ctx := context.Background() require.NoError(t, client.CheckTargetClusterFresh(ctx)) - require.NoError(t, client.CreateDatabases(ctx, []*metautil.Database{{Info: &model.DBInfo{Name: pmodel.NewCIStr("user_db")}}})) + require.NoError(t, client.CreateDatabases(ctx, []*metautil.Database{{Info: &model.DBInfo{Name: ast.NewCIStr("user_db")}}})) require.True(t, berrors.ErrRestoreNotFreshCluster.Equal(client.CheckTargetClusterFresh(ctx))) } @@ -173,7 +173,7 @@ func TestCheckTargetClusterFreshWithTable(t *testing.T) { ctx := context.Background() info, err := cluster.Domain.GetSnapshotInfoSchema(math.MaxUint64) require.NoError(t, err) - dbSchema, isExist := info.SchemaByName(pmodel.NewCIStr("test")) + dbSchema, isExist := info.SchemaByName(ast.NewCIStr("test")) require.True(t, isExist) intField := types.NewFieldType(mysql.TypeLong) intField.SetCharset("binary") @@ -181,10 +181,10 @@ func TestCheckTargetClusterFreshWithTable(t *testing.T) { DB: dbSchema, Info: &model.TableInfo{ ID: int64(1), - Name: pmodel.NewCIStr("t"), + Name: ast.NewCIStr("t"), Columns: []*model.ColumnInfo{{ ID: 1, - Name: pmodel.NewCIStr("id"), + Name: ast.NewCIStr("id"), FieldType: *intField, State: model.StatePublic, }}, diff --git a/br/pkg/restore/snap_client/placement_rule_manager_test.go b/br/pkg/restore/snap_client/placement_rule_manager_test.go index 2b2cfe3084476..6e9917e418538 100644 --- a/br/pkg/restore/snap_client/placement_rule_manager_test.go +++ b/br/pkg/restore/snap_client/placement_rule_manager_test.go @@ -27,7 +27,7 @@ import ( "github.com/pingcap/tidb/br/pkg/restore/split" restoreutils "github.com/pingcap/tidb/br/pkg/restore/utils" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/tablecodec" "github.com/pingcap/tidb/pkg/util/codec" "github.com/stretchr/testify/require" @@ -41,8 +41,8 @@ func generateTables() []*snapclient.CreatedTable { ID: 1, }, OldTable: &metautil.Table{ - DB: &model.DBInfo{Name: pmodel.NewCIStr("test")}, - Info: &model.TableInfo{Name: pmodel.NewCIStr("t1")}, + DB: &model.DBInfo{Name: ast.NewCIStr("test")}, + Info: &model.TableInfo{Name: ast.NewCIStr("t1")}, }, }, { @@ -50,8 +50,8 @@ func generateTables() []*snapclient.CreatedTable { ID: 100, }, OldTable: &metautil.Table{ - DB: &model.DBInfo{Name: pmodel.NewCIStr("test")}, - Info: &model.TableInfo{Name: pmodel.NewCIStr("t100")}, + DB: &model.DBInfo{Name: ast.NewCIStr("test")}, + Info: &model.TableInfo{Name: ast.NewCIStr("t100")}, }, }, } diff --git a/br/pkg/restore/snap_client/systable_restore.go b/br/pkg/restore/snap_client/systable_restore.go index 47c963d5f38d2..2e8dc71d1135e 100644 --- a/br/pkg/restore/snap_client/systable_restore.go +++ b/br/pkg/restore/snap_client/systable_restore.go @@ -17,7 +17,7 @@ import ( "github.com/pingcap/tidb/pkg/bindinfo" "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" filter "github.com/pingcap/tidb/pkg/util/table-filter" "go.uber.org/multierr" @@ -163,20 +163,20 @@ func (rc *SnapClient) restoreSystemSchema(ctx context.Context, f filter.Filter, // For fast querying whether a table exists and the temporary database of it. type database struct { ExistingTables map[string]*model.TableInfo - Name pmodel.CIStr - TemporaryName pmodel.CIStr + Name ast.CIStr + TemporaryName ast.CIStr } // getSystemDatabaseByName make a record of a system database, such as mysql and sys, from info schema by its name. func (rc *SnapClient) getSystemDatabaseByName(ctx context.Context, name string) (*database, bool, error) { infoSchema := rc.dom.InfoSchema() - schema, ok := infoSchema.SchemaByName(pmodel.NewCIStr(name)) + schema, ok := infoSchema.SchemaByName(ast.NewCIStr(name)) if !ok { return nil, false, nil } db := &database{ ExistingTables: map[string]*model.TableInfo{}, - Name: pmodel.NewCIStr(name), + Name: ast.NewCIStr(name), TemporaryName: utils.TemporaryDBName(name), } // It's OK to get all the tables from system tables. @@ -319,7 +319,7 @@ func CheckSysTableCompatibility(dom *domain.Domain, tables []*metautil.Table) er privilegeTablesInBackup = append(privilegeTablesInBackup, table) } } - sysDB := pmodel.NewCIStr(mysql.SystemDB) + sysDB := ast.NewCIStr(mysql.SystemDB) for _, table := range privilegeTablesInBackup { ti, err := restore.GetTableSchema(dom, sysDB, table.Info.Name) if err != nil { diff --git a/br/pkg/restore/snap_client/systable_restore_test.go b/br/pkg/restore/snap_client/systable_restore_test.go index cc95160482d45..504b3a45cd6fa 100644 --- a/br/pkg/restore/snap_client/systable_restore_test.go +++ b/br/pkg/restore/snap_client/systable_restore_test.go @@ -26,7 +26,7 @@ import ( "github.com/pingcap/tidb/br/pkg/restore/split" "github.com/pingcap/tidb/br/pkg/utils" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/session" "github.com/stretchr/testify/require" @@ -41,17 +41,17 @@ func TestCheckSysTableCompatibility(t *testing.T) { info, err := cluster.Domain.GetSnapshotInfoSchema(math.MaxUint64) require.NoError(t, err) - dbSchema, isExist := info.SchemaByName(pmodel.NewCIStr(mysql.SystemDB)) + dbSchema, isExist := info.SchemaByName(ast.NewCIStr(mysql.SystemDB)) require.True(t, isExist) tmpSysDB := dbSchema.Clone() tmpSysDB.Name = utils.TemporaryDBName(mysql.SystemDB) - sysDB := pmodel.NewCIStr(mysql.SystemDB) - userTI, err := restore.GetTableSchema(cluster.Domain, sysDB, pmodel.NewCIStr("user")) + sysDB := ast.NewCIStr(mysql.SystemDB) + userTI, err := restore.GetTableSchema(cluster.Domain, sysDB, ast.NewCIStr("user")) require.NoError(t, err) // user table in cluster have more columns(success) mockedUserTI := userTI.Clone() - userTI.Columns = append(userTI.Columns, &model.ColumnInfo{Name: pmodel.NewCIStr("new-name")}) + userTI.Columns = append(userTI.Columns, &model.ColumnInfo{Name: ast.NewCIStr("new-name")}) err = snapclient.CheckSysTableCompatibility(cluster.Domain, []*metautil.Table{{ DB: tmpSysDB, Info: mockedUserTI, @@ -61,7 +61,7 @@ func TestCheckSysTableCompatibility(t *testing.T) { // user table in cluster have less columns(failed) mockedUserTI = userTI.Clone() - mockedUserTI.Columns = append(mockedUserTI.Columns, &model.ColumnInfo{Name: pmodel.NewCIStr("new-name")}) + mockedUserTI.Columns = append(mockedUserTI.Columns, &model.ColumnInfo{Name: ast.NewCIStr("new-name")}) err = snapclient.CheckSysTableCompatibility(cluster.Domain, []*metautil.Table{{ DB: tmpSysDB, Info: mockedUserTI, @@ -95,12 +95,12 @@ func TestCheckSysTableCompatibility(t *testing.T) { require.NoError(t, err) // use the mysql.db table to test for column count mismatch. - dbTI, err := restore.GetTableSchema(cluster.Domain, sysDB, pmodel.NewCIStr("db")) + dbTI, err := restore.GetTableSchema(cluster.Domain, sysDB, ast.NewCIStr("db")) require.NoError(t, err) // other system tables in cluster have more columns(failed) mockedDBTI := dbTI.Clone() - dbTI.Columns = append(dbTI.Columns, &model.ColumnInfo{Name: pmodel.NewCIStr("new-name")}) + dbTI.Columns = append(dbTI.Columns, &model.ColumnInfo{Name: ast.NewCIStr("new-name")}) err = snapclient.CheckSysTableCompatibility(cluster.Domain, []*metautil.Table{{ DB: tmpSysDB, Info: mockedDBTI, diff --git a/br/pkg/restore/snap_client/tikv_sender_test.go b/br/pkg/restore/snap_client/tikv_sender_test.go index b5a38ffc839a3..fa3b6c6be788a 100644 --- a/br/pkg/restore/snap_client/tikv_sender_test.go +++ b/br/pkg/restore/snap_client/tikv_sender_test.go @@ -27,7 +27,7 @@ import ( restoreutils "github.com/pingcap/tidb/br/pkg/restore/utils" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/tablecodec" "github.com/stretchr/testify/require" ) @@ -78,7 +78,7 @@ func newPartitionID(ids []int64) *model.PartitionInfo { for i, id := range ids { definitions = append(definitions, model.PartitionDefinition{ ID: id, - Name: pmodel.NewCIStr(fmt.Sprintf("%d", i)), + Name: ast.NewCIStr(fmt.Sprintf("%d", i)), }) } return &model.PartitionInfo{Definitions: definitions} @@ -135,21 +135,21 @@ func generateCreatedTables(t *testing.T, upstreamTableIDs []int64, upstreamParti createdTable := &snapclient.CreatedTable{ Table: &model.TableInfo{ ID: downstreamTableID, - Name: pmodel.NewCIStr(fmt.Sprintf("tbl-%d", upstreamTableID)), + Name: ast.NewCIStr(fmt.Sprintf("tbl-%d", upstreamTableID)), Indices: []*model.IndexInfo{ - {Name: pmodel.NewCIStr("idx1"), ID: 1}, - {Name: pmodel.NewCIStr("idx2"), ID: 2}, - {Name: pmodel.NewCIStr("idx3"), ID: 3}, + {Name: ast.NewCIStr("idx1"), ID: 1}, + {Name: ast.NewCIStr("idx2"), ID: 2}, + {Name: ast.NewCIStr("idx3"), ID: 3}, }, }, OldTable: &metautil.Table{ - DB: &model.DBInfo{Name: pmodel.NewCIStr("test")}, + DB: &model.DBInfo{Name: ast.NewCIStr("test")}, Info: &model.TableInfo{ ID: upstreamTableID, Indices: []*model.IndexInfo{ - {Name: pmodel.NewCIStr("idx1"), ID: 1}, - {Name: pmodel.NewCIStr("idx2"), ID: 2}, - {Name: pmodel.NewCIStr("idx3"), ID: 3}, + {Name: ast.NewCIStr("idx1"), ID: 1}, + {Name: ast.NewCIStr("idx2"), ID: 2}, + {Name: ast.NewCIStr("idx3"), ID: 3}, }, }, }, @@ -161,11 +161,11 @@ func generateCreatedTables(t *testing.T, upstreamTableIDs []int64, upstreamParti upDefs := make([]model.PartitionDefinition, 0, len(partitionIDs)) for _, partitionID := range partitionIDs { downDefs = append(downDefs, model.PartitionDefinition{ - Name: pmodel.NewCIStr(fmt.Sprintf("p_%d", partitionID)), + Name: ast.NewCIStr(fmt.Sprintf("p_%d", partitionID)), ID: downstreamID(partitionID), }) upDefs = append(upDefs, model.PartitionDefinition{ - Name: pmodel.NewCIStr(fmt.Sprintf("p_%d", partitionID)), + Name: ast.NewCIStr(fmt.Sprintf("p_%d", partitionID)), ID: partitionID, }) } diff --git a/br/pkg/restore/tiflashrec/BUILD.bazel b/br/pkg/restore/tiflashrec/BUILD.bazel index 07791c626ec1e..ffaae84de12d9 100644 --- a/br/pkg/restore/tiflashrec/BUILD.bazel +++ b/br/pkg/restore/tiflashrec/BUILD.bazel @@ -27,7 +27,7 @@ go_test( ":tiflashrec", "//pkg/infoschema", "//pkg/meta/model", - "//pkg/parser/model", + "//pkg/parser/ast", "@com_github_stretchr_testify//require", ], ) diff --git a/br/pkg/restore/tiflashrec/tiflash_recorder_test.go b/br/pkg/restore/tiflashrec/tiflash_recorder_test.go index 532462f181eba..796bbcd79ae79 100644 --- a/br/pkg/restore/tiflashrec/tiflash_recorder_test.go +++ b/br/pkg/restore/tiflashrec/tiflash_recorder_test.go @@ -21,7 +21,7 @@ import ( "github.com/pingcap/tidb/br/pkg/restore/tiflashrec" "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/stretchr/testify/require" ) @@ -137,7 +137,7 @@ func TestGenSql(t *testing.T) { tInfo := func(id int, name string) *model.TableInfo { return &model.TableInfo{ ID: int64(id), - Name: pmodel.NewCIStr(name), + Name: ast.NewCIStr(name), } } fakeInfo := infoschema.MockInfoSchema([]*model.TableInfo{ @@ -176,7 +176,7 @@ func TestGenResetSql(t *testing.T) { tInfo := func(id int, name string) *model.TableInfo { return &model.TableInfo{ ID: int64(id), - Name: pmodel.NewCIStr(name), + Name: ast.NewCIStr(name), } } fakeInfo := infoschema.MockInfoSchema([]*model.TableInfo{ diff --git a/br/pkg/restore/utils/BUILD.bazel b/br/pkg/restore/utils/BUILD.bazel index a40f0a883ae09..ef0d8355b9ded 100644 --- a/br/pkg/restore/utils/BUILD.bazel +++ b/br/pkg/restore/utils/BUILD.bazel @@ -42,7 +42,7 @@ go_test( "//br/pkg/rtree", "//pkg/kv", "//pkg/meta/model", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/sessionctx/stmtctx", "//pkg/tablecodec", "//pkg/types", diff --git a/br/pkg/restore/utils/rewrite_rule_test.go b/br/pkg/restore/utils/rewrite_rule_test.go index a8a985f1d0e20..7f2217025f66c 100644 --- a/br/pkg/restore/utils/rewrite_rule_test.go +++ b/br/pkg/restore/utils/rewrite_rule_test.go @@ -26,7 +26,7 @@ import ( "github.com/pingcap/tidb/br/pkg/rtree" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/tablecodec" "github.com/pingcap/tidb/pkg/util/codec" "github.com/stretchr/testify/require" @@ -300,22 +300,22 @@ func generateRewriteTableInfos() (newTableInfo, oldTableInfo *model.TableInfo) { Indices: []*model.IndexInfo{ { ID: 1, - Name: pmodel.NewCIStr("i1"), + Name: ast.NewCIStr("i1"), }, { ID: 2, - Name: pmodel.NewCIStr("i2"), + Name: ast.NewCIStr("i2"), }, }, Partition: &model.PartitionInfo{ Definitions: []model.PartitionDefinition{ { ID: 100, - Name: pmodel.NewCIStr("p1"), + Name: ast.NewCIStr("p1"), }, { ID: 200, - Name: pmodel.NewCIStr("p2"), + Name: ast.NewCIStr("p2"), }, }, }, @@ -325,22 +325,22 @@ func generateRewriteTableInfos() (newTableInfo, oldTableInfo *model.TableInfo) { Indices: []*model.IndexInfo{ { ID: 1, - Name: pmodel.NewCIStr("i1"), + Name: ast.NewCIStr("i1"), }, { ID: 2, - Name: pmodel.NewCIStr("i2"), + Name: ast.NewCIStr("i2"), }, }, Partition: &model.PartitionInfo{ Definitions: []model.PartitionDefinition{ { ID: 101, - Name: pmodel.NewCIStr("p1"), + Name: ast.NewCIStr("p1"), }, { ID: 201, - Name: pmodel.NewCIStr("p2"), + Name: ast.NewCIStr("p2"), }, }, }, diff --git a/br/pkg/stream/BUILD.bazel b/br/pkg/stream/BUILD.bazel index 252f789c78b75..8d5eb63becacf 100644 --- a/br/pkg/stream/BUILD.bazel +++ b/br/pkg/stream/BUILD.bazel @@ -75,7 +75,6 @@ go_test( "//pkg/meta", "//pkg/meta/model", "//pkg/parser/ast", - "//pkg/parser/model", "//pkg/parser/mysql", "//pkg/tablecodec", "//pkg/types", diff --git a/br/pkg/stream/rewrite_meta_rawkv_test.go b/br/pkg/stream/rewrite_meta_rawkv_test.go index 81d21e5e5b5a7..c926f6a574bb6 100644 --- a/br/pkg/stream/rewrite_meta_rawkv_test.go +++ b/br/pkg/stream/rewrite_meta_rawkv_test.go @@ -11,7 +11,6 @@ import ( "github.com/pingcap/tidb/pkg/meta" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/ast" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/tablecodec" "github.com/pingcap/tidb/pkg/types" @@ -38,7 +37,7 @@ func MockEmptySchemasReplace(midr *mockInsertDeleteRange, dbMap map[UpstreamID]* func produceDBInfoValue(dbName string, dbID int64) ([]byte, error) { dbInfo := model.DBInfo{ ID: dbID, - Name: pmodel.NewCIStr(dbName), + Name: ast.NewCIStr(dbName), } return json.Marshal(&dbInfo) } @@ -46,7 +45,7 @@ func produceDBInfoValue(dbName string, dbID int64) ([]byte, error) { func produceTableInfoValue(tableName string, tableID int64) ([]byte, error) { tableInfo := model.TableInfo{ ID: tableID, - Name: pmodel.NewCIStr(tableName), + Name: ast.NewCIStr(tableName), } return json.Marshal(&tableInfo) @@ -260,11 +259,11 @@ func TestRewriteTableInfoForPartitionTable(t *testing.T) { // create tableinfo. pt1 := model.PartitionDefinition{ ID: pt1ID, - Name: pmodel.NewCIStr(pt1Name), + Name: ast.NewCIStr(pt1Name), } pt2 := model.PartitionDefinition{ ID: pt2ID, - Name: pmodel.NewCIStr(pt2Name), + Name: ast.NewCIStr(pt2Name), } pi := model.PartitionInfo{ @@ -276,7 +275,7 @@ func TestRewriteTableInfoForPartitionTable(t *testing.T) { tbl := model.TableInfo{ ID: tableID, - Name: pmodel.NewCIStr(tableName), + Name: ast.NewCIStr(tableName), Partition: &pi, } value, err := json.Marshal(&tbl) @@ -366,11 +365,11 @@ func TestRewriteTableInfoForExchangePartition(t *testing.T) { // construct table t1 with the partition pi(pt1, pt2). pt1 := model.PartitionDefinition{ ID: pt1ID, - Name: pmodel.NewCIStr(pt1Name), + Name: ast.NewCIStr(pt1Name), } pt2 := model.PartitionDefinition{ ID: pt2ID, - Name: pmodel.NewCIStr(pt2Name), + Name: ast.NewCIStr(pt2Name), } pi := model.PartitionInfo{ @@ -380,7 +379,7 @@ func TestRewriteTableInfoForExchangePartition(t *testing.T) { pi.Definitions = append(pi.Definitions, pt1, pt2) t1 := model.TableInfo{ ID: tableID1, - Name: pmodel.NewCIStr(tableName1), + Name: ast.NewCIStr(tableName1), Partition: &pi, } db1 := model.DBInfo{} @@ -388,7 +387,7 @@ func TestRewriteTableInfoForExchangePartition(t *testing.T) { // construct table t2 without partition. t2 := model.TableInfo{ ID: tableID2, - Name: pmodel.NewCIStr(tableName2), + Name: ast.NewCIStr(tableName2), } db2 := model.DBInfo{} @@ -457,16 +456,16 @@ func TestRewriteTableInfoForTTLTable(t *testing.T) { tbl := model.TableInfo{ ID: tableID, - Name: pmodel.NewCIStr(tableName), + Name: ast.NewCIStr(tableName), Columns: []*model.ColumnInfo{ { ID: colID, - Name: pmodel.NewCIStr(colName), + Name: ast.NewCIStr(colName), FieldType: *types.NewFieldType(mysql.TypeTimestamp), }, }, TTLInfo: &model.TTLInfo{ - ColumnName: pmodel.NewCIStr(colName), + ColumnName: ast.NewCIStr(colName), IntervalExprStr: "1", IntervalTimeUnit: int(ast.TimeUnitDay), Enable: true, diff --git a/br/pkg/task/BUILD.bazel b/br/pkg/task/BUILD.bazel index 12c476f34a544..d6d4aaaf0291e 100644 --- a/br/pkg/task/BUILD.bazel +++ b/br/pkg/task/BUILD.bazel @@ -58,7 +58,7 @@ go_library( "//pkg/infoschema/context", "//pkg/kv", "//pkg/meta/model", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/parser/mysql", "//pkg/sessionctx/stmtctx", "//pkg/sessionctx/variable", @@ -135,7 +135,7 @@ go_test( "//pkg/config", "//pkg/ddl", "//pkg/meta/model", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/parser/mysql", "//pkg/statistics/util", "//pkg/tablecodec", diff --git a/br/pkg/task/config_test.go b/br/pkg/task/config_test.go index 8bd7d72e4f3bb..fe632f88d1c7c 100644 --- a/br/pkg/task/config_test.go +++ b/br/pkg/task/config_test.go @@ -30,7 +30,7 @@ import ( "github.com/pingcap/tidb/br/pkg/storage" "github.com/pingcap/tidb/br/pkg/utils" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/statistics/util" "github.com/pingcap/tidb/pkg/tablecodec" "github.com/stretchr/testify/require" @@ -207,12 +207,12 @@ func mockReadSchemasFromBackupMeta(t *testing.T, db2Tables map[string][]string) mockSchemas := make([]*backuppb.Schema, 0) var dbID int64 = 1 for db, tables := range db2Tables { - dbName := pmodel.NewCIStr(db) + dbName := ast.NewCIStr(db) mockTblList := make([]*model.TableInfo, 0) tblBytesList, statsBytesList := make([][]byte, 0), make([][]byte, 0) for i, table := range tables { - tblName := pmodel.NewCIStr(table) + tblName := ast.NewCIStr(table) mockTbl := &model.TableInfo{ ID: dbID*100 + int64(i), Name: tblName, diff --git a/br/pkg/task/restore.go b/br/pkg/task/restore.go index 94572ff301ba6..1cb3fd9e92fe7 100644 --- a/br/pkg/task/restore.go +++ b/br/pkg/task/restore.go @@ -38,7 +38,7 @@ import ( "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/util/collate" "github.com/pingcap/tidb/pkg/util/engine" "github.com/spf13/cobra" @@ -1496,7 +1496,7 @@ func PreCheckTableClusterIndex( if job.Type == model.ActionCreateTable { tableInfo := job.BinlogInfo.TableInfo if tableInfo != nil { - oldTableInfo, err := restore.GetTableSchema(dom, pmodel.NewCIStr(job.SchemaName), tableInfo.Name) + oldTableInfo, err := restore.GetTableSchema(dom, ast.NewCIStr(job.SchemaName), tableInfo.Name) // table exists in database if err == nil { if tableInfo.IsCommonHandle != oldTableInfo.IsCommonHandle { diff --git a/br/pkg/task/restore_test.go b/br/pkg/task/restore_test.go index 86ceb3755ee09..27200e6bb486f 100644 --- a/br/pkg/task/restore_test.go +++ b/br/pkg/task/restore_test.go @@ -25,7 +25,7 @@ import ( utiltest "github.com/pingcap/tidb/br/pkg/utiltest" "github.com/pingcap/tidb/pkg/ddl" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/types" @@ -71,10 +71,10 @@ func TestPreCheckTableTiFlashReplicas(t *testing.T) { } tables[i] = &metautil.Table{ - DB: &model.DBInfo{Name: pmodel.NewCIStr("test")}, + DB: &model.DBInfo{Name: ast.NewCIStr("test")}, Info: &model.TableInfo{ ID: int64(i), - Name: pmodel.NewCIStr("test" + strconv.Itoa(i)), + Name: ast.NewCIStr("test" + strconv.Itoa(i)), TiFlashReplica: tiflashReplica, }, } @@ -115,7 +115,7 @@ func TestPreCheckTableClusterIndex(t *testing.T) { info, err := m.Domain.GetSnapshotInfoSchema(math.MaxUint64) require.NoError(t, err) - dbSchema, isExist := info.SchemaByName(pmodel.NewCIStr("test")) + dbSchema, isExist := info.SchemaByName(ast.NewCIStr("test")) require.True(t, isExist) tables := make([]*metautil.Table, 4) @@ -126,10 +126,10 @@ func TestPreCheckTableClusterIndex(t *testing.T) { DB: dbSchema, Info: &model.TableInfo{ ID: int64(i), - Name: pmodel.NewCIStr("test" + strconv.Itoa(i)), + Name: ast.NewCIStr("test" + strconv.Itoa(i)), Columns: []*model.ColumnInfo{{ ID: 1, - Name: pmodel.NewCIStr("id"), + Name: ast.NewCIStr("id"), FieldType: *intField, State: model.StatePublic, }}, @@ -155,7 +155,7 @@ func TestPreCheckTableClusterIndex(t *testing.T) { Query: "", BinlogInfo: &model.HistoryInfo{ TableInfo: &model.TableInfo{ - Name: pmodel.NewCIStr("test1"), + Name: ast.NewCIStr("test1"), IsCommonHandle: true, }, }, @@ -283,9 +283,9 @@ func TestFilterDDLJobs(t *testing.T) { require.NoErrorf(t, err, "Finially flush backupmeta failed", err) infoSchema, err := s.Mock.Domain.GetSnapshotInfoSchema(ts) require.NoErrorf(t, err, "Error get snapshot info schema: %s", err) - dbInfo, ok := infoSchema.SchemaByName(pmodel.NewCIStr("test_db")) + dbInfo, ok := infoSchema.SchemaByName(ast.NewCIStr("test_db")) require.Truef(t, ok, "DB info not exist") - tableInfo, err := infoSchema.TableByName(context.Background(), pmodel.NewCIStr("test_db"), pmodel.NewCIStr("test_table")) + tableInfo, err := infoSchema.TableByName(context.Background(), ast.NewCIStr("test_db"), ast.NewCIStr("test_table")) require.NoErrorf(t, err, "Error get table info: %s", err) tables := []*metautil.Table{{ DB: dbInfo, @@ -348,9 +348,9 @@ func TestFilterDDLJobsV2(t *testing.T) { infoSchema, err := s.Mock.Domain.GetSnapshotInfoSchema(ts) require.NoErrorf(t, err, "Error get snapshot info schema: %s", err) - dbInfo, ok := infoSchema.SchemaByName(pmodel.NewCIStr("test_db")) + dbInfo, ok := infoSchema.SchemaByName(ast.NewCIStr("test_db")) require.Truef(t, ok, "DB info not exist") - tableInfo, err := infoSchema.TableByName(context.Background(), pmodel.NewCIStr("test_db"), pmodel.NewCIStr("test_table")) + tableInfo, err := infoSchema.TableByName(context.Background(), ast.NewCIStr("test_db"), ast.NewCIStr("test_table")) require.NoErrorf(t, err, "Error get table info: %s", err) tables := []*metautil.Table{{ DB: dbInfo, diff --git a/br/pkg/utils/BUILD.bazel b/br/pkg/utils/BUILD.bazel index 424329b7b3134..85d5d3c36c7a1 100644 --- a/br/pkg/utils/BUILD.bazel +++ b/br/pkg/utils/BUILD.bazel @@ -31,7 +31,7 @@ go_library( "//pkg/errno", "//pkg/kv", "//pkg/meta/model", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/parser/mysql", "//pkg/parser/terror", "//pkg/parser/types", diff --git a/br/pkg/utils/schema.go b/br/pkg/utils/schema.go index 47ea86dcc9370..7c01b1538338c 100644 --- a/br/pkg/utils/schema.go +++ b/br/pkg/utils/schema.go @@ -7,7 +7,7 @@ import ( "strings" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" ) @@ -32,7 +32,7 @@ func EncloseDBAndTable(database, table string) string { } // IsTemplateSysDB checks wheterh the dbname is temporary system database(__TiDB_BR_Temporary_mysql or __TiDB_BR_Temporary_sys). -func IsTemplateSysDB(dbname pmodel.CIStr) bool { +func IsTemplateSysDB(dbname ast.CIStr) bool { return dbname.O == temporaryDBNamePrefix+mysql.SystemDB || dbname.O == temporaryDBNamePrefix+mysql.SysDB } @@ -43,12 +43,12 @@ func IsSysDB(dbLowerName string) bool { } // TemporaryDBName makes a 'private' database name. -func TemporaryDBName(db string) pmodel.CIStr { - return pmodel.NewCIStr(temporaryDBNamePrefix + db) +func TemporaryDBName(db string) ast.CIStr { + return ast.NewCIStr(temporaryDBNamePrefix + db) } // GetSysDBName get the original name of system DB -func GetSysDBName(tempDB pmodel.CIStr) (string, bool) { +func GetSysDBName(tempDB ast.CIStr) (string, bool) { if ok := strings.HasPrefix(tempDB.O, temporaryDBNamePrefix); !ok { return tempDB.O, false } @@ -56,7 +56,7 @@ func GetSysDBName(tempDB pmodel.CIStr) (string, bool) { } // GetSysDBCIStrName get the CIStr name of system DB -func GetSysDBCIStrName(tempDB pmodel.CIStr) (pmodel.CIStr, bool) { +func GetSysDBCIStrName(tempDB ast.CIStr) (ast.CIStr, bool) { if ok := strings.HasPrefix(tempDB.O, temporaryDBNamePrefix); !ok { return tempDB, false } diff --git a/cmd/ddltest/BUILD.bazel b/cmd/ddltest/BUILD.bazel index 6dce632241496..3b0340ecbc66f 100644 --- a/cmd/ddltest/BUILD.bazel +++ b/cmd/ddltest/BUILD.bazel @@ -19,7 +19,7 @@ go_test( "//pkg/ddl", "//pkg/domain", "//pkg/kv", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/parser/terror", "//pkg/session", "//pkg/session/types", diff --git a/cmd/ddltest/ddl_test.go b/cmd/ddltest/ddl_test.go index 78e53088e15bc..e810b1d02f06d 100644 --- a/cmd/ddltest/ddl_test.go +++ b/cmd/ddltest/ddl_test.go @@ -38,7 +38,7 @@ import ( "github.com/pingcap/tidb/pkg/ddl" "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/kv" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/session" sessiontypes "github.com/pingcap/tidb/pkg/session/types" @@ -495,7 +495,7 @@ func (s *ddlSuite) runDDL(sql string) chan error { } func (s *ddlSuite) getTable(t *testing.T, name string) table.Table { - tbl, err := domain.GetDomain(s.ctx).InfoSchema().TableByName(goctx.Background(), model.NewCIStr("test_ddl"), model.NewCIStr(name)) + tbl, err := domain.GetDomain(s.ctx).InfoSchema().TableByName(goctx.Background(), ast.NewCIStr("test_ddl"), ast.NewCIStr(name)) require.NoError(t, err) return tbl } diff --git a/dumpling/export/BUILD.bazel b/dumpling/export/BUILD.bazel index 83a5305f3c1c5..8422a877a9703 100644 --- a/dumpling/export/BUILD.bazel +++ b/dumpling/export/BUILD.bazel @@ -40,7 +40,6 @@ go_library( "//pkg/parser", "//pkg/parser/ast", "//pkg/parser/format", - "//pkg/parser/model", "//pkg/store/helper", "//pkg/tablecodec", "//pkg/util", diff --git a/dumpling/export/sql.go b/dumpling/export/sql.go index 67ee1dcc30047..a4bd99561dfd1 100644 --- a/dumpling/export/sql.go +++ b/dumpling/export/sql.go @@ -22,7 +22,7 @@ import ( dbconfig "github.com/pingcap/tidb/pkg/config" "github.com/pingcap/tidb/pkg/errno" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" pd "github.com/tikv/pd/client/http" "go.uber.org/multierr" "go.uber.org/zap" @@ -1559,7 +1559,7 @@ func GetDBInfo(db *sql.Conn, tables map[string]map[string]struct{}) ([]*model.DB } last := len(schemas) - 1 if last < 0 || schemas[last].Name.O != tableSchema { - dbInfo := &model.DBInfo{Name: pmodel.CIStr{O: tableSchema}} + dbInfo := &model.DBInfo{Name: ast.CIStr{O: tableSchema}} dbInfo.Deprecated.Tables = make([]*model.TableInfo, 0, len(tables[tableSchema])) schemas = append(schemas, dbInfo) last++ @@ -1571,14 +1571,14 @@ func GetDBInfo(db *sql.Conn, tables map[string]map[string]struct{}) ([]*model.DB for partitionName, partitionID := range ptm { partition.Definitions = append(partition.Definitions, model.PartitionDefinition{ ID: partitionID, - Name: pmodel.CIStr{O: partitionName}, + Name: ast.CIStr{O: partitionName}, }) } } } schemas[last].Deprecated.Tables = append(schemas[last].Deprecated.Tables, &model.TableInfo{ ID: tidbTableID, - Name: pmodel.CIStr{O: tableName}, + Name: ast.CIStr{O: tableName}, Partition: partition, }) return nil diff --git a/lightning/pkg/importer/BUILD.bazel b/lightning/pkg/importer/BUILD.bazel index 5e1991e279f33..6677b7b66d5ac 100644 --- a/lightning/pkg/importer/BUILD.bazel +++ b/lightning/pkg/importer/BUILD.bazel @@ -156,7 +156,6 @@ go_test( "//pkg/meta/model", "//pkg/parser", "//pkg/parser/ast", - "//pkg/parser/model", "//pkg/parser/mysql", "//pkg/store/mockstore", "//pkg/table/tables", diff --git a/lightning/pkg/importer/get_pre_info_test.go b/lightning/pkg/importer/get_pre_info_test.go index ca5b0cfbf337a..6310cf3fbd2a3 100644 --- a/lightning/pkg/importer/get_pre_info_test.go +++ b/lightning/pkg/importer/get_pre_info_test.go @@ -32,7 +32,7 @@ import ( "github.com/pingcap/tidb/pkg/lightning/common" "github.com/pingcap/tidb/pkg/lightning/config" "github.com/pingcap/tidb/pkg/lightning/mydump" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/types" "github.com/stretchr/testify/require" pqt_buf_src "github.com/xitongsys/parquet-go-source/buffer" @@ -80,12 +80,12 @@ func TestGetPreInfoGenerateTableInfo(t *testing.T) { createTblSQL := fmt.Sprintf("create table `%s`.`%s` (a varchar(16) not null, b varchar(8) default 'DEFA')", schemaName, tblName) tblInfo, err := newTableInfo(createTblSQL, 1) require.Nil(t, err) - require.Equal(t, model.NewCIStr(tblName), tblInfo.Name) + require.Equal(t, ast.NewCIStr(tblName), tblInfo.Name) require.Equal(t, len(tblInfo.Columns), 2) - require.Equal(t, model.NewCIStr("a"), tblInfo.Columns[0].Name) + require.Equal(t, ast.NewCIStr("a"), tblInfo.Columns[0].Name) require.Nil(t, tblInfo.Columns[0].DefaultValue) require.False(t, hasDefault(tblInfo.Columns[0])) - require.Equal(t, model.NewCIStr("b"), tblInfo.Columns[1].Name) + require.Equal(t, ast.NewCIStr("b"), tblInfo.Columns[1].Name) require.NotNil(t, tblInfo.Columns[1].DefaultValue) createTblSQL = fmt.Sprintf("create table `%s`.`%s` (a varchar(16), b varchar(8) default 'DEFAULT_BBBBB')", schemaName, tblName) // default value exceeds the length diff --git a/lightning/pkg/importer/mock/BUILD.bazel b/lightning/pkg/importer/mock/BUILD.bazel index 3724f015eb38b..1fcb45e385c0e 100644 --- a/lightning/pkg/importer/mock/BUILD.bazel +++ b/lightning/pkg/importer/mock/BUILD.bazel @@ -11,7 +11,7 @@ go_library( "//pkg/errno", "//pkg/lightning/mydump", "//pkg/meta/model", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/util/dbterror", "//pkg/util/filter", "@com_github_docker_go_units//:go-units", @@ -29,7 +29,7 @@ go_test( deps = [ "//lightning/pkg/importer", "//pkg/meta/model", - "//pkg/parser/model", + "//pkg/parser/ast", "@com_github_stretchr_testify//require", ], ) diff --git a/lightning/pkg/importer/mock/mock.go b/lightning/pkg/importer/mock/mock.go index 3a88b0cd2a56a..443f87a13e4c8 100644 --- a/lightning/pkg/importer/mock/mock.go +++ b/lightning/pkg/importer/mock/mock.go @@ -25,7 +25,7 @@ import ( "github.com/pingcap/tidb/pkg/errno" "github.com/pingcap/tidb/pkg/lightning/mydump" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/util/dbterror" "github.com/pingcap/tidb/pkg/util/filter" pdhttp "github.com/tikv/pd/client/http" @@ -218,7 +218,7 @@ func (t *TargetInfo) SetTableInfo(schemaName string, tableName string, tblInfo * func (t *TargetInfo) FetchRemoteDBModels(_ context.Context) ([]*model.DBInfo, error) { resultInfos := []*model.DBInfo{} for dbName := range t.dbTblInfoMap { - resultInfos = append(resultInfos, &model.DBInfo{Name: pmodel.NewCIStr(dbName)}) + resultInfos = append(resultInfos, &model.DBInfo{Name: ast.NewCIStr(dbName)}) } return resultInfos, nil } diff --git a/lightning/pkg/importer/mock/mock_test.go b/lightning/pkg/importer/mock/mock_test.go index 2620583bee966..90e3a3765e41d 100644 --- a/lightning/pkg/importer/mock/mock_test.go +++ b/lightning/pkg/importer/mock/mock_test.go @@ -21,7 +21,7 @@ import ( "github.com/pingcap/tidb/lightning/pkg/importer" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/stretchr/testify/require" ) @@ -164,16 +164,16 @@ func TestMockTargetInfoBasic(t *testing.T) { &TableInfo{ TableModel: &model.TableInfo{ ID: 1, - Name: pmodel.NewCIStr("testtbl1"), + Name: ast.NewCIStr("testtbl1"), Columns: []*model.ColumnInfo{ { ID: 1, - Name: pmodel.NewCIStr("c_1"), + Name: ast.NewCIStr("c_1"), Offset: 0, }, { ID: 2, - Name: pmodel.NewCIStr("c_2"), + Name: ast.NewCIStr("c_2"), Offset: 1, }, }, diff --git a/lightning/pkg/importer/table_import_test.go b/lightning/pkg/importer/table_import_test.go index 80d1f0206f956..f7456b8b64a54 100644 --- a/lightning/pkg/importer/table_import_test.go +++ b/lightning/pkg/importer/table_import_test.go @@ -55,7 +55,6 @@ import ( "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser" "github.com/pingcap/tidb/pkg/parser/ast" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/table/tables" "github.com/pingcap/tidb/pkg/types" @@ -1560,12 +1559,12 @@ func (s *tableRestoreSuite) TestSchemaIsValid() { Columns: []*model.ColumnInfo{ { // colA has the default value - Name: pmodel.NewCIStr("colA"), + Name: ast.NewCIStr("colA"), DefaultIsExpr: true, }, { // colB doesn't have the default value - Name: pmodel.NewCIStr("colB"), + Name: ast.NewCIStr("colB"), FieldType: types.NewFieldTypeBuilder().SetType(0).SetFlag(1).Build(), }, }, @@ -1609,7 +1608,7 @@ func (s *tableRestoreSuite) TestSchemaIsValid() { Columns: []*model.ColumnInfo{ { // colB has the default value - Name: pmodel.NewCIStr("colB"), + Name: ast.NewCIStr("colB"), DefaultIsExpr: true, }, }, @@ -1660,7 +1659,7 @@ func (s *tableRestoreSuite) TestSchemaIsValid() { Columns: []*model.ColumnInfo{ { // colB has the default value - Name: pmodel.NewCIStr("colB"), + Name: ast.NewCIStr("colB"), DefaultIsExpr: true, }, }, @@ -1712,12 +1711,12 @@ func (s *tableRestoreSuite) TestSchemaIsValid() { Columns: []*model.ColumnInfo{ { // colB has the default value - Name: pmodel.NewCIStr("colB"), + Name: ast.NewCIStr("colB"), DefaultIsExpr: true, }, { // colC doesn't have the default value - Name: pmodel.NewCIStr("colC"), + Name: ast.NewCIStr("colC"), FieldType: types.NewFieldTypeBuilder().SetType(0).SetFlag(1).Build(), }, }, @@ -1768,12 +1767,12 @@ func (s *tableRestoreSuite) TestSchemaIsValid() { Columns: []*model.ColumnInfo{ { // colB doesn't have the default value - Name: pmodel.NewCIStr("colB"), + Name: ast.NewCIStr("colB"), FieldType: types.NewFieldTypeBuilder().SetType(0).SetFlag(1).Build(), }, { // colC has the default value - Name: pmodel.NewCIStr("colC"), + Name: ast.NewCIStr("colC"), DefaultIsExpr: true, }, }, @@ -1858,7 +1857,7 @@ func (s *tableRestoreSuite) TestSchemaIsValid() { Columns: []*model.ColumnInfo{ { // colB has the default value - Name: pmodel.NewCIStr("colB"), + Name: ast.NewCIStr("colB"), DefaultIsExpr: true, }, }, @@ -1917,10 +1916,10 @@ func (s *tableRestoreSuite) TestSchemaIsValid() { Core: &model.TableInfo{ Columns: []*model.ColumnInfo{ { - Name: pmodel.NewCIStr("colA"), + Name: ast.NewCIStr("colA"), }, { - Name: pmodel.NewCIStr("colB"), + Name: ast.NewCIStr("colB"), }, }, }, @@ -1976,10 +1975,10 @@ func (s *tableRestoreSuite) TestSchemaIsValid() { Core: &model.TableInfo{ Columns: []*model.ColumnInfo{ { - Name: pmodel.NewCIStr("colA"), + Name: ast.NewCIStr("colA"), }, { - Name: pmodel.NewCIStr("colB"), + Name: ast.NewCIStr("colB"), }, }, }, @@ -2024,10 +2023,10 @@ func (s *tableRestoreSuite) TestSchemaIsValid() { Core: &model.TableInfo{ Columns: []*model.ColumnInfo{ { - Name: pmodel.NewCIStr("colA"), + Name: ast.NewCIStr("colA"), }, { - Name: pmodel.NewCIStr("colB"), + Name: ast.NewCIStr("colB"), }, }, }, @@ -2072,10 +2071,10 @@ func (s *tableRestoreSuite) TestSchemaIsValid() { Core: &model.TableInfo{ Columns: []*model.ColumnInfo{ { - Name: pmodel.NewCIStr("colA"), + Name: ast.NewCIStr("colA"), }, { - Name: pmodel.NewCIStr("colB"), + Name: ast.NewCIStr("colB"), }, }, }, @@ -2137,14 +2136,14 @@ func (s *tableRestoreSuite) TestSchemaIsValid() { Core: &model.TableInfo{ Columns: []*model.ColumnInfo{ { - Name: pmodel.NewCIStr("colA"), + Name: ast.NewCIStr("colA"), }, { - Name: pmodel.NewCIStr("colB"), + Name: ast.NewCIStr("colB"), DefaultIsExpr: true, }, { - Name: pmodel.NewCIStr("colC"), + Name: ast.NewCIStr("colC"), }, }, }, @@ -2260,11 +2259,11 @@ func (s *tableRestoreSuite) TestGBKEncodedSchemaIsValid() { Core: &model.TableInfo{ Columns: []*model.ColumnInfo{ { - Name: pmodel.NewCIStr("colA"), + Name: ast.NewCIStr("colA"), FieldType: types.NewFieldTypeBuilder().SetType(0).SetFlag(1).Build(), }, { - Name: pmodel.NewCIStr("colB"), + Name: ast.NewCIStr("colB"), FieldType: types.NewFieldTypeBuilder().SetType(0).SetFlag(1).Build(), }, }, diff --git a/pkg/autoid_service/BUILD.bazel b/pkg/autoid_service/BUILD.bazel index a5858539f6b0b..144875f8af71e 100644 --- a/pkg/autoid_service/BUILD.bazel +++ b/pkg/autoid_service/BUILD.bazel @@ -34,7 +34,7 @@ go_test( flaky = True, shard_count = 3, deps = [ - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/store/mockstore", "//pkg/testkit", "@com_github_pingcap_kvproto//pkg/autoid", diff --git a/pkg/autoid_service/autoid_test.go b/pkg/autoid_service/autoid_test.go index 8dd8f51d92618..5f9a40d5a8b96 100644 --- a/pkg/autoid_service/autoid_test.go +++ b/pkg/autoid_service/autoid_test.go @@ -24,7 +24,7 @@ import ( "github.com/pingcap/kvproto/pkg/autoid" "github.com/pingcap/kvproto/pkg/keyspacepb" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/store/mockstore" "github.com/pingcap/tidb/pkg/testkit" "github.com/stretchr/testify/require" @@ -68,10 +68,10 @@ func TestConcurrent(t *testing.T) { tk.MustExec("use test") tk.MustExec("create table t1 (id int key auto_increment);") is := dom.InfoSchema() - dbInfo, ok := is.SchemaByName(model.NewCIStr("test")) + dbInfo, ok := is.SchemaByName(ast.NewCIStr("test")) require.True(t, ok) - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t1")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t1")) require.NoError(t, err) tbInfo := tbl.Meta() @@ -164,10 +164,10 @@ func testAPIWithKeyspace(t *testing.T, keyspaceMeta *keyspacepb.KeyspaceMeta) { tk.MustExec("use test") tk.MustExec("create table t (id int key auto_increment);") is := dom.InfoSchema() - dbInfo, ok := is.SchemaByName(model.NewCIStr("test")) + dbInfo, ok := is.SchemaByName(ast.NewCIStr("test")) require.True(t, ok) - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tbInfo := tbl.Meta() diff --git a/pkg/bindinfo/tests/BUILD.bazel b/pkg/bindinfo/tests/BUILD.bazel index 27215c38c9f28..8db1e1252f208 100644 --- a/pkg/bindinfo/tests/BUILD.bazel +++ b/pkg/bindinfo/tests/BUILD.bazel @@ -16,7 +16,7 @@ go_test( "//pkg/domain", "//pkg/meta/model", "//pkg/parser", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/parser/mysql", "//pkg/session/types", "//pkg/testkit", diff --git a/pkg/bindinfo/tests/bind_test.go b/pkg/bindinfo/tests/bind_test.go index 83cb0749c9acb..98c4a11a4cbce 100644 --- a/pkg/bindinfo/tests/bind_test.go +++ b/pkg/bindinfo/tests/bind_test.go @@ -26,7 +26,7 @@ import ( "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/util" utilparser "github.com/pingcap/tidb/pkg/util/parser" @@ -325,7 +325,7 @@ func TestBindingWithIsolationRead(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ Count: 1, diff --git a/pkg/ddl/BUILD.bazel b/pkg/ddl/BUILD.bazel index 37769bb349369..4f1e9c4aacc21 100644 --- a/pkg/ddl/BUILD.bazel +++ b/pkg/ddl/BUILD.bazel @@ -120,7 +120,6 @@ go_library( "//pkg/parser/ast", "//pkg/parser/charset", "//pkg/parser/format", - "//pkg/parser/model", "//pkg/parser/mysql", "//pkg/parser/opcode", "//pkg/parser/terror", @@ -316,7 +315,6 @@ go_test( "//pkg/parser/ast", "//pkg/parser/auth", "//pkg/parser/charset", - "//pkg/parser/model", "//pkg/parser/mysql", "//pkg/parser/terror", "//pkg/parser/types", diff --git a/pkg/ddl/add_column.go b/pkg/ddl/add_column.go index b246c20685db1..5244d3d825f09 100644 --- a/pkg/ddl/add_column.go +++ b/pkg/ddl/add_column.go @@ -36,7 +36,6 @@ import ( "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/charset" "github.com/pingcap/tidb/pkg/parser/format" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" field_types "github.com/pingcap/tidb/pkg/parser/types" @@ -543,7 +542,7 @@ func columnDefToCol(ctx *metabuild.Context, offset int, colDef *ast.ColumnDef, o } col.GeneratedExprString = sb.String() col.GeneratedStored = v.Stored - _, dependColNames, err := findDependedColumnNames(pmodel.NewCIStr(""), pmodel.NewCIStr(""), colDef) + _, dependColNames, err := findDependedColumnNames(ast.NewCIStr(""), ast.NewCIStr(""), colDef) if err != nil { return nil, nil, errors.Trace(err) } diff --git a/pkg/ddl/backfilling.go b/pkg/ddl/backfilling.go index f98897325cb7a..c6d14004ca15b 100644 --- a/pkg/ddl/backfilling.go +++ b/pkg/ddl/backfilling.go @@ -38,7 +38,7 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/metrics" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "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/table" @@ -679,7 +679,7 @@ func loadDDLReorgVars(ctx context.Context, sessPool *sess.Pool) error { return ddlutil.LoadDDLReorgVars(ctx, sCtx) } -func makeupDecodeColMap(dbName pmodel.CIStr, t table.Table) (map[int64]decoder.Column, error) { +func makeupDecodeColMap(dbName ast.CIStr, t table.Table) (map[int64]decoder.Column, error) { writableColInfos := make([]*model.ColumnInfo, 0, len(t.WritableCols())) for _, col := range t.WritableCols() { writableColInfos = append(writableColInfos, col.ColumnInfo) diff --git a/pkg/ddl/backfilling_dist_scheduler_test.go b/pkg/ddl/backfilling_dist_scheduler_test.go index e5d5c01b776e8..d279a947d6fe8 100644 --- a/pkg/ddl/backfilling_dist_scheduler_test.go +++ b/pkg/ddl/backfilling_dist_scheduler_test.go @@ -30,7 +30,7 @@ import ( "github.com/pingcap/tidb/pkg/lightning/backend/external" "github.com/pingcap/tidb/pkg/meta" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/testkit" "github.com/stretchr/testify/require" @@ -53,7 +53,7 @@ func TestBackfillingSchedulerLocalMode(t *testing.T) { "PARTITION p2 VALUES LESS THAN (1000),\n" + "PARTITION p3 VALUES LESS THAN MAXVALUE\n);") task := createAddIndexTask(t, dom, "test", "tp1", proto.Backfill, false) - tbl, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("tp1")) + tbl, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("tp1")) require.NoError(t, err) tblInfo := tbl.Meta() @@ -284,9 +284,9 @@ func createAddIndexTask(t *testing.T, tblName string, taskType proto.TaskType, useGlobalSort bool) *proto.Task { - db, ok := dom.InfoSchema().SchemaByName(pmodel.NewCIStr(dbName)) + db, ok := dom.InfoSchema().SchemaByName(ast.NewCIStr(dbName)) require.True(t, ok) - tbl, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr(dbName), pmodel.NewCIStr(tblName)) + tbl, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr(dbName), ast.NewCIStr(tblName)) require.NoError(t, err) tblInfo := tbl.Meta() defaultSQLMode, err := mysql.GetSQLMode(mysql.DefaultSQLMode) diff --git a/pkg/ddl/bdr_test.go b/pkg/ddl/bdr_test.go index bf3997af1dd75..000e7d8192295 100644 --- a/pkg/ddl/bdr_test.go +++ b/pkg/ddl/bdr_test.go @@ -20,7 +20,6 @@ import ( "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/ast" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/types" "github.com/stretchr/testify/assert" @@ -479,7 +478,7 @@ func TestDeniedByBDR(t *testing.T) { indexArgs := &model.ModifyIndexArgs{ IndexArgs: []*model.IndexArg{{ Global: false, - IndexName: pmodel.NewCIStr("idx1"), + IndexName: ast.NewCIStr("idx1"), IndexPartSpecifications: []*ast.IndexPartSpecification{{Length: 2}}, IndexOption: &ast.IndexOption{}, HiddenCols: nil, diff --git a/pkg/ddl/bench_test.go b/pkg/ddl/bench_test.go index bfd2f47a1a5b1..d4d95dc0c45bb 100644 --- a/pkg/ddl/bench_test.go +++ b/pkg/ddl/bench_test.go @@ -22,7 +22,7 @@ import ( "github.com/pingcap/tidb/pkg/ddl/copr" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/table/tables" "github.com/pingcap/tidb/pkg/testkit" @@ -41,7 +41,7 @@ func BenchmarkExtractDatumByOffsets(b *testing.B) { for i := 0; i < 8; i++ { tk.MustExec("insert into t values (?, ?)", i, i) } - tbl, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(b, err) tblInfo := tbl.Meta() idxInfo := tblInfo.FindIndexByName("idx") @@ -81,7 +81,7 @@ func BenchmarkGenerateIndexKV(b *testing.B) { for i := 0; i < 8; i++ { tk.MustExec("insert into t values (?, ?)", i, i) } - tbl, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(b, err) tblInfo := tbl.Meta() idxInfo := tblInfo.FindIndexByName("idx") diff --git a/pkg/ddl/column.go b/pkg/ddl/column.go index 237538456886e..e609e6511c957 100644 --- a/pkg/ddl/column.go +++ b/pkg/ddl/column.go @@ -36,7 +36,6 @@ import ( "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/metrics" "github.com/pingcap/tidb/pkg/parser/ast" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/sessionctx" @@ -266,7 +265,7 @@ func checkDropColumn(jobCtx *jobContext, job *model.Job) (*model.TableInfo, *mod return tblInfo, colInfo, idxInfos, false, nil } -func isDroppableColumn(tblInfo *model.TableInfo, colName pmodel.CIStr) error { +func isDroppableColumn(tblInfo *model.TableInfo, colName ast.CIStr) error { if ok, dep, isHidden := hasDependentByGeneratedColumn(tblInfo, colName); ok { if isHidden { return dbterror.ErrDependentByFunctionalIndex.GenWithStackByArgs(dep) @@ -300,7 +299,7 @@ func onSetDefaultValue(jobCtx *jobContext, job *model.Job) (ver int64, _ error) return updateColumnDefaultValue(jobCtx, job, newCol, &newCol.Name) } -func setIdxIDName(idxInfo *model.IndexInfo, newID int64, newName pmodel.CIStr) { +func setIdxIDName(idxInfo *model.IndexInfo, newID int64, newName ast.CIStr) { idxInfo.ID = newID idxInfo.Name = newName } @@ -334,7 +333,7 @@ func removeChangingColAndIdxs(tblInfo *model.TableInfo, changingColID int64) { } func replaceOldColumn(tblInfo *model.TableInfo, oldCol, changingCol *model.ColumnInfo, - newName pmodel.CIStr) *model.ColumnInfo { + newName ast.CIStr) *model.ColumnInfo { tblInfo.MoveColumnInfo(changingCol.Offset, len(tblInfo.Columns)-1) changingCol = updateChangingCol(changingCol, newName, oldCol.Offset) tblInfo.Columns[oldCol.Offset] = changingCol @@ -365,7 +364,7 @@ func replaceOldIndexes(tblInfo *model.TableInfo, changingIdxs []*model.IndexInfo idxName := getChangingIndexOriginName(cIdx) for i, idx := range tblInfo.Indices { if strings.EqualFold(idxName, idx.Name.O) { - cIdx.Name = pmodel.NewCIStr(idxName) + cIdx.Name = ast.NewCIStr(idxName) tblInfo.Indices[i] = cIdx break } @@ -375,7 +374,7 @@ func replaceOldIndexes(tblInfo *model.TableInfo, changingIdxs []*model.IndexInfo // updateNewIdxColsNameOffset updates the name&offset of the index column. func updateNewIdxColsNameOffset(changingIdxs []*model.IndexInfo, - oldName pmodel.CIStr, changingCol *model.ColumnInfo) { + oldName ast.CIStr, changingCol *model.ColumnInfo) { for _, idx := range changingIdxs { for _, col := range idx.Columns { if col.Name.L == oldName.L { @@ -386,7 +385,7 @@ func updateNewIdxColsNameOffset(changingIdxs []*model.IndexInfo, } // filterIndexesToRemove filters out the indexes that can be removed. -func filterIndexesToRemove(changingIdxs []*model.IndexInfo, colName pmodel.CIStr, tblInfo *model.TableInfo) []*model.IndexInfo { +func filterIndexesToRemove(changingIdxs []*model.IndexInfo, colName ast.CIStr, tblInfo *model.TableInfo) []*model.IndexInfo { indexesToRemove := make([]*model.IndexInfo, 0, len(changingIdxs)) for _, idx := range changingIdxs { var hasOtherChangingCol bool @@ -407,7 +406,7 @@ func filterIndexesToRemove(changingIdxs []*model.IndexInfo, colName pmodel.CIStr return indexesToRemove } -func updateChangingCol(col *model.ColumnInfo, newName pmodel.CIStr, newOffset int) *model.ColumnInfo { +func updateChangingCol(col *model.ColumnInfo, newName ast.CIStr, newOffset int) *model.ColumnInfo { col.Name = newName col.ChangeStateInfo = nil col.Offset = newOffset @@ -1002,7 +1001,7 @@ func applyNewAutoRandomBits(jobCtx *jobContext, dbInfo *model.DBInfo, // checkForNullValue ensure there are no null values of the column of this table. // `isDataTruncated` indicates whether the new field and the old field type are the same, in order to be compatible with mysql. -func checkForNullValue(ctx context.Context, sctx sessionctx.Context, isDataTruncated bool, schema, table pmodel.CIStr, newCol *model.ColumnInfo, oldCols ...*model.ColumnInfo) error { +func checkForNullValue(ctx context.Context, sctx sessionctx.Context, isDataTruncated bool, schema, table ast.CIStr, newCol *model.ColumnInfo, oldCols ...*model.ColumnInfo) error { needCheckNullValue := false for _, oldCol := range oldCols { if oldCol.GetType() != mysql.TypeTimestamp && newCol.GetType() == mysql.TypeTimestamp { @@ -1043,7 +1042,7 @@ func checkForNullValue(ctx context.Context, sctx sessionctx.Context, isDataTrunc return nil } -func updateColumnDefaultValue(jobCtx *jobContext, job *model.Job, newCol *model.ColumnInfo, oldColName *pmodel.CIStr) (ver int64, _ error) { +func updateColumnDefaultValue(jobCtx *jobContext, job *model.Job, newCol *model.ColumnInfo, oldColName *ast.CIStr) (ver int64, _ error) { tblInfo, err := GetTableInfoAndCancelFaultJob(jobCtx.metaMut, job, job.SchemaID) if err != nil { return ver, errors.Trace(err) @@ -1329,7 +1328,7 @@ func getChangingColumnOriginName(changingColumn *model.ColumnInfo) string { return columnName[:pos] } -func getExpressionIndexOriginName(originalName pmodel.CIStr) string { +func getExpressionIndexOriginName(originalName ast.CIStr) string { columnName := strings.TrimPrefix(originalName.O, expressionIndexPrefix+"_") var pos int if pos = strings.LastIndex(columnName, "_"); pos == -1 { diff --git a/pkg/ddl/column_modify_test.go b/pkg/ddl/column_modify_test.go index 3b2f01e641894..04f0c86ca042c 100644 --- a/pkg/ddl/column_modify_test.go +++ b/pkg/ddl/column_modify_test.go @@ -28,7 +28,7 @@ import ( "github.com/pingcap/tidb/pkg/errno" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/sessiontxn" "github.com/pingcap/tidb/pkg/store/mockstore" @@ -160,7 +160,7 @@ AddLoop: defer tk.MustExec("drop table test_on_update_c;") tk.MustExec("alter table test_on_update_c add column c3 timestamp null default '2017-02-11' on update current_timestamp;") is := domain.GetDomain(tk.Session()).InfoSchema() - tbl, err = is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("test_on_update_c")) + tbl, err = is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("test_on_update_c")) require.NoError(t, err) tblInfo := tbl.Meta() colC := tblInfo.Columns[2] @@ -170,7 +170,7 @@ AddLoop: tk.MustExec("create table test_on_update_d (c1 int, c2 datetime);") tk.MustExec("alter table test_on_update_d add column c3 datetime on update current_timestamp;") is = domain.GetDomain(tk.Session()).InfoSchema() - tbl, err = is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("test_on_update_d")) + tbl, err = is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("test_on_update_d")) require.NoError(t, err) tblInfo = tbl.Meta() colC = tblInfo.Columns[2] @@ -305,7 +305,7 @@ func TestChangeColumn(t *testing.T) { // for no default flag tk.MustExec("alter table t3 change d dd bigint not null") is := domain.GetDomain(tk.Session()).InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t3")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t3")) require.NoError(t, err) tblInfo := tbl.Meta() colD := tblInfo.Columns[2] @@ -313,7 +313,7 @@ func TestChangeColumn(t *testing.T) { // for the following definitions: 'not null', 'null', 'default value' and 'comment' tk.MustExec("alter table t3 change b b varchar(20) null default 'c' comment 'my comment'") is = domain.GetDomain(tk.Session()).InfoSchema() - tbl, err = is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t3")) + tbl, err = is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t3")) require.NoError(t, err) tblInfo = tbl.Meta() colB := tblInfo.Columns[1] @@ -325,7 +325,7 @@ func TestChangeColumn(t *testing.T) { tk.MustExec("alter table t3 add column c timestamp not null") tk.MustExec("alter table t3 change c c timestamp null default '2017-02-11' comment 'col c comment' on update current_timestamp") is = domain.GetDomain(tk.Session()).InfoSchema() - tbl, err = is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t3")) + tbl, err = is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t3")) require.NoError(t, err) tblInfo = tbl.Meta() colC := tblInfo.Columns[3] @@ -339,7 +339,7 @@ func TestChangeColumn(t *testing.T) { tk.MustExec("create table t (k char(10), v int, INDEX(k(7)));") tk.MustExec("alter table t change column k k tinytext") is = domain.GetDomain(tk.Session()).InfoSchema() - tbl, err = is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err = is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) // for failing tests @@ -381,7 +381,7 @@ func TestVirtualColumnDDL(t *testing.T) { tk.MustExec("use test") tk.MustExec(`create global temporary table test_gv_ddl(a int, b int as (a+8) virtual, c int as (b + 2) stored) on commit delete rows;`) is := sessiontxn.GetTxnManager(tk.Session()).GetTxnInfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("test_gv_ddl")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("test_gv_ddl")) require.NoError(t, err) testCases := []struct { generatedExprString string @@ -405,7 +405,7 @@ func TestVirtualColumnDDL(t *testing.T) { // for local temporary table tk.MustExec(`create temporary table test_local_gv_ddl(a int, b int as (a+8) virtual, c int as (b + 2) stored);`) is = sessiontxn.GetTxnManager(tk.Session()).GetTxnInfoSchema() - tbl, err = is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("test_local_gv_ddl")) + tbl, err = is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("test_local_gv_ddl")) require.NoError(t, err) for i, column := range tbl.Meta().Columns { require.Equal(t, testCases[i].generatedExprString, column.GeneratedExprString) diff --git a/pkg/ddl/column_test.go b/pkg/ddl/column_test.go index 93e3242e5acd0..5c8241eb48018 100644 --- a/pkg/ddl/column_test.go +++ b/pkg/ddl/column_test.go @@ -26,7 +26,7 @@ import ( "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "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/store/mockstore" @@ -107,7 +107,7 @@ func testDropTable(tk *testkit.TestKit, t *testing.T, dbName, tblName string, do idi, _ := strconv.Atoi(tk.MustQuery("admin show ddl jobs 1;").Rows()[0][0].(string)) id := int64(idi) require.NoError(t, dom.Reload()) - _, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr(dbName), pmodel.NewCIStr(tblName)) + _, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr(dbName), ast.NewCIStr(tblName)) require.Error(t, err) return id } diff --git a/pkg/ddl/constraint.go b/pkg/ddl/constraint.go index eabb2bf607936..8b7cc26c68509 100644 --- a/pkg/ddl/constraint.go +++ b/pkg/ddl/constraint.go @@ -27,7 +27,6 @@ import ( "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/format" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/util/dbterror" @@ -302,8 +301,8 @@ func allocateConstraintID(tblInfo *model.TableInfo) int64 { return tblInfo.MaxConstraintID } -func buildConstraintInfo(tblInfo *model.TableInfo, dependedCols []pmodel.CIStr, constr *ast.Constraint, state model.SchemaState) (*model.ConstraintInfo, error) { - constraintName := pmodel.NewCIStr(constr.Name) +func buildConstraintInfo(tblInfo *model.TableInfo, dependedCols []ast.CIStr, constr *ast.Constraint, state model.SchemaState) (*model.ConstraintInfo, error) { + constraintName := ast.NewCIStr(constr.Name) if err := checkTooLongConstraint(constraintName); err != nil { return nil, errors.Trace(err) } @@ -334,7 +333,7 @@ func buildConstraintInfo(tblInfo *model.TableInfo, dependedCols []pmodel.CIStr, return constraintInfo, nil } -func checkTooLongConstraint(constr pmodel.CIStr) error { +func checkTooLongConstraint(constr ast.CIStr) error { if len(constr.L) > mysql.MaxConstraintIdentifierLen { return dbterror.ErrTooLongIdent.GenWithStackByArgs(constr) } @@ -404,13 +403,13 @@ func setNameForConstraintInfo(tableLowerName string, namesMap map[string]bool, i cnt++ constrName = fmt.Sprintf("%s%d", constraintPrefix, cnt) } - constrInfo.Name = pmodel.NewCIStr(constrName) + constrInfo.Name = ast.NewCIStr(constrName) } } } // IsColumnDroppableWithCheckConstraint check whether the column in check-constraint whose dependent col is more than 1 -func IsColumnDroppableWithCheckConstraint(col pmodel.CIStr, tblInfo *model.TableInfo) error { +func IsColumnDroppableWithCheckConstraint(col ast.CIStr, tblInfo *model.TableInfo) error { for _, cons := range tblInfo.Constraints { if len(cons.ConstraintCols) > 1 { for _, colName := range cons.ConstraintCols { @@ -424,7 +423,7 @@ func IsColumnDroppableWithCheckConstraint(col pmodel.CIStr, tblInfo *model.Table } // IsColumnRenameableWithCheckConstraint check whether the column is referenced in check-constraint -func IsColumnRenameableWithCheckConstraint(col pmodel.CIStr, tblInfo *model.TableInfo) error { +func IsColumnRenameableWithCheckConstraint(col ast.CIStr, tblInfo *model.TableInfo) error { for _, cons := range tblInfo.Constraints { for _, colName := range cons.ConstraintCols { if colName.L == col.L { diff --git a/pkg/ddl/copr/BUILD.bazel b/pkg/ddl/copr/BUILD.bazel index 71f40bcb9ded6..2a7f0cbc5352b 100644 --- a/pkg/ddl/copr/BUILD.bazel +++ b/pkg/ddl/copr/BUILD.bazel @@ -11,7 +11,7 @@ go_library( "//pkg/expression/exprctx", "//pkg/infoschema", "//pkg/meta/model", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/table/tables", "//pkg/types", "@com_github_pingcap_errors//:errors", @@ -29,7 +29,7 @@ go_test( "//pkg/expression", "//pkg/expression/exprstatic", "//pkg/meta/model", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/parser/mysql", "//pkg/types", "//pkg/util/mock", diff --git a/pkg/ddl/copr/copr_ctx.go b/pkg/ddl/copr/copr_ctx.go index e4736ffa78c41..233a9bc92054b 100644 --- a/pkg/ddl/copr/copr_ctx.go +++ b/pkg/ddl/copr/copr_ctx.go @@ -22,7 +22,7 @@ import ( // make sure mock.MockInfoschema is initialized to make sure the test pass _ "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/table/tables" "github.com/pingcap/tidb/pkg/types" ) @@ -123,7 +123,7 @@ func NewCopContextBase( } expColInfos, _, err := expression.ColumnInfos2ColumnsAndNames(exprCtx, - pmodel.CIStr{} /* unused */, tblInfo.Name, colInfos, tblInfo) + ast.CIStr{} /* unused */, tblInfo.Name, colInfos, tblInfo) if err != nil { return nil, err } diff --git a/pkg/ddl/copr/copr_ctx_test.go b/pkg/ddl/copr/copr_ctx_test.go index 34cde816a0738..bb807febca526 100644 --- a/pkg/ddl/copr/copr_ctx_test.go +++ b/pkg/ddl/copr/copr_ctx_test.go @@ -21,7 +21,7 @@ import ( "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/expression/exprstatic" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/mock" @@ -35,7 +35,7 @@ func TestNewCopContextSingleIndex(t *testing.T) { mockColInfos = append(mockColInfos, &model.ColumnInfo{ ID: int64(i), Offset: i, - Name: pmodel.NewCIStr(fmt.Sprintf("c%d", i)), + Name: ast.NewCIStr(fmt.Sprintf("c%d", i)), FieldType: *types.NewFieldType(1), State: model.StatePublic, }) @@ -69,18 +69,18 @@ func TestNewCopContextSingleIndex(t *testing.T) { var idxCols []*model.IndexColumn for _, cn := range tt.cols { idxCols = append(idxCols, &model.IndexColumn{ - Name: pmodel.NewCIStr(cn), + Name: ast.NewCIStr(cn), Offset: findColByName(cn).Offset, }) } mockIdxInfo := &model.IndexInfo{ ID: int64(i), - Name: pmodel.NewCIStr(fmt.Sprintf("i%d", i)), + Name: ast.NewCIStr(fmt.Sprintf("i%d", i)), Columns: idxCols, State: model.StatePublic, } mockTableInfo := &model.TableInfo{ - Name: pmodel.NewCIStr("t"), + Name: ast.NewCIStr("t"), Columns: mockColInfos, Indices: []*model.IndexInfo{mockIdxInfo}, PKIsHandle: tt.pkType == pkTypePKHandle, @@ -93,11 +93,11 @@ func TestNewCopContextSingleIndex(t *testing.T) { mockTableInfo.Indices = append(mockTableInfo.Indices, &model.IndexInfo{ Columns: []*model.IndexColumn{ { - Name: pmodel.NewCIStr("c2"), + Name: ast.NewCIStr("c2"), Offset: 2, }, { - Name: pmodel.NewCIStr("c4"), + Name: ast.NewCIStr("c4"), Offset: 4, }, }, diff --git a/pkg/ddl/create_table.go b/pkg/ddl/create_table.go index 135add5676c45..cbe7b9dd23a12 100644 --- a/pkg/ddl/create_table.go +++ b/pkg/ddl/create_table.go @@ -40,7 +40,6 @@ import ( "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/format" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" field_types "github.com/pingcap/tidb/pkg/parser/types" "github.com/pingcap/tidb/pkg/sessionctx/variable" @@ -383,7 +382,7 @@ func findTableIDFromInfoSchema(is infoschema.InfoSchema, schemaID int64, tableNa if !ok { return 0, infoschema.ErrDatabaseNotExists.GenWithStackByArgs("") } - tbl, err := is.TableByName(context.Background(), schema.Name, pmodel.NewCIStr(tableName)) + tbl, err := is.TableByName(context.Background(), schema.Name, ast.NewCIStr(tableName)) if err != nil { return 0, err } @@ -474,7 +473,7 @@ func checkTableInfoValidWithStmt(ctx *metabuild.Context, tbInfo *model.TableInfo return nil } -func checkGeneratedColumn(ctx *metabuild.Context, schemaName pmodel.CIStr, tableName pmodel.CIStr, colDefs []*ast.ColumnDef) error { +func checkGeneratedColumn(ctx *metabuild.Context, schemaName ast.CIStr, tableName ast.CIStr, colDefs []*ast.ColumnDef) error { var colName2Generation = make(map[string]columnGenerationInDDL, len(colDefs)) var exists bool var autoIncrementColumn string @@ -527,7 +526,7 @@ func checkGeneratedColumn(ctx *metabuild.Context, schemaName pmodel.CIStr, table return nil } -func checkVectorIndexIfNeedTiFlashReplica(store kv.Storage, dbName pmodel.CIStr, tblInfo *model.TableInfo) error { +func checkVectorIndexIfNeedTiFlashReplica(store kv.Storage, dbName ast.CIStr, tblInfo *model.TableInfo) error { var hasVectorIndex bool for _, idx := range tblInfo.Indices { if idx.VectorInfo != nil { @@ -570,7 +569,7 @@ func checkVectorIndexIfNeedTiFlashReplica(store kv.Storage, dbName pmodel.CIStr, // name length and column count. // (checkTableInfoValid is also used in repairing objects which don't perform // these checks. Perhaps the two functions should be merged together regardless?) -func checkTableInfoValidExtra(ec errctx.Context, store kv.Storage, dbName pmodel.CIStr, tbInfo *model.TableInfo) error { +func checkTableInfoValidExtra(ec errctx.Context, store kv.Storage, dbName ast.CIStr, tbInfo *model.TableInfo) error { if err := checkTooLongTable(tbInfo.Name); err != nil { return err } @@ -884,7 +883,7 @@ func handleTableOptions(options []*ast.TableOption, tbInfo *model.TableInfo) err // We don't handle charset and collate here since they're handled in `GetCharsetAndCollateInTableOption`. case ast.TableOptionPlacementPolicy: tbInfo.PlacementPolicyRef = &model.PolicyRefInfo{ - Name: pmodel.NewCIStr(op.StrValue), + Name: ast.NewCIStr(op.StrValue), } case ast.TableOptionTTL, ast.TableOptionTTLEnable, ast.TableOptionTTLJobInterval: if ttlOptionsHandled { @@ -1017,7 +1016,7 @@ func setEmptyConstraintName(namesMap map[string]bool, constr *ast.Constraint) { } } -func checkConstraintNames(tableName pmodel.CIStr, constraints []*ast.Constraint) error { +func checkConstraintNames(tableName ast.CIStr, constraints []*ast.Constraint) error { constrNames := map[string]bool{} fkNames := map[string]bool{} @@ -1196,7 +1195,7 @@ func BuildTableInfoWithLike(ident ast.Ident, referTblInfo *model.TableInfo, s *a func renameCheckConstraint(tblInfo *model.TableInfo) { for _, cons := range tblInfo.Constraints { - cons.Name = pmodel.NewCIStr("") + cons.Name = ast.NewCIStr("") cons.Table = tblInfo.Name } setNameForConstraintInfo(tblInfo.Name.L, map[string]bool{}, tblInfo.Constraints) @@ -1205,7 +1204,7 @@ func renameCheckConstraint(tblInfo *model.TableInfo) { // BuildTableInfo creates a TableInfo. func BuildTableInfo( ctx *metabuild.Context, - tableName pmodel.CIStr, + tableName ast.CIStr, cols []*table.Column, constraints []*ast.Constraint, charset string, @@ -1230,7 +1229,7 @@ func BuildTableInfo( var hiddenCols []*model.ColumnInfo if constr.Tp != ast.ConstraintVector { // Build hidden columns if necessary. - hiddenCols, err = buildHiddenColumnInfoWithCheck(ctx, constr.Keys, pmodel.NewCIStr(constr.Name), tbInfo, tblColumns) + hiddenCols, err = buildHiddenColumnInfoWithCheck(ctx, constr.Keys, ast.NewCIStr(constr.Name), tbInfo, tblColumns) if err != nil { return nil, err } @@ -1243,17 +1242,17 @@ func BuildTableInfo( tblColumns = append(tblColumns, table.ToColumn(hiddenCol)) } // Check clustered on non-primary key. - if constr.Option != nil && constr.Option.PrimaryKeyTp != pmodel.PrimaryKeyTypeDefault && + if constr.Option != nil && constr.Option.PrimaryKeyTp != ast.PrimaryKeyTypeDefault && constr.Tp != ast.ConstraintPrimaryKey { return nil, dbterror.ErrUnsupportedClusteredSecondaryKey } if constr.Tp == ast.ConstraintForeignKey { - var fkName pmodel.CIStr + var fkName ast.CIStr foreignKeyID++ if constr.Name != "" { - fkName = pmodel.NewCIStr(constr.Name) + fkName = ast.NewCIStr(constr.Name) } else { - fkName = pmodel.NewCIStr(fmt.Sprintf("fk_%d", foreignKeyID)) + fkName = ast.NewCIStr(fmt.Sprintf("fk_%d", foreignKeyID)) } if model.FindFKInfoByName(tbInfo.ForeignKeys, fkName.L) != nil { return nil, infoschema.ErrCannotAddForeign @@ -1334,16 +1333,16 @@ func BuildTableInfo( if ok, err := table.IsSupportedExpr(constr); !ok { return nil, err } - var dependedCols []pmodel.CIStr + var dependedCols []ast.CIStr dependedColsMap := findDependentColsInExpr(constr.Expr) if !constr.InColumn { - dependedCols = make([]pmodel.CIStr, 0, len(dependedColsMap)) + dependedCols = make([]ast.CIStr, 0, len(dependedColsMap)) for k := range dependedColsMap { if _, ok := existedColsMap[k]; !ok { // The table constraint depended on a non-existed column. return nil, dbterror.ErrTableCheckConstraintReferUnknown.GenWithStackByArgs(constr.Name, k) } - dependedCols = append(dependedCols, pmodel.NewCIStr(k)) + dependedCols = append(dependedCols, ast.NewCIStr(k)) } } else { // Check the column-type constraint dependency. @@ -1359,7 +1358,7 @@ func BuildTableInfo( if _, ok := dependedColsMap[constr.InColumnName]; !ok { return nil, dbterror.ErrColumnCheckConstraintReferOther.GenWithStackByArgs(constr.Name) } - dependedCols = []pmodel.CIStr{pmodel.NewCIStr(constr.InColumnName)} + dependedCols = []ast.CIStr{ast.NewCIStr(constr.InColumnName)} } } // check auto-increment column @@ -1388,7 +1387,7 @@ func BuildTableInfo( idxInfo, err := BuildIndexInfo( ctx, tbInfo, - pmodel.NewCIStr(indexName), + ast.NewCIStr(indexName), primary, unique, vector, @@ -1417,7 +1416,7 @@ func BuildTableInfo( func precheckBuildHiddenColumnInfo( indexPartSpecifications []*ast.IndexPartSpecification, - indexName pmodel.CIStr, + indexName ast.CIStr, ) error { for i, idxPart := range indexPartSpecifications { if idxPart.Expr == nil { @@ -1436,7 +1435,7 @@ func precheckBuildHiddenColumnInfo( return nil } -func buildHiddenColumnInfoWithCheck(ctx *metabuild.Context, indexPartSpecifications []*ast.IndexPartSpecification, indexName pmodel.CIStr, tblInfo *model.TableInfo, existCols []*table.Column) ([]*model.ColumnInfo, error) { +func buildHiddenColumnInfoWithCheck(ctx *metabuild.Context, indexPartSpecifications []*ast.IndexPartSpecification, indexName ast.CIStr, tblInfo *model.TableInfo, existCols []*table.Column) ([]*model.ColumnInfo, error) { if err := precheckBuildHiddenColumnInfo(indexPartSpecifications, indexName); err != nil { return nil, err } @@ -1444,13 +1443,13 @@ func buildHiddenColumnInfoWithCheck(ctx *metabuild.Context, indexPartSpecificati } // BuildHiddenColumnInfo builds hidden column info. -func BuildHiddenColumnInfo(ctx *metabuild.Context, indexPartSpecifications []*ast.IndexPartSpecification, indexName pmodel.CIStr, tblInfo *model.TableInfo, existCols []*table.Column) ([]*model.ColumnInfo, error) { +func BuildHiddenColumnInfo(ctx *metabuild.Context, indexPartSpecifications []*ast.IndexPartSpecification, indexName ast.CIStr, tblInfo *model.TableInfo, existCols []*table.Column) ([]*model.ColumnInfo, error) { hiddenCols := make([]*model.ColumnInfo, 0, len(indexPartSpecifications)) for i, idxPart := range indexPartSpecifications { if idxPart.Expr == nil { continue } - idxPart.Column = &ast.ColumnName{Name: pmodel.NewCIStr(fmt.Sprintf("%s_%s_%d", expressionIndexPrefix, indexName, i))} + idxPart.Column = &ast.ColumnName{Name: ast.NewCIStr(fmt.Sprintf("%s_%s_%d", expressionIndexPrefix, indexName, i))} // Check whether the hidden columns have existed. col := table.FindCol(existCols, idxPart.Column.Name.L) if col != nil { @@ -1580,7 +1579,7 @@ 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 { - if opt == nil || opt.PrimaryKeyTp == pmodel.PrimaryKeyTypeDefault { + if opt == nil || opt.PrimaryKeyTp == ast.PrimaryKeyTypeDefault { switch mode { case variable.ClusteredIndexDefModeOn: return true @@ -1590,7 +1589,7 @@ func ShouldBuildClusteredIndex(mode variable.ClusteredIndexDefMode, opt *ast.Ind return false } } - return opt.PrimaryKeyTp == pmodel.PrimaryKeyTypeClustered + return opt.PrimaryKeyTp == ast.PrimaryKeyTypeClustered } // BuildViewInfo builds a ViewInfo structure from an ast.CreateViewStmt. diff --git a/pkg/ddl/db_integration_test.go b/pkg/ddl/db_integration_test.go index 9686c2fd71f74..89a367aac28eb 100644 --- a/pkg/ddl/db_integration_test.go +++ b/pkg/ddl/db_integration_test.go @@ -36,9 +36,9 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta" "github.com/pingcap/tidb/pkg/meta/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/auth" "github.com/pingcap/tidb/pkg/parser/charset" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/session" @@ -515,7 +515,7 @@ func TestChangingTableCharset(t *testing.T) { ddlChecker.Disable() // Mock table info with charset is "". Old TiDB maybe create table with charset is "". - db, ok := dom.InfoSchema().SchemaByName(pmodel.NewCIStr("test")) + db, ok := dom.InfoSchema().SchemaByName(ast.NewCIStr("test")) require.True(t, ok) tbl := external.GetTableByName(t, tk, "test", "t") tblInfo := tbl.Meta().Clone() @@ -756,7 +756,7 @@ func TestCaseInsensitiveCharsetAndCollate(t *testing.T) { tk.MustExec("create table t5(a varchar(20)) ENGINE=InnoDB DEFAULT CHARSET=UTF8MB4 COLLATE=UTF8MB4_GENERAL_CI;") tk.MustExec("insert into t5 values ('特克斯和凯科斯群岛')") - db, ok := dom.InfoSchema().SchemaByName(pmodel.NewCIStr("test_charset_collate")) + db, ok := dom.InfoSchema().SchemaByName(ast.NewCIStr("test_charset_collate")) require.True(t, ok) tbl := external.GetTableByName(t, tk, "test_charset_collate", "t5") tblInfo := tbl.Meta().Clone() @@ -806,7 +806,7 @@ func TestZeroFillCreateTable(t *testing.T) { tk.MustExec("drop table if exists abc;") tk.MustExec("create table abc(y year, z tinyint(10) zerofill, primary key(y));") is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("abc")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("abc")) require.NoError(t, err) var yearCol, zCol *model.ColumnInfo for _, col := range tbl.Meta().Columns { @@ -1022,7 +1022,7 @@ func TestResolveCharset(t *testing.T) { tk.MustExec(`CREATE TABLE resolve_charset (a varchar(255) DEFAULT NULL) DEFAULT CHARSET=latin1`) ctx := tk.Session() is := domain.GetDomain(ctx).InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("resolve_charset")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("resolve_charset")) require.NoError(t, err) require.Equal(t, "latin1", tbl.Cols()[0].GetCharset()) tk.MustExec("INSERT INTO resolve_charset VALUES('鰈')") @@ -1032,14 +1032,14 @@ func TestResolveCharset(t *testing.T) { tk.MustExec(`CREATE TABLE resolve_charset (a varchar(255) DEFAULT NULL) DEFAULT CHARSET=latin1`) is = domain.GetDomain(ctx).InfoSchema() - tbl, err = is.TableByName(context.Background(), pmodel.NewCIStr("resolve_charset"), pmodel.NewCIStr("resolve_charset")) + tbl, err = is.TableByName(context.Background(), ast.NewCIStr("resolve_charset"), ast.NewCIStr("resolve_charset")) require.NoError(t, err) require.Equal(t, "latin1", tbl.Cols()[0].GetCharset()) require.Equal(t, "latin1", tbl.Meta().Charset) tk.MustExec(`CREATE TABLE resolve_charset1 (a varchar(255) DEFAULT NULL)`) is = domain.GetDomain(ctx).InfoSchema() - tbl, err = is.TableByName(context.Background(), pmodel.NewCIStr("resolve_charset"), pmodel.NewCIStr("resolve_charset1")) + tbl, err = is.TableByName(context.Background(), ast.NewCIStr("resolve_charset"), ast.NewCIStr("resolve_charset1")) require.NoError(t, err) require.Equal(t, "binary", tbl.Cols()[0].GetCharset()) require.Equal(t, "binary", tbl.Meta().Charset) @@ -1128,7 +1128,7 @@ func TestAlterColumn(t *testing.T) { tk.MustQuery("select a from test_alter_column").Check(testkit.Rows("111")) ctx := tk.Session() is := domain.GetDomain(ctx).InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("test_alter_column")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("test_alter_column")) require.NoError(t, err) tblInfo := tbl.Meta() colA := tblInfo.Columns[0] @@ -1138,7 +1138,7 @@ func TestAlterColumn(t *testing.T) { tk.MustExec("insert into test_alter_column set b = 'b', c = 'bb'") tk.MustQuery("select a from test_alter_column").Check(testkit.Rows("111", "222")) is = domain.GetDomain(ctx).InfoSchema() - tbl, err = is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("test_alter_column")) + tbl, err = is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("test_alter_column")) require.NoError(t, err) tblInfo = tbl.Meta() colA = tblInfo.Columns[0] @@ -1148,7 +1148,7 @@ func TestAlterColumn(t *testing.T) { tk.MustExec("insert into test_alter_column set c = 'cc'") tk.MustQuery("select b from test_alter_column").Check(testkit.Rows("a", "b", "")) is = domain.GetDomain(ctx).InfoSchema() - tbl, err = is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("test_alter_column")) + tbl, err = is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("test_alter_column")) require.NoError(t, err) tblInfo = tbl.Meta() colC := tblInfo.Columns[2] @@ -1158,7 +1158,7 @@ func TestAlterColumn(t *testing.T) { tk.MustExec("insert into test_alter_column set a = 123") tk.MustQuery("select c from test_alter_column").Check(testkit.Rows("aa", "bb", "cc", "xx")) is = domain.GetDomain(ctx).InfoSchema() - tbl, err = is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("test_alter_column")) + tbl, err = is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("test_alter_column")) require.NoError(t, err) tblInfo = tbl.Meta() colC = tblInfo.Columns[2] @@ -1428,7 +1428,7 @@ func TestTreatOldVersionUTF8AsUTF8MB4(t *testing.T) { ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin")) // Mock old version table info with column charset is utf8. - db, ok := domain.GetDomain(tk.Session()).InfoSchema().SchemaByName(pmodel.NewCIStr("test")) + db, ok := domain.GetDomain(tk.Session()).InfoSchema().SchemaByName(ast.NewCIStr("test")) tbl := external.GetTableByName(t, tk, "test", "t") tblInfo := tbl.Meta().Clone() tblInfo.Version = model.TableInfoVersion0 @@ -1717,7 +1717,7 @@ func TestChangingDBCharset(t *testing.T) { // Make sure the table schema is the new schema. err := dom.Reload() require.NoError(t, err) - dbInfo, ok := dom.InfoSchema().SchemaByName(pmodel.NewCIStr(dbName)) + dbInfo, ok := dom.InfoSchema().SchemaByName(ast.NewCIStr(dbName)) require.Equal(t, true, ok) require.Equal(t, chs, dbInfo.Charset) require.Equal(t, coll, dbInfo.Collate) @@ -1900,7 +1900,7 @@ func TestAddExpressionIndex(t *testing.T) { tk.MustExec("alter table t add index idx((a+b));") tk.MustQuery("SELECT * FROM INFORMATION_SCHEMA.KEY_COLUMN_USAGE WHERE table_name = 't'").Check(testkit.Rows()) - tblInfo, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tblInfo, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) columns := tblInfo.Meta().Columns require.Equal(t, 3, len(columns)) @@ -1909,7 +1909,7 @@ func TestAddExpressionIndex(t *testing.T) { tk.MustQuery("select * from t;").Check(testkit.Rows("1 2.1")) tk.MustExec("alter table t add index idx_multi((a+b),(a+1), b);") - tblInfo, err = dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tblInfo, err = dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) columns = tblInfo.Meta().Columns require.Equal(t, 5, len(columns)) @@ -1919,7 +1919,7 @@ func TestAddExpressionIndex(t *testing.T) { tk.MustQuery("select * from t;").Check(testkit.Rows("1 2.1")) tk.MustExec("alter table t drop index idx;") - tblInfo, err = dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tblInfo, err = dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) columns = tblInfo.Meta().Columns require.Equal(t, 4, len(columns)) @@ -1927,7 +1927,7 @@ func TestAddExpressionIndex(t *testing.T) { tk.MustQuery("select * from t;").Check(testkit.Rows("1 2.1")) tk.MustExec("alter table t drop index idx_multi;") - tblInfo, err = dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tblInfo, err = dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) columns = tblInfo.Meta().Columns require.Equal(t, 2, len(columns)) @@ -2746,7 +2746,7 @@ func TestDropTemporaryTable(t *testing.T) { sessionVars := tk.Session().GetSessionVars() sessVarsTempTable := sessionVars.LocalTemporaryTables localTemporaryTable := sessVarsTempTable.(*infoschema.SessionTables) - tbl, exist := localTemporaryTable.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("a_local_temp_table_7")) + tbl, exist := localTemporaryTable.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("a_local_temp_table_7")) require.True(t, exist) tblInfo := tbl.Meta() tablePrefix := tablecodec.EncodeTablePrefix(tblInfo.ID) @@ -2848,7 +2848,7 @@ func TestTruncateLocalTemporaryTable(t *testing.T) { // truncate temporary table will clear session data localTemporaryTables := tk.Session().GetSessionVars().LocalTemporaryTables.(*infoschema.SessionTables) - tb1, exist := localTemporaryTables.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t1")) + tb1, exist := localTemporaryTables.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t1")) tbl1Info := tb1.Meta() tablePrefix := tablecodec.EncodeTablePrefix(tbl1Info.ID) endTablePrefix := tablecodec.EncodeTablePrefix(tbl1Info.ID + 1) @@ -3046,7 +3046,7 @@ func TestIssue52680(t *testing.T) { tk.MustQuery("select * from issue52680").Check(testkit.Rows("1", "2")) is := dom.InfoSchema() - ti, err := is.TableInfoByName(pmodel.NewCIStr("test"), pmodel.NewCIStr("issue52680")) + ti, err := is.TableInfoByName(ast.NewCIStr("test"), ast.NewCIStr("issue52680")) require.NoError(t, err) ddlutil.EmulatorGCDisable() @@ -3089,7 +3089,7 @@ func TestIssue52680(t *testing.T) { )) is = dom.InfoSchema() - ti1, err := is.TableInfoByName(pmodel.NewCIStr("test"), pmodel.NewCIStr("issue52680")) + ti1, err := is.TableInfoByName(ast.NewCIStr("test"), ast.NewCIStr("issue52680")) require.NoError(t, err) require.Equal(t, ti1.ID, ti.ID) diff --git a/pkg/ddl/db_rename_test.go b/pkg/ddl/db_rename_test.go index 842bbe35cdc9b..350c2c98d4b7c 100644 --- a/pkg/ddl/db_rename_test.go +++ b/pkg/ddl/db_rename_test.go @@ -24,7 +24,7 @@ import ( "github.com/pingcap/tidb/pkg/ddl/logutil" "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/errno" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/store/mockstore" "github.com/pingcap/tidb/pkg/testkit" "github.com/stretchr/testify/require" @@ -84,7 +84,7 @@ func renameTableTest(t *testing.T, sql string, isAlterTable bool) { tk.MustExec("insert t values (1, 1), (2, 2)") ctx := tk.Session() is := domain.GetDomain(ctx).InfoSchema() - oldTblInfo, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + oldTblInfo, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) oldTblID := oldTblInfo.Meta().ID oldDBID := oldTblInfo.Meta().DBID @@ -94,7 +94,7 @@ func renameTableTest(t *testing.T, sql string, isAlterTable bool) { tk.MustExec("use test1") tk.MustExec(fmt.Sprintf(sql, "test.t", "test1.t1")) is = domain.GetDomain(ctx).InfoSchema() - newTblInfo, err := is.TableByName(context.Background(), model.NewCIStr("test1"), model.NewCIStr("t1")) + newTblInfo, err := is.TableByName(context.Background(), ast.NewCIStr("test1"), ast.NewCIStr("t1")) require.NoError(t, err) require.Equal(t, oldTblID, newTblInfo.Meta().ID) require.NotEqual(t, newTblInfo.Meta().DBID, oldDBID) @@ -111,12 +111,12 @@ func renameTableTest(t *testing.T, sql string, isAlterTable bool) { tk.MustExec("use test1") tk.MustExec(fmt.Sprintf(sql, "t1", "t2")) is = domain.GetDomain(ctx).InfoSchema() - newTblInfo, err = is.TableByName(context.Background(), model.NewCIStr("test1"), model.NewCIStr("t2")) + newTblInfo, err = is.TableByName(context.Background(), ast.NewCIStr("test1"), ast.NewCIStr("t2")) require.NoError(t, err) require.Equal(t, oldTblID, newTblInfo.Meta().ID) require.Equal(t, oldDBID, newTblInfo.Meta().DBID) tk.MustQuery("select * from t2").Check(testkit.Rows("1 1", "2 2")) - isExist := is.TableExists(model.NewCIStr("test1"), model.NewCIStr("t1")) + isExist := is.TableExists(ast.NewCIStr("test1"), ast.NewCIStr("t1")) require.False(t, isExist) tk.MustQuery("show tables").Check(testkit.Rows("t2")) @@ -189,46 +189,46 @@ func TestRenameMultiTables(t *testing.T) { tk.MustExec("insert t2 values (1, 1), (2, 2)") ctx := tk.Session() is := domain.GetDomain(ctx).InfoSchema() - oldTblInfo1, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t1")) + oldTblInfo1, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t1")) require.NoError(t, err) oldTblID1 := oldTblInfo1.Meta().ID - oldTblInfo2, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t2")) + oldTblInfo2, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t2")) require.NoError(t, err) oldTblID2 := oldTblInfo2.Meta().ID tk.MustExec("create database test1") tk.MustExec("use test1") tk.MustExec("rename table test.t1 to test1.t1, test.t2 to test1.t2") is = domain.GetDomain(ctx).InfoSchema() - newTblInfo1, err := is.TableByName(context.Background(), model.NewCIStr("test1"), model.NewCIStr("t1")) + newTblInfo1, err := is.TableByName(context.Background(), ast.NewCIStr("test1"), ast.NewCIStr("t1")) require.NoError(t, err) require.Equal(t, oldTblID1, newTblInfo1.Meta().ID) - newTblInfo2, err := is.TableByName(context.Background(), model.NewCIStr("test1"), model.NewCIStr("t2")) + newTblInfo2, err := is.TableByName(context.Background(), ast.NewCIStr("test1"), ast.NewCIStr("t2")) require.NoError(t, err) require.Equal(t, oldTblID2, newTblInfo2.Meta().ID) tk.MustQuery("select * from t1").Check(testkit.Rows("1 1", "2 2")) tk.MustQuery("select * from t2").Check(testkit.Rows("1 1", "2 2")) // Make sure t1,t2 doesn't exist. - isExist := is.TableExists(model.NewCIStr("test"), model.NewCIStr("t1")) + isExist := is.TableExists(ast.NewCIStr("test"), ast.NewCIStr("t1")) require.False(t, isExist) - isExist = is.TableExists(model.NewCIStr("test"), model.NewCIStr("t2")) + isExist = is.TableExists(ast.NewCIStr("test"), ast.NewCIStr("t2")) require.False(t, isExist) // for the same database tk.MustExec("use test1") tk.MustExec("rename table test1.t1 to test1.t3, test1.t2 to test1.t4") is = domain.GetDomain(ctx).InfoSchema() - newTblInfo1, err = is.TableByName(context.Background(), model.NewCIStr("test1"), model.NewCIStr("t3")) + newTblInfo1, err = is.TableByName(context.Background(), ast.NewCIStr("test1"), ast.NewCIStr("t3")) require.NoError(t, err) require.Equal(t, oldTblID1, newTblInfo1.Meta().ID) - newTblInfo2, err = is.TableByName(context.Background(), model.NewCIStr("test1"), model.NewCIStr("t4")) + newTblInfo2, err = is.TableByName(context.Background(), ast.NewCIStr("test1"), ast.NewCIStr("t4")) require.NoError(t, err) require.Equal(t, oldTblID2, newTblInfo2.Meta().ID) tk.MustQuery("select * from t3").Check(testkit.Rows("1 1", "2 2")) - isExist = is.TableExists(model.NewCIStr("test1"), model.NewCIStr("t1")) + isExist = is.TableExists(ast.NewCIStr("test1"), ast.NewCIStr("t1")) require.False(t, isExist) tk.MustQuery("select * from t4").Check(testkit.Rows("1 1", "2 2")) - isExist = is.TableExists(model.NewCIStr("test1"), model.NewCIStr("t2")) + isExist = is.TableExists(ast.NewCIStr("test1"), ast.NewCIStr("t2")) require.False(t, isExist) tk.MustQuery("show tables").Check(testkit.Rows("t3", "t4")) @@ -236,18 +236,18 @@ func TestRenameMultiTables(t *testing.T) { tk.MustExec("create table t5 (c1 int, c2 int)") tk.MustExec("insert t5 values (1, 1), (2, 2)") is = domain.GetDomain(ctx).InfoSchema() - oldTblInfo3, err := is.TableByName(context.Background(), model.NewCIStr("test1"), model.NewCIStr("t5")) + oldTblInfo3, err := is.TableByName(context.Background(), ast.NewCIStr("test1"), ast.NewCIStr("t5")) require.NoError(t, err) oldTblID3 := oldTblInfo3.Meta().ID tk.MustExec("rename table test1.t3 to test1.t1, test1.t4 to test1.t2, test1.t5 to test1.t3") is = domain.GetDomain(ctx).InfoSchema() - newTblInfo1, err = is.TableByName(context.Background(), model.NewCIStr("test1"), model.NewCIStr("t1")) + newTblInfo1, err = is.TableByName(context.Background(), ast.NewCIStr("test1"), ast.NewCIStr("t1")) require.NoError(t, err) require.Equal(t, oldTblID1, newTblInfo1.Meta().ID) - newTblInfo2, err = is.TableByName(context.Background(), model.NewCIStr("test1"), model.NewCIStr("t2")) + newTblInfo2, err = is.TableByName(context.Background(), ast.NewCIStr("test1"), ast.NewCIStr("t2")) require.NoError(t, err) require.Equal(t, oldTblID2, newTblInfo2.Meta().ID) - newTblInfo3, err := is.TableByName(context.Background(), model.NewCIStr("test1"), model.NewCIStr("t3")) + newTblInfo3, err := is.TableByName(context.Background(), ast.NewCIStr("test1"), ast.NewCIStr("t3")) require.NoError(t, err) require.Equal(t, oldTblID3, newTblInfo3.Meta().ID) tk.MustQuery("show tables").Check(testkit.Rows("t1", "t2", "t3")) @@ -256,13 +256,13 @@ func TestRenameMultiTables(t *testing.T) { tk.MustExec("use test") tk.MustExec("rename table test1.t1 to test.t2, test1.t2 to test.t3, test1.t3 to test.t4") is = domain.GetDomain(ctx).InfoSchema() - newTblInfo1, err = is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t2")) + newTblInfo1, err = is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t2")) require.NoError(t, err) require.Equal(t, oldTblID1, newTblInfo1.Meta().ID) - newTblInfo2, err = is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t3")) + newTblInfo2, err = is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t3")) require.NoError(t, err) require.Equal(t, oldTblID2, newTblInfo2.Meta().ID) - newTblInfo3, err = is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t4")) + newTblInfo3, err = is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t4")) require.NoError(t, err) require.Equal(t, oldTblID3, newTblInfo3.Meta().ID) tk.MustQuery("show tables").Check(testkit.Rows("t2", "t3", "t4")) @@ -313,7 +313,7 @@ func TestRenameConcurrentAutoID(t *testing.T) { ctx := tk1.Session() is := domain.GetDomain(ctx).InfoSchema() - tblInfo, err := is.TableByName(context.Background(), model.NewCIStr("test1"), model.NewCIStr("t1")) + tblInfo, err := is.TableByName(context.Background(), ast.NewCIStr("test1"), ast.NewCIStr("t1")) require.NoError(t, err) require.Equal(t, int64(0), tblInfo.Meta().AutoIDSchemaID) origAllocs := tblInfo.Allocators(nil) @@ -374,7 +374,7 @@ func TestRenameConcurrentAutoID(t *testing.T) { // and if you rename multiple time (so it does not lose the autoID). ctx = tk1.Session() is = domain.GetDomain(ctx).InfoSchema() - tblInfo, err = is.TableByName(context.Background(), model.NewCIStr("test2"), model.NewCIStr("t2")) + tblInfo, err = is.TableByName(context.Background(), ast.NewCIStr("test2"), ast.NewCIStr("t2")) require.NoError(t, err) originalSchemaID := tblInfo.Meta().AutoIDSchemaID require.NotEqual(t, int64(0), originalSchemaID) @@ -404,7 +404,7 @@ func TestRenameConcurrentAutoID(t *testing.T) { ctx = tk1.Session() is = domain.GetDomain(ctx).InfoSchema() - tblInfo, err = is.TableByName(context.Background(), model.NewCIStr("test2"), model.NewCIStr("t1")) + tblInfo, err = is.TableByName(context.Background(), ast.NewCIStr("test2"), ast.NewCIStr("t1")) require.NoError(t, err) require.Equal(t, originalSchemaID, tblInfo.Meta().AutoIDSchemaID) origAllocs = tblInfo.Allocators(nil) @@ -444,7 +444,7 @@ func TestRenameConcurrentAutoID(t *testing.T) { ctx = tk1.Session() is = domain.GetDomain(ctx).InfoSchema() - tblInfo, err = is.TableByName(context.Background(), model.NewCIStr("test1"), model.NewCIStr("t1")) + tblInfo, err = is.TableByName(context.Background(), ast.NewCIStr("test1"), ast.NewCIStr("t1")) require.NoError(t, err) // Should be cleared when moved back to the original SchemaID require.Equal(t, int64(0), tblInfo.Meta().AutoIDSchemaID) @@ -461,7 +461,7 @@ func TestRenameConcurrentAutoID(t *testing.T) { ctx = tk1.Session() is = domain.GetDomain(ctx).InfoSchema() - tblInfo, err = is.TableByName(context.Background(), model.NewCIStr("test1"), model.NewCIStr("t1")) + tblInfo, err = is.TableByName(context.Background(), ast.NewCIStr("test1"), ast.NewCIStr("t1")) require.NoError(t, err) require.NotEqual(t, int64(0), tblInfo.Meta().AutoIDSchemaID) origAllocs = tblInfo.Allocators(nil) diff --git a/pkg/ddl/db_table_test.go b/pkg/ddl/db_table_test.go index 8363a454c8f93..164cf6852a59f 100644 --- a/pkg/ddl/db_table_test.go +++ b/pkg/ddl/db_table_test.go @@ -33,8 +33,8 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta" "github.com/pingcap/tidb/pkg/meta/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/auth" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/store/mockstore" @@ -315,10 +315,10 @@ func TestCreateTableWithInfo(t *testing.T) { require.NotNil(t, d) info := []*model.TableInfo{{ ID: 42042, // Note, we must ensure the table ID is globally unique! - Name: pmodel.NewCIStr("t"), + Name: ast.NewCIStr("t"), }} - require.NoError(t, d.BatchCreateTableWithInfo(tk.Session(), pmodel.NewCIStr("test"), info, ddl.WithOnExist(ddl.OnExistError), ddl.WithIDAllocated(true))) + require.NoError(t, d.BatchCreateTableWithInfo(tk.Session(), ast.NewCIStr("test"), info, ddl.WithOnExist(ddl.OnExistError), ddl.WithIDAllocated(true))) tk.MustQuery("select tidb_table_id from information_schema.tables where table_name = 't'").Check(testkit.Rows("42042")) ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnOthers) @@ -333,10 +333,10 @@ func TestCreateTableWithInfo(t *testing.T) { require.NoError(t, err) info = []*model.TableInfo{{ ID: 42, - Name: pmodel.NewCIStr("tt"), + Name: ast.NewCIStr("tt"), }} tk.Session().SetValue(sessionctx.QueryString, "skip") - require.NoError(t, d.BatchCreateTableWithInfo(tk.Session(), pmodel.NewCIStr("test"), info, ddl.WithOnExist(ddl.OnExistError))) + require.NoError(t, d.BatchCreateTableWithInfo(tk.Session(), ast.NewCIStr("test"), info, ddl.WithOnExist(ddl.OnExistError))) idGen, ok := tk.MustQuery("select tidb_table_id from information_schema.tables where table_name = 'tt'").Rows()[0][0].(string) require.True(t, ok) idGenNum, err := strconv.ParseInt(idGen, 10, 64) @@ -355,18 +355,18 @@ func TestBatchCreateTable(t *testing.T) { d := dom.DDLExecutor() infos := []*model.TableInfo{} infos = append(infos, &model.TableInfo{ - Name: pmodel.NewCIStr("tables_1"), + Name: ast.NewCIStr("tables_1"), }) infos = append(infos, &model.TableInfo{ - Name: pmodel.NewCIStr("tables_2"), + Name: ast.NewCIStr("tables_2"), }) infos = append(infos, &model.TableInfo{ - Name: pmodel.NewCIStr("tables_3"), + Name: ast.NewCIStr("tables_3"), }) // correct name tk.Session().SetValue(sessionctx.QueryString, "skip") - err := d.BatchCreateTableWithInfo(tk.Session(), pmodel.NewCIStr("test"), infos, ddl.WithOnExist(ddl.OnExistError)) + err := d.BatchCreateTableWithInfo(tk.Session(), ast.NewCIStr("test"), infos, ddl.WithOnExist(ddl.OnExistError)) require.NoError(t, err) tk.MustQuery("show tables like '%tables_%'").Check(testkit.Rows("tables_1", "tables_2", "tables_3")) @@ -379,23 +379,23 @@ func TestBatchCreateTable(t *testing.T) { // c.Assert(job[6], Matches, "[^,]+,[^,]+,[^,]+") // duplicated name - infos[1].Name = pmodel.NewCIStr("tables_1") + infos[1].Name = ast.NewCIStr("tables_1") tk.Session().SetValue(sessionctx.QueryString, "skip") - err = d.BatchCreateTableWithInfo(tk.Session(), pmodel.NewCIStr("test"), infos, ddl.WithOnExist(ddl.OnExistError)) + err = d.BatchCreateTableWithInfo(tk.Session(), ast.NewCIStr("test"), infos, ddl.WithOnExist(ddl.OnExistError)) require.True(t, terror.ErrorEqual(err, infoschema.ErrTableExists)) newinfo := &model.TableInfo{ - Name: pmodel.NewCIStr("tables_4"), + Name: ast.NewCIStr("tables_4"), } { colNum := 2 cols := make([]*model.ColumnInfo, colNum) - viewCols := make([]pmodel.CIStr, colNum) + viewCols := make([]ast.CIStr, colNum) var stmtBuffer bytes.Buffer stmtBuffer.WriteString("SELECT ") for i := range cols { col := &model.ColumnInfo{ - Name: pmodel.NewCIStr(fmt.Sprintf("c%d", i+1)), + Name: ast.NewCIStr(fmt.Sprintf("c%d", i+1)), Offset: i, State: model.StatePublic, } @@ -405,12 +405,12 @@ func TestBatchCreateTable(t *testing.T) { } stmtBuffer.WriteString("1 FROM t") newinfo.Columns = cols - newinfo.View = &model.ViewInfo{Cols: viewCols, Security: pmodel.SecurityDefiner, Algorithm: pmodel.AlgorithmMerge, SelectStmt: stmtBuffer.String(), CheckOption: pmodel.CheckOptionCascaded, Definer: &auth.UserIdentity{CurrentUser: true}} + newinfo.View = &model.ViewInfo{Cols: viewCols, Security: ast.SecurityDefiner, Algorithm: ast.AlgorithmMerge, SelectStmt: stmtBuffer.String(), CheckOption: ast.CheckOptionCascaded, Definer: &auth.UserIdentity{CurrentUser: true}} } tk.Session().SetValue(sessionctx.QueryString, "skip") tk.Session().SetValue(sessionctx.QueryString, "skip") - err = d.BatchCreateTableWithInfo(tk.Session(), pmodel.NewCIStr("test"), []*model.TableInfo{newinfo}, ddl.WithOnExist(ddl.OnExistError)) + err = d.BatchCreateTableWithInfo(tk.Session(), ast.NewCIStr("test"), []*model.TableInfo{newinfo}, ddl.WithOnExist(ddl.OnExistError)) require.NoError(t, err) } @@ -427,12 +427,12 @@ func TestTableLock(t *testing.T) { tk.MustExec("lock tables t1 write") tk.MustExec("insert into t1 values(NULL)") tk.MustExec("unlock tables") - checkTableLock(t, tk, "test", "t1", pmodel.TableLockNone) + checkTableLock(t, tk, "test", "t1", ast.TableLockNone) tk.MustExec("lock tables t1 write") tk.MustExec("insert into t1 values(NULL)") tk.MustExec("unlock tables") - checkTableLock(t, tk, "test", "t1", pmodel.TableLockNone) + checkTableLock(t, tk, "test", "t1", ast.TableLockNone) tk.MustExec("drop table if exists t1") @@ -474,12 +474,12 @@ func TestTableLocksLostCommit(t *testing.T) { tk.MustExec("unlock tables") } -func checkTableLock(t *testing.T, tk *testkit.TestKit, dbName, tableName string, lockTp pmodel.TableLockType) { +func checkTableLock(t *testing.T, tk *testkit.TestKit, dbName, tableName string, lockTp ast.TableLockType) { tb := external.GetTableByName(t, tk, dbName, tableName) dom := domain.GetDomain(tk.Session()) err := dom.Reload() require.NoError(t, err) - if lockTp != pmodel.TableLockNone { + if lockTp != ast.TableLockNone { require.NotNil(t, tb.Meta().Lock) require.Equal(t, lockTp, tb.Meta().Lock.Tp) require.Equal(t, model.TableLockStatePublic, tb.Meta().Lock.State) @@ -555,31 +555,31 @@ func TestLockTables(t *testing.T) { // Test lock 1 table. tk.MustExec("lock tables t1 write") - checkTableLock(t, tk, "test", "t1", pmodel.TableLockWrite) + checkTableLock(t, tk, "test", "t1", ast.TableLockWrite) // still locked after truncate. tk.MustExec("truncate table t1") - checkTableLock(t, tk, "test", "t1", pmodel.TableLockWrite) + checkTableLock(t, tk, "test", "t1", ast.TableLockWrite) // should unlock the new table id. tk.MustExec("unlock tables") - checkTableLock(t, tk, "test", "t1", pmodel.TableLockNone) + checkTableLock(t, tk, "test", "t1", ast.TableLockNone) tk.MustExec("lock tables t1 read") - checkTableLock(t, tk, "test", "t1", pmodel.TableLockRead) + checkTableLock(t, tk, "test", "t1", ast.TableLockRead) tk.MustExec("lock tables t1 write") - checkTableLock(t, tk, "test", "t1", pmodel.TableLockWrite) + checkTableLock(t, tk, "test", "t1", ast.TableLockWrite) // Test lock multi tables. tk.MustExec("lock tables t1 write, t2 read") - checkTableLock(t, tk, "test", "t1", pmodel.TableLockWrite) - checkTableLock(t, tk, "test", "t2", pmodel.TableLockRead) + checkTableLock(t, tk, "test", "t1", ast.TableLockWrite) + checkTableLock(t, tk, "test", "t2", ast.TableLockRead) tk.MustExec("lock tables t1 read, t2 write") - checkTableLock(t, tk, "test", "t1", pmodel.TableLockRead) - checkTableLock(t, tk, "test", "t2", pmodel.TableLockWrite) + checkTableLock(t, tk, "test", "t1", ast.TableLockRead) + checkTableLock(t, tk, "test", "t2", ast.TableLockWrite) tk.MustExec("lock tables t2 write") - checkTableLock(t, tk, "test", "t2", pmodel.TableLockWrite) - checkTableLock(t, tk, "test", "t1", pmodel.TableLockNone) + checkTableLock(t, tk, "test", "t2", ast.TableLockWrite) + checkTableLock(t, tk, "test", "t1", ast.TableLockNone) tk.MustExec("lock tables t1 write") - checkTableLock(t, tk, "test", "t1", pmodel.TableLockWrite) - checkTableLock(t, tk, "test", "t2", pmodel.TableLockNone) + checkTableLock(t, tk, "test", "t1", ast.TableLockWrite) + checkTableLock(t, tk, "test", "t2", ast.TableLockNone) tk2 := testkit.NewTestKit(t, store) tk2.MustExec("use test") @@ -697,15 +697,15 @@ func TestLockTables(t *testing.T) { tk.MustExec("lock table t1 write, t2 write") tk2.MustGetDBError("lock tables t1 write, t2 read", infoschema.ErrTableLocked) tk2.MustExec("admin cleanup table lock t1,t2") - checkTableLock(t, tk, "test", "t1", pmodel.TableLockNone) - checkTableLock(t, tk, "test", "t2", pmodel.TableLockNone) + checkTableLock(t, tk, "test", "t1", ast.TableLockNone) + checkTableLock(t, tk, "test", "t2", ast.TableLockNone) // cleanup unlocked table. tk2.MustExec("admin cleanup table lock t1,t2") - checkTableLock(t, tk, "test", "t1", pmodel.TableLockNone) - checkTableLock(t, tk, "test", "t2", pmodel.TableLockNone) + checkTableLock(t, tk, "test", "t1", ast.TableLockNone) + checkTableLock(t, tk, "test", "t2", ast.TableLockNone) tk2.MustExec("lock tables t1 write, t2 read") - checkTableLock(t, tk2, "test", "t1", pmodel.TableLockWrite) - checkTableLock(t, tk2, "test", "t2", pmodel.TableLockRead) + checkTableLock(t, tk2, "test", "t1", ast.TableLockWrite) + checkTableLock(t, tk2, "test", "t2", ast.TableLockRead) tk.MustExec("unlock tables") tk2.MustExec("unlock tables") @@ -723,7 +723,7 @@ func TestTablesLockDelayClean(t *testing.T) { tk.MustExec("create table t2 (a int)") tk.MustExec("lock tables t1 write") - checkTableLock(t, tk, "test", "t1", pmodel.TableLockWrite) + checkTableLock(t, tk, "test", "t1", ast.TableLockWrite) config.UpdateGlobal(func(conf *config.Config) { conf.DelayCleanTableLock = 100 }) @@ -734,10 +734,10 @@ func TestTablesLockDelayClean(t *testing.T) { tk.Session().Close() }) time.Sleep(50 * time.Millisecond) - checkTableLock(t, tk, "test", "t1", pmodel.TableLockWrite) + checkTableLock(t, tk, "test", "t1", ast.TableLockWrite) wg.Wait() require.True(t, time.Since(startTime).Seconds() > 0.1) - checkTableLock(t, tk, "test", "t1", pmodel.TableLockNone) + checkTableLock(t, tk, "test", "t1", ast.TableLockNone) config.UpdateGlobal(func(conf *config.Config) { conf.DelayCleanTableLock = 0 }) diff --git a/pkg/ddl/db_test.go b/pkg/ddl/db_test.go index 528f9067e0646..2327afd517d08 100644 --- a/pkg/ddl/db_test.go +++ b/pkg/ddl/db_test.go @@ -38,7 +38,6 @@ import ( "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/auth" "github.com/pingcap/tidb/pkg/parser/charset" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" parsertypes "github.com/pingcap/tidb/pkg/parser/types" @@ -323,7 +322,7 @@ func TestForbidCacheTableForSystemTable(t *testing.T) { for _, one := range sysTables { err := tk.ExecToErr(fmt.Sprintf("alter table `%s` cache", one)) if db == "MySQL" || db == "SYS" { - tbl, err1 := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr(db), pmodel.NewCIStr(one)) + tbl, err1 := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr(db), ast.NewCIStr(one)) require.NoError(t, err1) if tbl.Meta().View != nil { require.ErrorIs(t, err, dbterror.ErrWrongObject) @@ -649,10 +648,10 @@ func TestSnapshotVersion(t *testing.T) { require.Equal(t, is.SchemaMetaVersion(), currSnapIs.SchemaMetaVersion()) // for GetSnapshotMeta - dbInfo, ok := currSnapIs.SchemaByName(pmodel.NewCIStr("test2")) + dbInfo, ok := currSnapIs.SchemaByName(ast.NewCIStr("test2")) require.True(t, ok) - tbl, err := currSnapIs.TableByName(context.Background(), pmodel.NewCIStr("test2"), pmodel.NewCIStr("t")) + tbl, err := currSnapIs.TableByName(context.Background(), ast.NewCIStr("test2"), ast.NewCIStr("t")) require.NoError(t, err) m := dom.GetSnapshotMeta(snapTS) diff --git a/pkg/ddl/ddl_test.go b/pkg/ddl/ddl_test.go index 23e6fd2ecd36e..ed5d14d2f389a 100644 --- a/pkg/ddl/ddl_test.go +++ b/pkg/ddl/ddl_test.go @@ -31,7 +31,6 @@ import ( "github.com/pingcap/tidb/pkg/parser" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/charset" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/store/mockstore" @@ -168,7 +167,7 @@ func TestFieldCase(t *testing.T) { colObjects := make([]*model.ColumnInfo, len(fields)) for i, name := range fields { colObjects[i] = &model.ColumnInfo{ - Name: pmodel.NewCIStr(name), + Name: ast.NewCIStr(name), } } err := checkDuplicateColumn(colObjects) diff --git a/pkg/ddl/executor.go b/pkg/ddl/executor.go index daeb8ca24e69f..b35f577dbdba9 100644 --- a/pkg/ddl/executor.go +++ b/pkg/ddl/executor.go @@ -49,7 +49,6 @@ import ( "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/charset" "github.com/pingcap/tidb/pkg/parser/format" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" parser_types "github.com/pingcap/tidb/pkg/parser/types" @@ -146,14 +145,14 @@ type Executor interface { // in-place. If you want to keep using `info`, please call Clone() first. CreateTableWithInfo( ctx sessionctx.Context, - schema pmodel.CIStr, + schema ast.CIStr, info *model.TableInfo, involvingRef []model.InvolvingSchemaInfo, cs ...CreateTableOption) error // BatchCreateTableWithInfo is like CreateTableWithInfo, but can handle multiple tables. BatchCreateTableWithInfo(ctx sessionctx.Context, - schema pmodel.CIStr, + schema ast.CIStr, info []*model.TableInfo, cs ...CreateTableOption) error @@ -222,7 +221,7 @@ func (e *executor) CreateSchema(ctx sessionctx.Context, stmt *ast.CreateDatabase explicitCollation = true case ast.DatabaseOptionPlacementPolicy: placementPolicyRef = &model.PolicyRefInfo{ - Name: pmodel.NewCIStr(val.Value), + Name: ast.NewCIStr(val.Value), } } } @@ -714,7 +713,7 @@ func (e *executor) AlterSchema(sctx sessionctx.Context, stmt *ast.AlterDatabaseS toCollate = info.Name isAlterCharsetAndCollate = true case ast.DatabaseOptionPlacementPolicy: - placementPolicyRef = &model.PolicyRefInfo{Name: pmodel.NewCIStr(val.Value)} + placementPolicyRef = &model.PolicyRefInfo{Name: ast.NewCIStr(val.Value)} case ast.DatabaseSetTiFlashReplica: tiflashReplica = val.TiFlashReplica } @@ -829,35 +828,35 @@ func (e *executor) RecoverSchema(ctx sessionctx.Context, recoverSchemaInfo *mode return errors.Trace(err) } -func checkTooLongSchema(schema pmodel.CIStr) error { +func checkTooLongSchema(schema ast.CIStr) error { if utf8.RuneCountInString(schema.L) > mysql.MaxDatabaseNameLength { return dbterror.ErrTooLongIdent.GenWithStackByArgs(schema) } return nil } -func checkTooLongTable(table pmodel.CIStr) error { +func checkTooLongTable(table ast.CIStr) error { if utf8.RuneCountInString(table.L) > mysql.MaxTableNameLength { return dbterror.ErrTooLongIdent.GenWithStackByArgs(table) } return nil } -func checkTooLongIndex(index pmodel.CIStr) error { +func checkTooLongIndex(index ast.CIStr) error { if utf8.RuneCountInString(index.L) > mysql.MaxIndexIdentifierLen { return dbterror.ErrTooLongIdent.GenWithStackByArgs(index) } return nil } -func checkTooLongColumn(col pmodel.CIStr) error { +func checkTooLongColumn(col ast.CIStr) error { if utf8.RuneCountInString(col.L) > mysql.MaxColumnNameLength { return dbterror.ErrTooLongIdent.GenWithStackByArgs(col) } return nil } -func checkTooLongForeignKey(fk pmodel.CIStr) error { +func checkTooLongForeignKey(fk ast.CIStr) error { if utf8.RuneCountInString(fk.L) > mysql.MaxForeignKeyIdentifierLen { return dbterror.ErrTooLongIdent.GenWithStackByArgs(fk) } @@ -952,7 +951,7 @@ func checkInvisibleIndexOnPK(tblInfo *model.TableInfo) error { // checkGlobalIndex check if the index is allowed to have global index func checkGlobalIndex(ec errctx.Context, tblInfo *model.TableInfo, indexInfo *model.IndexInfo) error { pi := tblInfo.GetPartitionInfo() - isPartitioned := pi != nil && pi.Type != pmodel.PartitionTypeNone + isPartitioned := pi != nil && pi.Type != ast.PartitionTypeNone if indexInfo.Global { if !isPartitioned { // Makes no sense with LOCAL/GLOBAL index for non-partitioned tables, since we don't support @@ -1043,7 +1042,7 @@ func (e *executor) CreateTable(ctx sessionctx.Context, s *ast.CreateTableStmt) ( // WARNING: it may return a nil job, which means you don't need to submit any DDL job. func (e *executor) createTableWithInfoJob( ctx sessionctx.Context, - dbName pmodel.CIStr, + dbName ast.CIStr, tbInfo *model.TableInfo, involvingRef []model.InvolvingSchemaInfo, cfg CreateTableConfig, @@ -1188,7 +1187,7 @@ func (e *executor) createTableWithInfoPost( func (e *executor) CreateTableWithInfo( ctx sessionctx.Context, - dbName pmodel.CIStr, + dbName ast.CIStr, tbInfo *model.TableInfo, involvingRef []model.InvolvingSchemaInfo, cs ...CreateTableOption, @@ -1218,7 +1217,7 @@ func (e *executor) CreateTableWithInfo( } func (e *executor) BatchCreateTableWithInfo(ctx sessionctx.Context, - dbName pmodel.CIStr, + dbName ast.CIStr, infos []*model.TableInfo, cs ...CreateTableOption, ) error { @@ -1762,9 +1761,9 @@ func (e *executor) AlterTable(ctx context.Context, sctx sessionctx.Context, stmt case ast.AlterTableDropColumn: err = e.DropColumn(sctx, ident, spec) case ast.AlterTableDropIndex: - err = e.dropIndex(sctx, ident, pmodel.NewCIStr(spec.Name), spec.IfExists, false) + err = e.dropIndex(sctx, ident, ast.NewCIStr(spec.Name), spec.IfExists, false) case ast.AlterTableDropPrimaryKey: - err = e.dropIndex(sctx, ident, pmodel.NewCIStr(mysql.PrimaryKeyName), spec.IfExists, false) + err = e.dropIndex(sctx, ident, ast.NewCIStr(mysql.PrimaryKeyName), spec.IfExists, false) case ast.AlterTableRenameIndex: err = e.RenameIndex(sctx, ident, spec) case ast.AlterTableDropPartition, ast.AlterTableDropFirstPartition: @@ -1783,7 +1782,7 @@ func (e *executor) AlterTable(ctx context.Context, sctx sessionctx.Context, stmt TableLocks: []ast.TableLock{ { Table: tName, - Type: pmodel.TableLockReadOnly, + Type: ast.TableLockReadOnly, }, }, } @@ -1795,33 +1794,33 @@ func (e *executor) AlterTable(ctx context.Context, sctx sessionctx.Context, stmt constr := spec.Constraint switch spec.Constraint.Tp { case ast.ConstraintKey, ast.ConstraintIndex: - err = e.createIndex(sctx, ident, ast.IndexKeyTypeNone, pmodel.NewCIStr(constr.Name), + err = e.createIndex(sctx, ident, ast.IndexKeyTypeNone, ast.NewCIStr(constr.Name), spec.Constraint.Keys, constr.Option, constr.IfNotExists) case ast.ConstraintUniq, ast.ConstraintUniqIndex, ast.ConstraintUniqKey: - err = e.createIndex(sctx, ident, ast.IndexKeyTypeUnique, pmodel.NewCIStr(constr.Name), + err = e.createIndex(sctx, ident, ast.IndexKeyTypeUnique, ast.NewCIStr(constr.Name), spec.Constraint.Keys, constr.Option, false) // IfNotExists should be not applied case ast.ConstraintForeignKey: // NOTE: we do not handle `symbol` and `index_name` well in the parser and we do not check ForeignKey already exists, // so we just also ignore the `if not exists` check. - err = e.CreateForeignKey(sctx, ident, pmodel.NewCIStr(constr.Name), spec.Constraint.Keys, spec.Constraint.Refer) + err = e.CreateForeignKey(sctx, ident, ast.NewCIStr(constr.Name), spec.Constraint.Keys, spec.Constraint.Refer) case ast.ConstraintPrimaryKey: - err = e.CreatePrimaryKey(sctx, ident, pmodel.NewCIStr(constr.Name), spec.Constraint.Keys, constr.Option) + err = e.CreatePrimaryKey(sctx, ident, ast.NewCIStr(constr.Name), spec.Constraint.Keys, constr.Option) case ast.ConstraintFulltext: sctx.GetSessionVars().StmtCtx.AppendWarning(dbterror.ErrTableCantHandleFt) case ast.ConstraintCheck: if !variable.EnableCheckConstraint.Load() { sctx.GetSessionVars().StmtCtx.AppendWarning(errCheckConstraintIsOff) } else { - err = e.CreateCheckConstraint(sctx, ident, pmodel.NewCIStr(constr.Name), spec.Constraint) + err = e.CreateCheckConstraint(sctx, ident, ast.NewCIStr(constr.Name), spec.Constraint) } case ast.ConstraintVector: - err = e.createVectorIndex(sctx, ident, pmodel.NewCIStr(constr.Name), spec.Constraint.Keys, constr.Option, constr.IfNotExists) + err = e.createVectorIndex(sctx, ident, ast.NewCIStr(constr.Name), spec.Constraint.Keys, constr.Option, constr.IfNotExists) default: // Nothing to do now. } case ast.AlterTableDropForeignKey: // NOTE: we do not check `if not exists` and `if exists` for ForeignKey now. - err = e.DropForeignKey(sctx, ident, pmodel.NewCIStr(spec.Name)) + err = e.DropForeignKey(sctx, ident, ast.NewCIStr(spec.Name)) case ast.AlterTableModifyColumn: err = e.ModifyColumn(ctx, sctx, ident, spec) case ast.AlterTableChangeColumn: @@ -1874,7 +1873,7 @@ func (e *executor) AlterTable(ctx context.Context, sctx sessionctx.Context, stmt handledCharsetOrCollate = true case ast.TableOptionPlacementPolicy: placementPolicyRef = &model.PolicyRefInfo{ - Name: pmodel.NewCIStr(opt.StrValue), + Name: ast.NewCIStr(opt.StrValue), } case ast.TableOptionEngine: case ast.TableOptionRowFormat: @@ -1915,13 +1914,13 @@ func (e *executor) AlterTable(ctx context.Context, sctx sessionctx.Context, stmt if !variable.EnableCheckConstraint.Load() { sctx.GetSessionVars().StmtCtx.AppendWarning(errCheckConstraintIsOff) } else { - err = e.AlterCheckConstraint(sctx, ident, pmodel.NewCIStr(spec.Constraint.Name), spec.Constraint.Enforced) + err = e.AlterCheckConstraint(sctx, ident, ast.NewCIStr(spec.Constraint.Name), spec.Constraint.Enforced) } case ast.AlterTableDropCheck: if !variable.EnableCheckConstraint.Load() { sctx.GetSessionVars().StmtCtx.AppendWarning(errCheckConstraintIsOff) } else { - err = e.DropCheckConstraint(sctx, ident, pmodel.NewCIStr(spec.Constraint.Name)) + err = e.DropCheckConstraint(sctx, ident, ast.NewCIStr(spec.Constraint.Name)) } case ast.AlterTableWithValidation: sctx.GetSessionVars().StmtCtx.AppendWarning(dbterror.ErrUnsupportedAlterTableWithValidation) @@ -2223,7 +2222,7 @@ func (e *executor) AddTablePartitions(ctx sessionctx.Context, ident ast.Ident, s if pi == nil { return errors.Trace(dbterror.ErrPartitionMgmtOnNonpartitioned) } - if pi.Type == pmodel.PartitionTypeHash || pi.Type == pmodel.PartitionTypeKey { + if pi.Type == ast.PartitionTypeHash || pi.Type == ast.PartitionTypeKey { // Add partition for hash/key is actually a reorganize partition // operation and not a metadata only change! switch spec.Tp { @@ -2241,7 +2240,7 @@ func (e *executor) AddTablePartitions(ctx sessionctx.Context, ident ast.Ident, s if err != nil { return errors.Trace(err) } - if pi.Type == pmodel.PartitionTypeList { + if pi.Type == ast.PartitionTypeList { // TODO: make sure that checks in ddl_api and ddl_worker is the same. if meta.Partition.GetDefaultListPartition() != -1 { return dbterror.ErrGeneralUnsupportedDDL.GenWithStackByArgs("ADD List partition, already contains DEFAULT partition. Please use REORGANIZE PARTITION instead") @@ -2311,7 +2310,7 @@ func (e *executor) AddTablePartitions(ctx sessionctx.Context, ident ast.Ident, s // getReorganizedDefinitions return the definitions as they would look like after the REORGANIZE PARTITION is done. func getReorganizedDefinitions(pi *model.PartitionInfo, firstPartIdx, lastPartIdx int, idMap map[int]struct{}) []model.PartitionDefinition { tmpDefs := make([]model.PartitionDefinition, 0, len(pi.Definitions)+len(pi.AddingDefinitions)-len(idMap)) - if pi.Type == pmodel.PartitionTypeList { + if pi.Type == ast.PartitionTypeList { replaced := false for i := range pi.Definitions { if _, ok := idMap[i]; ok { @@ -2363,12 +2362,12 @@ func getReplacedPartitionIDs(names []string, pi *model.PartitionInfo) (firstPart } } switch pi.Type { - case pmodel.PartitionTypeRange: + case ast.PartitionTypeRange: if len(idMap) != (lastPartIdx - firstPartIdx + 1) { return 0, 0, nil, errors.Trace(dbterror.ErrGeneralUnsupportedDDL.GenWithStackByArgs( "REORGANIZE PARTITION of RANGE; not adjacent partitions")) } - case pmodel.PartitionTypeHash, pmodel.PartitionTypeKey: + case ast.PartitionTypeHash, ast.PartitionTypeKey: if len(idMap) != len(pi.Definitions) { return 0, 0, nil, errors.Trace(dbterror.ErrGeneralUnsupportedDDL.GenWithStackByArgs( "REORGANIZE PARTITION of HASH/RANGE; must reorganize all partitions")) @@ -2380,10 +2379,10 @@ func getReplacedPartitionIDs(names []string, pi *model.PartitionInfo) (firstPart func getPartitionInfoTypeNone() *model.PartitionInfo { return &model.PartitionInfo{ - Type: pmodel.PartitionTypeNone, + Type: ast.PartitionTypeNone, Enable: true, Definitions: []model.PartitionDefinition{{ - Name: pmodel.NewCIStr("pFullTable"), + Name: ast.NewCIStr("pFullTable"), Comment: "Intermediate partition during ALTER TABLE ... PARTITION BY ...", }}, Num: 1, @@ -2469,8 +2468,8 @@ func (e *executor) ReorganizePartitions(ctx sessionctx.Context, ident ast.Ident, return dbterror.ErrPartitionMgmtOnNonpartitioned } switch pi.Type { - case pmodel.PartitionTypeRange, pmodel.PartitionTypeList: - case pmodel.PartitionTypeHash, pmodel.PartitionTypeKey: + case ast.PartitionTypeRange, ast.PartitionTypeList: + case ast.PartitionTypeHash, ast.PartitionTypeKey: if spec.Tp != ast.AlterTableCoalescePartitions && spec.Tp != ast.AlterTableAddPartitions { return errors.Trace(dbterror.ErrUnsupportedReorganizePartition) @@ -2550,13 +2549,13 @@ func (e *executor) RemovePartitioning(ctx sessionctx.Context, ident ast.Ident, s newSpec.Tp = spec.Tp defs := make([]*ast.PartitionDefinition, 1) defs[0] = &ast.PartitionDefinition{} - defs[0].Name = pmodel.NewCIStr("CollapsedPartitions") + defs[0].Name = ast.NewCIStr("CollapsedPartitions") newSpec.PartDefinitions = defs partNames := make([]string, len(pi.Definitions)) for i := range pi.Definitions { partNames[i] = pi.Definitions[i].Name.L } - meta.Partition.Type = pmodel.PartitionTypeNone + meta.Partition.Type = ast.PartitionTypeNone partInfo, err := BuildAddedPartitionInfo(ctx.GetExprCtx(), meta, newSpec) if err != nil { return errors.Trace(err) @@ -2610,7 +2609,7 @@ func checkReorgPartitionDefs(ctx sessionctx.Context, action model.ActionType, tb return errors.Trace(err) } if action == model.ActionReorganizePartition { - if pi.Type == pmodel.PartitionTypeRange { + if pi.Type == ast.PartitionTypeRange { if lastPartIdx == len(pi.Definitions)-1 { // Last partition dropped, OK to change the end range // Also includes MAXVALUE @@ -2678,7 +2677,7 @@ func (e *executor) CoalescePartitions(sctx sessionctx.Context, ident ast.Ident, } switch pi.Type { - case pmodel.PartitionTypeHash, pmodel.PartitionTypeKey: + case ast.PartitionTypeHash, ast.PartitionTypeKey: return e.hashPartitionManagement(sctx, ident, spec, pi) // Coalesce partition can only be used on hash/key partitions. @@ -2689,7 +2688,7 @@ func (e *executor) CoalescePartitions(sctx sessionctx.Context, ident ast.Ident, func (e *executor) hashPartitionManagement(sctx sessionctx.Context, ident ast.Ident, spec *ast.AlterTableSpec, pi *model.PartitionInfo) error { newSpec := *spec - newSpec.PartitionNames = make([]pmodel.CIStr, len(pi.Definitions)) + newSpec.PartitionNames = make([]ast.CIStr, len(pi.Definitions)) for i := 0; i < len(pi.Definitions); i++ { // reorganize ALL partitions into the new number of partitions newSpec.PartitionNames[i] = pi.Definitions[i].Name @@ -3207,7 +3206,7 @@ func checkIsDroppableColumn(ctx sessionctx.Context, is infoschema.InfoSchema, sc } // checkDropColumnWithPartitionConstraint is used to check the partition constraint of the drop column. -func checkDropColumnWithPartitionConstraint(t table.Table, colName pmodel.CIStr) error { +func checkDropColumnWithPartitionConstraint(t table.Table, colName ast.CIStr) error { if t.Meta().Partition == nil { return nil } @@ -3275,7 +3274,7 @@ func checkModifyCharsetAndCollation(toCharset, toCollate, origCharset, origColla func (e *executor) getModifiableColumnJob( ctx context.Context, sctx sessionctx.Context, - ident ast.Ident, originalColName pmodel.CIStr, spec *ast.AlterTableSpec, + ident ast.Ident, originalColName ast.CIStr, spec *ast.AlterTableSpec, ) (*JobWrapper, error) { is := e.infoCache.GetLatest() schema, ok := is.SchemaByName(ident.Schema) @@ -3625,7 +3624,7 @@ func shouldModifyTiFlashReplica(tbReplicaInfo *model.TiFlashReplicaInfo, replica } // addHypoTiFlashReplicaIntoCtx adds this hypothetical tiflash replica into this ctx. -func (*executor) setHypoTiFlashReplica(ctx sessionctx.Context, schemaName, tableName pmodel.CIStr, replicaInfo *ast.TiFlashReplicaSpec) error { +func (*executor) setHypoTiFlashReplica(ctx sessionctx.Context, schemaName, tableName ast.CIStr, replicaInfo *ast.TiFlashReplicaSpec) error { sctx := ctx.GetSessionVars() if sctx.HypoTiFlashReplicas == nil { sctx.HypoTiFlashReplicas = make(map[string]map[string]struct{}) @@ -3780,7 +3779,7 @@ func (e *executor) AlterTableRemoveTTL(ctx sessionctx.Context, ident ast.Ident) return nil } -func isTableTiFlashSupported(dbName pmodel.CIStr, tbl *model.TableInfo) error { +func isTableTiFlashSupported(dbName ast.CIStr, tbl *model.TableInfo) error { // Memory tables and system tables are not supported by TiFlash if util.IsMemOrSysDB(dbName.L) { return errors.Trace(dbterror.ErrUnsupportedTiFlashOperationForSysOrMemTable) @@ -4508,7 +4507,7 @@ func getAnonymousIndexPrefix(isVector bool) string { } // GetName4AnonymousIndex returns a valid name for anonymous index. -func GetName4AnonymousIndex(t table.Table, colName pmodel.CIStr, idxName pmodel.CIStr) pmodel.CIStr { +func GetName4AnonymousIndex(t table.Table, colName ast.CIStr, idxName ast.CIStr) ast.CIStr { // `id` is used to indicated the index name's suffix. id := 2 l := len(t.Indices()) @@ -4519,14 +4518,14 @@ func GetName4AnonymousIndex(t table.Table, colName pmodel.CIStr, idxName pmodel. id = 3 } if strings.EqualFold(indexName.L, mysql.PrimaryKeyName) { - indexName = pmodel.NewCIStr(fmt.Sprintf("%s_%d", colName.O, id)) + indexName = ast.NewCIStr(fmt.Sprintf("%s_%d", colName.O, id)) id = 3 } for i := 0; i < l; i++ { if t.Indices()[i].Meta().Name.L == indexName.L { - indexName = pmodel.NewCIStr(fmt.Sprintf("%s_%d", colName.O, id)) + indexName = ast.NewCIStr(fmt.Sprintf("%s_%d", colName.O, id)) if err := checkTooLongIndex(indexName); err != nil { - indexName = GetName4AnonymousIndex(t, pmodel.NewCIStr(colName.O[:30]), pmodel.NewCIStr(fmt.Sprintf("%s_%d", colName.O[:30], 2))) + indexName = GetName4AnonymousIndex(t, ast.NewCIStr(colName.O[:30]), ast.NewCIStr(fmt.Sprintf("%s_%d", colName.O[:30], 2))) } i = -1 id++ @@ -4558,9 +4557,9 @@ func checkCreateUniqueGlobalIndex(ec errctx.Context, tblInfo *model.TableInfo, i return nil } -func (e *executor) CreatePrimaryKey(ctx sessionctx.Context, ti ast.Ident, indexName pmodel.CIStr, +func (e *executor) CreatePrimaryKey(ctx sessionctx.Context, ti ast.Ident, indexName ast.CIStr, indexPartSpecifications []*ast.IndexPartSpecification, indexOption *ast.IndexOption) error { - if indexOption != nil && indexOption.PrimaryKeyTp == pmodel.PrimaryKeyTypeClustered { + if indexOption != nil && indexOption.PrimaryKeyTp == ast.PrimaryKeyTypeClustered { return dbterror.ErrUnsupportedModifyPrimaryKey.GenWithStack("Adding clustered primary key is not supported. " + "Please consider adding NONCLUSTERED primary key instead") } @@ -4573,7 +4572,7 @@ func (e *executor) CreatePrimaryKey(ctx sessionctx.Context, ti ast.Ident, indexN return dbterror.ErrTooLongIdent.GenWithStackByArgs(mysql.PrimaryKeyName) } - indexName = pmodel.NewCIStr(mysql.PrimaryKeyName) + indexName = ast.NewCIStr(mysql.PrimaryKeyName) if indexInfo := t.Meta().FindIndexByName(indexName.L); indexInfo != nil || // If the table's PKIsHandle is true, it also means that this table has a primary key. t.Meta().PKIsHandle { @@ -4659,15 +4658,15 @@ func (e *executor) CreatePrimaryKey(ctx sessionctx.Context, ti ast.Ident, indexN return errors.Trace(err) } -func checkIndexNameAndColumns(ctx *metabuild.Context, t table.Table, indexName pmodel.CIStr, - indexPartSpecifications []*ast.IndexPartSpecification, isVector, ifNotExists bool) (pmodel.CIStr, []*model.ColumnInfo, error) { +func checkIndexNameAndColumns(ctx *metabuild.Context, t table.Table, indexName ast.CIStr, + indexPartSpecifications []*ast.IndexPartSpecification, isVector, ifNotExists bool) (ast.CIStr, []*model.ColumnInfo, error) { // Deal with anonymous index. if len(indexName.L) == 0 { - colName := pmodel.NewCIStr(getAnonymousIndexPrefix(isVector)) + colName := ast.NewCIStr(getAnonymousIndexPrefix(isVector)) if indexPartSpecifications[0].Column != nil { colName = indexPartSpecifications[0].Column.Name } - indexName = GetName4AnonymousIndex(t, colName, pmodel.NewCIStr("")) + indexName = GetName4AnonymousIndex(t, colName, ast.NewCIStr("")) } var err error @@ -4682,13 +4681,13 @@ func checkIndexNameAndColumns(ctx *metabuild.Context, t table.Table, indexName p } if ifNotExists { ctx.AppendNote(err) - return pmodel.CIStr{}, nil, nil + return ast.CIStr{}, nil, nil } - return pmodel.CIStr{}, nil, err + return ast.CIStr{}, nil, err } if err = checkTooLongIndex(indexName); err != nil { - return pmodel.CIStr{}, nil, errors.Trace(err) + return ast.CIStr{}, nil, errors.Trace(err) } // Build hidden columns if necessary. @@ -4696,11 +4695,11 @@ func checkIndexNameAndColumns(ctx *metabuild.Context, t table.Table, indexName p if !isVector { hiddenCols, err = buildHiddenColumnInfoWithCheck(ctx, indexPartSpecifications, indexName, t.Meta(), t.Cols()) if err != nil { - return pmodel.CIStr{}, nil, err + return ast.CIStr{}, nil, err } } if err = checkAddColumnTooManyColumns(len(t.Cols()) + len(hiddenCols)); err != nil { - return pmodel.CIStr{}, nil, errors.Trace(err) + return ast.CIStr{}, nil, errors.Trace(err) } return indexName, hiddenCols, nil @@ -4723,7 +4722,7 @@ func checkTableTypeForVectorIndex(tblInfo *model.TableInfo) error { return nil } -func (e *executor) createVectorIndex(ctx sessionctx.Context, ti ast.Ident, indexName pmodel.CIStr, +func (e *executor) createVectorIndex(ctx sessionctx.Context, ti ast.Ident, indexName ast.CIStr, indexPartSpecifications []*ast.IndexPartSpecification, indexOption *ast.IndexOption, ifNotExists bool) error { schema, t, err := e.getSchemaAndTableByIdent(ti) if err != nil { @@ -4807,12 +4806,12 @@ func buildAddIndexJobWithoutTypeAndArgs(ctx sessionctx.Context, schema *model.DB func (e *executor) CreateIndex(ctx sessionctx.Context, stmt *ast.CreateIndexStmt) error { ident := ast.Ident{Schema: stmt.Table.Schema, Name: stmt.Table.Name} - return e.createIndex(ctx, ident, stmt.KeyType, pmodel.NewCIStr(stmt.IndexName), + return e.createIndex(ctx, ident, stmt.KeyType, ast.NewCIStr(stmt.IndexName), stmt.IndexPartSpecifications, stmt.IndexOption, stmt.IfNotExists) } // addHypoIndexIntoCtx adds this index as a hypo-index into this ctx. -func (*executor) addHypoIndexIntoCtx(ctx sessionctx.Context, schemaName, tableName pmodel.CIStr, indexInfo *model.IndexInfo) error { +func (*executor) addHypoIndexIntoCtx(ctx sessionctx.Context, schemaName, tableName ast.CIStr, indexInfo *model.IndexInfo) error { sctx := ctx.GetSessionVars() indexName := indexInfo.Name @@ -4833,7 +4832,7 @@ func (*executor) addHypoIndexIntoCtx(ctx sessionctx.Context, schemaName, tableNa return nil } -func (e *executor) createIndex(ctx sessionctx.Context, ti ast.Ident, keyType ast.IndexKeyType, indexName pmodel.CIStr, +func (e *executor) createIndex(ctx sessionctx.Context, ti ast.Ident, keyType ast.IndexKeyType, indexName ast.CIStr, indexPartSpecifications []*ast.IndexPartSpecification, indexOption *ast.IndexOption, ifNotExists bool) error { // not support Spatial and FullText index if keyType == ast.IndexKeyTypeFullText || keyType == ast.IndexKeyTypeSpatial { @@ -4884,7 +4883,7 @@ func (e *executor) createIndex(ctx sessionctx.Context, ti ast.Ident, keyType ast } } - if indexOption != nil && indexOption.Tp == pmodel.IndexTypeHypo { // for hypo-index + if indexOption != nil && indexOption.Tp == ast.IndexTypeHypo { // for hypo-index indexInfo, err := BuildIndexInfo(metaBuildCtx, tblInfo, indexName, false, unique, false, indexPartSpecifications, indexOption, model.StatePublic) if err != nil { @@ -5076,7 +5075,7 @@ func buildIndexPresplitOpt(indexOpt *ast.IndexOption) (*model.IndexArgSplitOpt, // LastReorgMetaFastReorgDisabled is used for test. var LastReorgMetaFastReorgDisabled bool -func buildFKInfo(fkName pmodel.CIStr, keys []*ast.IndexPartSpecification, refer *ast.ReferenceDef, cols []*table.Column) (*model.FKInfo, error) { +func buildFKInfo(fkName ast.CIStr, keys []*ast.IndexPartSpecification, refer *ast.ReferenceDef, cols []*table.Column) (*model.FKInfo, error) { if len(keys) != len(refer.IndexPartSpecifications) { return nil, infoschema.ErrForeignKeyNotMatch.GenWithStackByArgs(fkName, "Key reference and table reference don't match") } @@ -5104,7 +5103,7 @@ func buildFKInfo(fkName pmodel.CIStr, keys []*ast.IndexPartSpecification, refer Name: fkName, RefSchema: refer.Table.Schema, RefTable: refer.Table.Name, - Cols: make([]pmodel.CIStr, len(keys)), + Cols: make([]ast.CIStr, len(keys)), } if variable.EnableForeignKey.Load() { fkInfo.Version = model.FKVersion1 @@ -5125,12 +5124,12 @@ func buildFKInfo(fkName pmodel.CIStr, keys []*ast.IndexPartSpecification, refer // Check wrong reference options of foreign key on stored generated columns switch refer.OnUpdate.ReferOpt { - case pmodel.ReferOptionCascade, pmodel.ReferOptionSetNull, pmodel.ReferOptionSetDefault: + case ast.ReferOptionCascade, ast.ReferOptionSetNull, ast.ReferOptionSetDefault: //nolint: gosec return nil, dbterror.ErrWrongFKOptionForGeneratedColumn.GenWithStackByArgs("ON UPDATE " + refer.OnUpdate.ReferOpt.String()) } switch refer.OnDelete.ReferOpt { - case pmodel.ReferOptionSetNull, pmodel.ReferOptionSetDefault: + case ast.ReferOptionSetNull, ast.ReferOptionSetDefault: //nolint: gosec return nil, dbterror.ErrWrongFKOptionForGeneratedColumn.GenWithStackByArgs("ON DELETE " + refer.OnDelete.ReferOpt.String()) } @@ -5139,11 +5138,11 @@ func buildFKInfo(fkName pmodel.CIStr, keys []*ast.IndexPartSpecification, refer // Check wrong reference options of foreign key on base columns of stored generated columns if _, ok := baseCols[col.Name.L]; ok { switch refer.OnUpdate.ReferOpt { - case pmodel.ReferOptionCascade, pmodel.ReferOptionSetNull, pmodel.ReferOptionSetDefault: + case ast.ReferOptionCascade, ast.ReferOptionSetNull, ast.ReferOptionSetDefault: return nil, infoschema.ErrCannotAddForeign } switch refer.OnDelete.ReferOpt { - case pmodel.ReferOptionCascade, pmodel.ReferOptionSetNull, pmodel.ReferOptionSetDefault: + case ast.ReferOptionCascade, ast.ReferOptionSetNull, ast.ReferOptionSetDefault: return nil, infoschema.ErrCannotAddForeign } } @@ -5152,13 +5151,13 @@ func buildFKInfo(fkName pmodel.CIStr, keys []*ast.IndexPartSpecification, refer if col == nil { return nil, dbterror.ErrKeyColumnDoesNotExits.GenWithStackByArgs(key.Column.Name) } - if mysql.HasNotNullFlag(col.GetFlag()) && (refer.OnDelete.ReferOpt == pmodel.ReferOptionSetNull || refer.OnUpdate.ReferOpt == pmodel.ReferOptionSetNull) { + if mysql.HasNotNullFlag(col.GetFlag()) && (refer.OnDelete.ReferOpt == ast.ReferOptionSetNull || refer.OnUpdate.ReferOpt == ast.ReferOptionSetNull) { return nil, infoschema.ErrForeignKeyColumnNotNull.GenWithStackByArgs(col.Name.O, fkName) } fkInfo.Cols[i] = key.Column.Name } - fkInfo.RefCols = make([]pmodel.CIStr, len(refer.IndexPartSpecifications)) + fkInfo.RefCols = make([]ast.CIStr, len(refer.IndexPartSpecifications)) for i, key := range refer.IndexPartSpecifications { if err := checkTooLongColumn(key.Column.Name); err != nil { return nil, err @@ -5172,7 +5171,7 @@ func buildFKInfo(fkName pmodel.CIStr, keys []*ast.IndexPartSpecification, refer return fkInfo, nil } -func (e *executor) CreateForeignKey(ctx sessionctx.Context, ti ast.Ident, fkName pmodel.CIStr, keys []*ast.IndexPartSpecification, refer *ast.ReferenceDef) error { +func (e *executor) CreateForeignKey(ctx sessionctx.Context, ti ast.Ident, fkName ast.CIStr, keys []*ast.IndexPartSpecification, refer *ast.ReferenceDef) error { is := e.infoCache.GetLatest() schema, ok := is.SchemaByName(ti.Schema) if !ok { @@ -5188,7 +5187,7 @@ func (e *executor) CreateForeignKey(ctx sessionctx.Context, ti ast.Ident, fkName } if fkName.L == "" { - fkName = pmodel.NewCIStr(fmt.Sprintf("fk_%d", t.Meta().MaxForeignKeyID+1)) + fkName = ast.NewCIStr(fmt.Sprintf("fk_%d", t.Meta().MaxForeignKeyID+1)) } err = checkFKDupName(t.Meta(), fkName) if err != nil { @@ -5253,7 +5252,7 @@ func (e *executor) CreateForeignKey(ctx sessionctx.Context, ti ast.Ident, fkName return errors.Trace(err) } -func (e *executor) DropForeignKey(ctx sessionctx.Context, ti ast.Ident, fkName pmodel.CIStr) error { +func (e *executor) DropForeignKey(ctx sessionctx.Context, ti ast.Ident, fkName ast.CIStr) error { is := e.infoCache.GetLatest() schema, ok := is.SchemaByName(ti.Schema) if !ok { @@ -5296,7 +5295,7 @@ func (e *executor) DropForeignKey(ctx sessionctx.Context, ti ast.Ident, fkName p func (e *executor) DropIndex(ctx sessionctx.Context, stmt *ast.DropIndexStmt) error { ti := ast.Ident{Schema: stmt.Table.Schema, Name: stmt.Table.Name} - err := e.dropIndex(ctx, ti, pmodel.NewCIStr(stmt.IndexName), stmt.IfExists, stmt.IsHypo) + err := e.dropIndex(ctx, ti, ast.NewCIStr(stmt.IndexName), stmt.IfExists, stmt.IsHypo) if (infoschema.ErrDatabaseNotExists.Equal(err) || infoschema.ErrTableNotExists.Equal(err)) && stmt.IfExists { err = nil } @@ -5304,7 +5303,7 @@ func (e *executor) DropIndex(ctx sessionctx.Context, stmt *ast.DropIndexStmt) er } // dropHypoIndexFromCtx drops this hypo-index from this ctx. -func (*executor) dropHypoIndexFromCtx(ctx sessionctx.Context, schema, table, index pmodel.CIStr, ifExists bool) error { +func (*executor) dropHypoIndexFromCtx(ctx sessionctx.Context, schema, table, index ast.CIStr, ifExists bool) error { sctx := ctx.GetSessionVars() if sctx.HypoIndexes != nil && sctx.HypoIndexes[schema.L] != nil && @@ -5321,7 +5320,7 @@ func (*executor) dropHypoIndexFromCtx(ctx sessionctx.Context, schema, table, ind // dropIndex drops the specified index. // isHypo is used to indicate whether this operation is for a hypo-index. -func (e *executor) dropIndex(ctx sessionctx.Context, ti ast.Ident, indexName pmodel.CIStr, ifExist, isHypo bool) error { +func (e *executor) dropIndex(ctx sessionctx.Context, ti ast.Ident, indexName ast.CIStr, ifExist, isHypo bool) error { is := e.infoCache.GetLatest() schema, ok := is.SchemaByName(ti.Schema) if !ok { @@ -5394,7 +5393,7 @@ func (e *executor) dropIndex(ctx sessionctx.Context, ti ast.Ident, indexName pmo } // CheckIsDropPrimaryKey checks if we will drop PK, there are many PK implementations so we provide a helper function. -func CheckIsDropPrimaryKey(indexName pmodel.CIStr, indexInfo *model.IndexInfo, t table.Table) (bool, error) { +func CheckIsDropPrimaryKey(indexName ast.CIStr, indexInfo *model.IndexInfo, t table.Table) (bool, error) { var isPK bool if indexName.L == strings.ToLower(mysql.PrimaryKeyName) && // Before we fixed #14243, there might be a general index named `primary` but not a primary key. @@ -5460,9 +5459,9 @@ func validateGlobalIndexWithGeneratedColumns(ec errctx.Context, tblInfo *model.T func BuildAddedPartitionInfo(ctx expression.BuildContext, meta *model.TableInfo, spec *ast.AlterTableSpec) (*model.PartitionInfo, error) { numParts := uint64(0) switch meta.Partition.Type { - case pmodel.PartitionTypeNone: + case ast.PartitionTypeNone: // OK - case pmodel.PartitionTypeList: + case ast.PartitionTypeList: if len(spec.PartDefinitions) == 0 { return nil, ast.ErrPartitionsMustBeDefined.GenWithStackByArgs(meta.Partition.Type) } @@ -5471,7 +5470,7 @@ func BuildAddedPartitionInfo(ctx expression.BuildContext, meta *model.TableInfo, return nil, err } - case pmodel.PartitionTypeRange: + case ast.PartitionTypeRange: if spec.Tp == ast.AlterTableAddLastPartition { err := buildAddedPartitionDefs(ctx, meta, spec) if err != nil { @@ -5483,7 +5482,7 @@ func BuildAddedPartitionInfo(ctx expression.BuildContext, meta *model.TableInfo, return nil, ast.ErrPartitionsMustBeDefined.GenWithStackByArgs(meta.Partition.Type) } } - case pmodel.PartitionTypeHash, pmodel.PartitionTypeKey: + case ast.PartitionTypeHash, ast.PartitionTypeKey: switch spec.Tp { case ast.AlterTableRemovePartitioning: numParts = 1 @@ -5893,7 +5892,7 @@ func (e *executor) DropSequence(ctx sessionctx.Context, stmt *ast.DropSequenceSt return e.dropTableObject(ctx, stmt.Sequences, stmt.IfExists, sequenceObject) } -func (e *executor) AlterIndexVisibility(ctx sessionctx.Context, ident ast.Ident, indexName pmodel.CIStr, visibility ast.IndexVisibility) error { +func (e *executor) AlterIndexVisibility(ctx sessionctx.Context, ident ast.Ident, indexName ast.CIStr, visibility ast.IndexVisibility) error { schema, tb, err := e.getSchemaAndTableByIdent(ident) if err != nil { return err @@ -6024,7 +6023,7 @@ func (e *executor) AlterTablePartitionOptions(ctx sessionctx.Context, ident ast. switch op.Tp { case ast.TableOptionPlacementPolicy: policyRefInfo = &model.PolicyRefInfo{ - Name: pmodel.NewCIStr(op.StrValue), + Name: ast.NewCIStr(op.StrValue), } default: return errors.Trace(errors.New("unknown partition option")) @@ -6457,7 +6456,7 @@ func (e *executor) AlterTableNoCache(ctx sessionctx.Context, ti ast.Ident) (err return e.doDDLJob2(ctx, job, &model.EmptyArgs{}) } -func (e *executor) CreateCheckConstraint(ctx sessionctx.Context, ti ast.Ident, constrName pmodel.CIStr, constr *ast.Constraint) error { +func (e *executor) CreateCheckConstraint(ctx sessionctx.Context, ti ast.Ident, constrName ast.CIStr, constr *ast.Constraint) error { schema, t, err := e.getSchemaAndTableByIdent(ti) if err != nil { return errors.Trace(err) @@ -6485,13 +6484,13 @@ func (e *executor) CreateCheckConstraint(ctx sessionctx.Context, ti ast.Ident, c } dependedColsMap := findDependentColsInExpr(constr.Expr) - dependedCols := make([]pmodel.CIStr, 0, len(dependedColsMap)) + dependedCols := make([]ast.CIStr, 0, len(dependedColsMap)) for k := range dependedColsMap { if _, ok := existedColsMap[k]; !ok { // The table constraint depended on a non-existed column. return dbterror.ErrBadField.GenWithStackByArgs(k, "check constraint "+constr.Name+" expression") } - dependedCols = append(dependedCols, pmodel.NewCIStr(k)) + dependedCols = append(dependedCols, ast.NewCIStr(k)) } // build constraint meta info. @@ -6533,7 +6532,7 @@ func (e *executor) CreateCheckConstraint(ctx sessionctx.Context, ti ast.Ident, c return errors.Trace(err) } -func (e *executor) DropCheckConstraint(ctx sessionctx.Context, ti ast.Ident, constrName pmodel.CIStr) error { +func (e *executor) DropCheckConstraint(ctx sessionctx.Context, ti ast.Ident, constrName ast.CIStr) error { is := e.infoCache.GetLatest() schema, ok := is.SchemaByName(ti.Schema) if !ok { @@ -6569,7 +6568,7 @@ func (e *executor) DropCheckConstraint(ctx sessionctx.Context, ti ast.Ident, con return errors.Trace(err) } -func (e *executor) AlterCheckConstraint(ctx sessionctx.Context, ti ast.Ident, constrName pmodel.CIStr, enforced bool) error { +func (e *executor) AlterCheckConstraint(ctx sessionctx.Context, ti ast.Ident, constrName ast.CIStr, enforced bool) error { is := e.infoCache.GetLatest() schema, ok := is.SchemaByName(ti.Schema) if !ok { diff --git a/pkg/ddl/executor_nokit_test.go b/pkg/ddl/executor_nokit_test.go index d89426fad0a6e..ac1298d3aa472 100644 --- a/pkg/ddl/executor_nokit_test.go +++ b/pkg/ddl/executor_nokit_test.go @@ -21,7 +21,7 @@ import ( "testing" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/stretchr/testify/require" ) @@ -75,14 +75,14 @@ func TestMergeCreateTableJobsOfSameSchema(t *testing.T) { Type: model.ActionCreateTable, BinlogInfo: &model.HistoryInfo{}, Query: "create table db1.t1 (c1 int, c2 int)", - }, &model.CreateTableArgs{TableInfo: &model.TableInfo{Name: pmodel.CIStr{O: "t1", L: "t1"}}}, false) + }, &model.CreateTableArgs{TableInfo: &model.TableInfo{Name: ast.CIStr{O: "t1", L: "t1"}}}, false) job2 := NewJobWrapperWithArgs(&model.Job{ Version: model.GetJobVerInUse(), SchemaID: 1, Type: model.ActionCreateTable, BinlogInfo: &model.HistoryInfo{}, Query: "create table db1.t2 (c1 int, c2 int);", - }, &model.CreateTableArgs{TableInfo: &model.TableInfo{Name: pmodel.CIStr{O: "t2", L: "t2"}}, FKCheck: true}, false) + }, &model.CreateTableArgs{TableInfo: &model.TableInfo{Name: ast.CIStr{O: "t2", L: "t2"}}, FKCheck: true}, false) job, err := mergeCreateTableJobsOfSameSchema([]*JobWrapper{job1, job2}) require.NoError(t, err) require.Equal(t, "create table db1.t1 (c1 int, c2 int); create table db1.t2 (c1 int, c2 int);", job.Query) @@ -102,10 +102,10 @@ func TestMergeCreateTableJobs(t *testing.T) { t.Run("non create table are not merged", func(t *testing.T) { jobWs := []*JobWrapper{ {Job: &model.Job{Version: model.GetJobVerInUse(), SchemaName: "db", Type: model.ActionCreateTable}, - JobArgs: &model.CreateTableArgs{TableInfo: &model.TableInfo{Name: pmodel.NewCIStr("t1")}}}, + JobArgs: &model.CreateTableArgs{TableInfo: &model.TableInfo{Name: ast.NewCIStr("t1")}}}, {Job: &model.Job{SchemaName: "db", Type: model.ActionAddColumn}}, {Job: &model.Job{Version: model.GetJobVerInUse(), SchemaName: "db", Type: model.ActionCreateTable}, - JobArgs: &model.CreateTableArgs{TableInfo: &model.TableInfo{Name: pmodel.NewCIStr("t2")}}}, + JobArgs: &model.CreateTableArgs{TableInfo: &model.TableInfo{Name: ast.NewCIStr("t2")}}}, } newWs, err := mergeCreateTableJobs(jobWs) require.NoError(t, err) @@ -123,9 +123,9 @@ func TestMergeCreateTableJobs(t *testing.T) { t.Run("jobs of pre allocated ids are not merged", func(t *testing.T) { jobWs := []*JobWrapper{ {Job: &model.Job{Version: model.GetJobVerInUse(), SchemaName: "db", Type: model.ActionCreateTable}, - JobArgs: &model.CreateTableArgs{TableInfo: &model.TableInfo{Name: pmodel.NewCIStr("t1")}}, IDAllocated: true}, + JobArgs: &model.CreateTableArgs{TableInfo: &model.TableInfo{Name: ast.NewCIStr("t1")}}, IDAllocated: true}, {Job: &model.Job{Version: model.GetJobVerInUse(), SchemaName: "db", Type: model.ActionCreateTable}, - JobArgs: &model.CreateTableArgs{TableInfo: &model.TableInfo{Name: pmodel.NewCIStr("t2")}}}, + JobArgs: &model.CreateTableArgs{TableInfo: &model.TableInfo{Name: ast.NewCIStr("t2")}}}, } newWs, err := mergeCreateTableJobs(jobWs) slices.SortFunc(newWs, func(a, b *JobWrapper) int { @@ -142,7 +142,7 @@ func TestMergeCreateTableJobs(t *testing.T) { {Job: &model.Job{Version: model.GetJobVerInUse(), SchemaName: "db", Type: model.ActionCreateTable}, JobArgs: &model.CreateTableArgs{TableInfo: &model.TableInfo{ForeignKeys: []*model.FKInfo{{}}}}}, {Job: &model.Job{Version: model.GetJobVerInUse(), SchemaName: "db", Type: model.ActionCreateTable}, - JobArgs: &model.CreateTableArgs{TableInfo: &model.TableInfo{Name: pmodel.NewCIStr("t2")}}}, + JobArgs: &model.CreateTableArgs{TableInfo: &model.TableInfo{Name: ast.NewCIStr("t2")}}}, } newWs, err := mergeCreateTableJobs(jobWs) slices.SortFunc(newWs, func(a, b *JobWrapper) int { @@ -157,9 +157,9 @@ func TestMergeCreateTableJobs(t *testing.T) { t.Run("jobs of different schema are not merged", func(t *testing.T) { jobWs := []*JobWrapper{ {Job: &model.Job{Version: model.GetJobVerInUse(), SchemaName: "db1", Type: model.ActionCreateTable}, - JobArgs: &model.CreateTableArgs{TableInfo: &model.TableInfo{Name: pmodel.NewCIStr("t1")}}}, + JobArgs: &model.CreateTableArgs{TableInfo: &model.TableInfo{Name: ast.NewCIStr("t1")}}}, {Job: &model.Job{Version: model.GetJobVerInUse(), SchemaName: "db2", Type: model.ActionCreateTable}, - JobArgs: &model.CreateTableArgs{TableInfo: &model.TableInfo{Name: pmodel.NewCIStr("t2")}}}, + JobArgs: &model.CreateTableArgs{TableInfo: &model.TableInfo{Name: ast.NewCIStr("t2")}}}, } newWs, err := mergeCreateTableJobs(jobWs) slices.SortFunc(newWs, func(a, b *JobWrapper) int { @@ -173,7 +173,7 @@ func TestMergeCreateTableJobs(t *testing.T) { jobWs := make([]*JobWrapper, 0, 100) jobWs = append(jobWs, NewJobWrapper(&model.Job{SchemaName: "db0", Type: model.ActionAddColumn}, false)) jobW := NewJobWrapperWithArgs(&model.Job{Version: model.GetJobVerInUse(), SchemaName: "db1", Type: model.ActionCreateTable}, - &model.CreateTableArgs{TableInfo: &model.TableInfo{Name: pmodel.NewCIStr("t1")}}, true) + &model.CreateTableArgs{TableInfo: &model.TableInfo{Name: ast.NewCIStr("t1")}}, true) jobWs = append(jobWs, jobW) jobW = NewJobWrapperWithArgs(&model.Job{Version: model.GetJobVerInUse(), SchemaName: "db2", Type: model.ActionCreateTable}, &model.CreateTableArgs{TableInfo: &model.TableInfo{ForeignKeys: []*model.FKInfo{{}}}}, false) @@ -186,7 +186,7 @@ func TestMergeCreateTableJobs(t *testing.T) { for i := 0; i < cnt; i++ { tblName := fmt.Sprintf("t%d", i) jobW := NewJobWrapperWithArgs(&model.Job{Version: model.GetJobVerInUse(), SchemaName: db, Type: model.ActionCreateTable}, - &model.CreateTableArgs{TableInfo: &model.TableInfo{Name: pmodel.NewCIStr(tblName)}}, false) + &model.CreateTableArgs{TableInfo: &model.TableInfo{Name: ast.NewCIStr(tblName)}}, false) jobWs = append(jobWs, jobW) } } diff --git a/pkg/ddl/executor_test.go b/pkg/ddl/executor_test.go index 8448a211e6349..bc40f0802b90d 100644 --- a/pkg/ddl/executor_test.go +++ b/pkg/ddl/executor_test.go @@ -27,7 +27,7 @@ import ( "github.com/pingcap/tidb/pkg/ddl" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" sessiontypes "github.com/pingcap/tidb/pkg/session/types" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/testkit" @@ -263,7 +263,7 @@ func TestHandleLockTable(t *testing.T) { se := tk.Session().(sessionctx.Context) require.False(t, se.HasLockedTables()) - checkTableLocked := func(tblID int64, tp pmodel.TableLockType) { + checkTableLocked := func(tblID int64, tp ast.TableLockType) { locked, lockType := se.CheckTableLocked(tblID) require.True(t, locked) require.Equal(t, tp, lockType) @@ -291,28 +291,28 @@ func TestHandleLockTable(t *testing.T) { t.Run("ddl success", func(t *testing.T) { se.ReleaseAllTableLocks() require.False(t, se.HasLockedTables()) - se.AddTableLock([]model.TableLockTpInfo{{SchemaID: 1, TableID: 1, Tp: pmodel.TableLockRead}}) + se.AddTableLock([]model.TableLockTpInfo{{SchemaID: 1, TableID: 1, Tp: ast.TableLockRead}}) ddl.HandleLockTablesOnSuccessSubmit(tk.Session(), jobW) require.Len(t, se.GetAllTableLocks(), 2) - checkTableLocked(1, pmodel.TableLockRead) - checkTableLocked(2, pmodel.TableLockRead) + checkTableLocked(1, ast.TableLockRead) + checkTableLocked(2, ast.TableLockRead) ddl.HandleLockTablesOnFinish(se, jobW, nil) require.Len(t, se.GetAllTableLocks(), 1) - checkTableLocked(2, pmodel.TableLockRead) + checkTableLocked(2, ast.TableLockRead) }) t.Run("ddl fail", func(t *testing.T) { se.ReleaseAllTableLocks() require.False(t, se.HasLockedTables()) - se.AddTableLock([]model.TableLockTpInfo{{SchemaID: 1, TableID: 1, Tp: pmodel.TableLockRead}}) + se.AddTableLock([]model.TableLockTpInfo{{SchemaID: 1, TableID: 1, Tp: ast.TableLockRead}}) ddl.HandleLockTablesOnSuccessSubmit(tk.Session(), jobW) require.Len(t, se.GetAllTableLocks(), 2) - checkTableLocked(1, pmodel.TableLockRead) - checkTableLocked(2, pmodel.TableLockRead) + checkTableLocked(1, ast.TableLockRead) + checkTableLocked(2, ast.TableLockRead) ddl.HandleLockTablesOnFinish(se, jobW, errors.New("test error")) require.Len(t, se.GetAllTableLocks(), 1) - checkTableLocked(1, pmodel.TableLockRead) + checkTableLocked(1, ast.TableLockRead) }) } diff --git a/pkg/ddl/foreign_key.go b/pkg/ddl/foreign_key.go index c63f59b586a37..249a4d8977591 100644 --- a/pkg/ddl/foreign_key.go +++ b/pkg/ddl/foreign_key.go @@ -26,7 +26,6 @@ import ( "github.com/pingcap/tidb/pkg/meta" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/ast" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/variable" @@ -111,7 +110,7 @@ func onDropForeignKey(jobCtx *jobContext, job *model.Job) (ver int64, _ error) { return dropForeignKey(jobCtx, job, tblInfo, args.FkName) } -func dropForeignKey(jobCtx *jobContext, job *model.Job, tblInfo *model.TableInfo, fkName pmodel.CIStr) (ver int64, err error) { +func dropForeignKey(jobCtx *jobContext, job *model.Job, tblInfo *model.TableInfo, fkName ast.CIStr) (ver int64, err error) { var fkInfo *model.FKInfo for _, fk := range tblInfo.ForeignKeys { if fk.Name.L == fkName.L { @@ -422,7 +421,7 @@ func checkDropTableHasForeignKeyReferredInOwner(infoCache *infoschema.InfoCache, } func checkTruncateTableHasForeignKeyReferredInOwner(infoCache *infoschema.InfoCache, job *model.Job, tblInfo *model.TableInfo, fkCheck bool) error { - referredFK, err := checkTableHasForeignKeyReferredInOwner(infoCache, job.SchemaName, job.TableName, []ast.Ident{{Name: tblInfo.Name, Schema: pmodel.NewCIStr(job.SchemaName)}}, fkCheck) + referredFK, err := checkTableHasForeignKeyReferredInOwner(infoCache, job.SchemaName, job.TableName, []ast.Ident{{Name: tblInfo.Name, Schema: ast.NewCIStr(job.SchemaName)}}, fkCheck) if err != nil { return err } @@ -455,7 +454,7 @@ func checkIndexNeededInForeignKey(is infoschema.InfoSchema, dbName string, tbInf } remainIdxs = append(remainIdxs, idx) } - checkFn := func(cols []pmodel.CIStr) error { + checkFn := func(cols []ast.CIStr) error { if !model.IsIndexPrefixCovered(tbInfo, idxInfo, cols...) { return nil } @@ -561,7 +560,7 @@ func (h *foreignKeyHelper) getLoadedTables() []schemaIDAndTableInfo { return tableList } -func (h *foreignKeyHelper) getTableFromStorage(is infoschema.InfoSchema, t *meta.Mutator, schema, table pmodel.CIStr) (result schemaIDAndTableInfo, _ error) { +func (h *foreignKeyHelper) getTableFromStorage(is infoschema.InfoSchema, t *meta.Mutator, schema, table ast.CIStr) (result schemaIDAndTableInfo, _ error) { k := schemaAndTable{schema: schema.L, table: table.L} if info, ok := h.loaded[k]; ok { return info, nil @@ -583,7 +582,7 @@ func (h *foreignKeyHelper) getTableFromStorage(is infoschema.InfoSchema, t *meta return result, nil } -func checkDatabaseHasForeignKeyReferred(ctx context.Context, is infoschema.InfoSchema, schema pmodel.CIStr, fkCheck bool) error { +func checkDatabaseHasForeignKeyReferred(ctx context.Context, is infoschema.InfoSchema, schema ast.CIStr, fkCheck bool) error { if !fkCheck { return nil } @@ -617,14 +616,14 @@ func checkDatabaseHasForeignKeyReferredInOwner(jobCtx *jobContext, job *model.Jo return nil } is := jobCtx.infoCache.GetLatest() - err = checkDatabaseHasForeignKeyReferred(jobCtx.stepCtx, is, pmodel.NewCIStr(job.SchemaName), fkCheck) + err = checkDatabaseHasForeignKeyReferred(jobCtx.stepCtx, is, ast.NewCIStr(job.SchemaName), fkCheck) if err != nil { job.State = model.JobStateCancelled } return errors.Trace(err) } -func checkFKDupName(tbInfo *model.TableInfo, fkName pmodel.CIStr) error { +func checkFKDupName(tbInfo *model.TableInfo, fkName ast.CIStr) error { for _, fkInfo := range tbInfo.ForeignKeys { if fkName.L == fkInfo.Name.L { return dbterror.ErrFkDupName.GenWithStackByArgs(fkName.O) diff --git a/pkg/ddl/foreign_key_test.go b/pkg/ddl/foreign_key_test.go index a4db227b3f903..6ffa6eb19f6a5 100644 --- a/pkg/ddl/foreign_key_test.go +++ b/pkg/ddl/foreign_key_test.go @@ -24,7 +24,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/ddl" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessiontxn" "github.com/pingcap/tidb/pkg/table" @@ -34,17 +34,17 @@ import ( "github.com/stretchr/testify/require" ) -func testCreateForeignKey(t *testing.T, d ddl.ExecutorForTest, ctx sessionctx.Context, dbInfo *model.DBInfo, tblInfo *model.TableInfo, fkName string, keys []string, refTable string, refKeys []string, onDelete pmodel.ReferOptionType, onUpdate pmodel.ReferOptionType) *model.Job { - FKName := pmodel.NewCIStr(fkName) - Keys := make([]pmodel.CIStr, len(keys)) +func testCreateForeignKey(t *testing.T, d ddl.ExecutorForTest, ctx sessionctx.Context, dbInfo *model.DBInfo, tblInfo *model.TableInfo, fkName string, keys []string, refTable string, refKeys []string, onDelete ast.ReferOptionType, onUpdate ast.ReferOptionType) *model.Job { + FKName := ast.NewCIStr(fkName) + Keys := make([]ast.CIStr, len(keys)) for i, key := range keys { - Keys[i] = pmodel.NewCIStr(key) + Keys[i] = ast.NewCIStr(key) } - RefTable := pmodel.NewCIStr(refTable) - RefKeys := make([]pmodel.CIStr, len(refKeys)) + RefTable := ast.NewCIStr(refTable) + RefKeys := make([]ast.CIStr, len(refKeys)) for i, key := range refKeys { - RefKeys[i] = pmodel.NewCIStr(key) + RefKeys[i] = ast.NewCIStr(key) } fkInfo := &model.FKInfo{ @@ -87,7 +87,7 @@ func testDropForeignKey(t *testing.T, ctx sessionctx.Context, d ddl.ExecutorForT BinlogInfo: &model.HistoryInfo{}, } ctx.SetValue(sessionctx.QueryString, "skip") - args := &model.DropForeignKeyArgs{FkName: pmodel.NewCIStr(foreignKeyName)} + args := &model.DropForeignKeyArgs{FkName: ast.NewCIStr(foreignKeyName)} err := d.DoDDLJobWrapper(ctx, ddl.NewJobWrapperWithArgs(job, args, true)) require.NoError(t, err) v := getSchemaVer(t, ctx) @@ -119,10 +119,10 @@ func TestForeignKey(t *testing.T) { require.NoError(t, err) tblInfo.Indices = append(tblInfo.Indices, &model.IndexInfo{ ID: 1, - Name: pmodel.NewCIStr("idx_fk"), - Table: pmodel.NewCIStr("t"), + Name: ast.NewCIStr("idx_fk"), + Table: ast.NewCIStr("t"), Columns: []*model.IndexColumn{{ - Name: pmodel.NewCIStr("c1"), + Name: ast.NewCIStr("c1"), Offset: 0, Length: types.UnspecifiedLength, }}, @@ -155,7 +155,7 @@ func TestForeignKey(t *testing.T) { }) ctx := testkit.NewTestKit(t, store).Session() - job := testCreateForeignKey(t, de, ctx, dbInfo, tblInfo, "c1_fk", []string{"c1"}, "t2", []string{"c1"}, pmodel.ReferOptionCascade, pmodel.ReferOptionSetNull) + job := testCreateForeignKey(t, de, ctx, dbInfo, tblInfo, "c1_fk", []string{"c1"}, "t2", []string{"c1"}, ast.ReferOptionCascade, ast.ReferOptionSetNull) testCheckJobDone(t, store, job.ID, true) require.NoError(t, err) mu.Lock() diff --git a/pkg/ddl/generated_column.go b/pkg/ddl/generated_column.go index 1f2e06484e1d3..83967e724ef35 100644 --- a/pkg/ddl/generated_column.go +++ b/pkg/ddl/generated_column.go @@ -23,7 +23,6 @@ import ( "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/ast" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/table" @@ -123,7 +122,7 @@ func findPositionRelativeColumn(cols []*table.Column, pos *ast.ColumnPosition) ( // findDependedColumnNames returns a set of string, which indicates // the names of the columns that are depended by colDef. -func findDependedColumnNames(schemaName pmodel.CIStr, tableName pmodel.CIStr, colDef *ast.ColumnDef) (generated bool, colsMap map[string]struct{}, err error) { +func findDependedColumnNames(schemaName ast.CIStr, tableName ast.CIStr, colDef *ast.ColumnDef) (generated bool, colsMap map[string]struct{}, err error) { colsMap = make(map[string]struct{}) for _, option := range colDef.Options { if option.Tp == ast.ColumnOptionGenerated { @@ -152,7 +151,7 @@ func FindColumnNamesInExpr(expr ast.ExprNode) []*ast.ColumnName { } // hasDependentByGeneratedColumn checks whether there are other columns depend on this column or not. -func hasDependentByGeneratedColumn(tblInfo *model.TableInfo, colName pmodel.CIStr) (bool, string, bool) { +func hasDependentByGeneratedColumn(tblInfo *model.TableInfo, colName ast.CIStr) (bool, string, bool) { for _, col := range tblInfo.Columns { for dep := range col.Dependences { if dep == colName.L { @@ -198,7 +197,7 @@ func (c *generatedColumnChecker) Leave(inNode ast.Node) (node ast.Node, ok bool) // 3. check if the modified expr contains non-deterministic functions // 4. check whether new column refers to any auto-increment columns. // 5. check if the new column is indexed or stored -func checkModifyGeneratedColumn(sctx sessionctx.Context, schemaName pmodel.CIStr, tbl table.Table, oldCol, newCol *table.Column, newColDef *ast.ColumnDef, pos *ast.ColumnPosition) error { +func checkModifyGeneratedColumn(sctx sessionctx.Context, schemaName ast.CIStr, tbl table.Table, oldCol, newCol *table.Column, newColDef *ast.ColumnDef, pos *ast.ColumnPosition) error { // rule 1. oldColIsStored := !oldCol.IsGenerated() || oldCol.GeneratedStored newColIsStored := !newCol.IsGenerated() || newCol.GeneratedStored diff --git a/pkg/ddl/index.go b/pkg/ddl/index.go index 075edbe2114ec..3c92e525a7df1 100644 --- a/pkg/ddl/index.go +++ b/pkg/ddl/index.go @@ -53,7 +53,6 @@ import ( "github.com/pingcap/tidb/pkg/metrics" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/charset" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/sessionctx" @@ -320,7 +319,7 @@ func calcBytesLengthForDecimal(m int) int { func BuildIndexInfo( ctx *metabuild.Context, tblInfo *model.TableInfo, - indexName pmodel.CIStr, + indexName ast.CIStr, isPrimary, isUnique, isVector bool, indexPartSpecifications []*ast.IndexPartSpecification, indexOption *ast.IndexOption, @@ -358,10 +357,10 @@ func BuildIndexInfo( if indexOption.Visibility == ast.IndexVisibilityInvisible { idxInfo.Invisible = true } - if indexOption.Tp == pmodel.IndexTypeInvalid { + if indexOption.Tp == ast.IndexTypeInvalid { // Use btree as default index type. - idxInfo.Tp = pmodel.IndexTypeBtree - } else if !isVector && indexOption.Tp == pmodel.IndexTypeHNSW { + idxInfo.Tp = ast.IndexTypeBtree + } else if !isVector && indexOption.Tp == ast.IndexTypeHNSW { return nil, dbterror.ErrUnsupportedAddVectorIndex.FastGenByArgs("Only support vector index with HNSW type, but it's non-vector index") } else { idxInfo.Tp = indexOption.Tp @@ -369,7 +368,7 @@ func BuildIndexInfo( idxInfo.Global = indexOption.Global } else { // Use btree as default index type. - idxInfo.Tp = pmodel.IndexTypeBtree + idxInfo.Tp = ast.IndexTypeBtree } return idxInfo, nil @@ -477,7 +476,7 @@ func DropIndexColumnFlag(tblInfo *model.TableInfo, indexInfo *model.IndexInfo) { } // ValidateRenameIndex checks if index name is ok to be renamed. -func ValidateRenameIndex(from, to pmodel.CIStr, tbl *model.TableInfo) (ignore bool, err error) { +func ValidateRenameIndex(from, to ast.CIStr, tbl *model.TableInfo) (ignore bool, err error) { if fromIdx := tbl.FindIndexByName(from.L); fromIdx == nil { return false, errors.Trace(infoschema.ErrKeyNotExists.GenWithStackByArgs(from.O, tbl.Name)) } @@ -520,7 +519,7 @@ func onRenameIndex(jobCtx *jobContext, job *model.Job) (ver int64, _ error) { return ver, nil } -func validateAlterIndexVisibility(ctx sessionctx.Context, indexName pmodel.CIStr, invisible bool, tbl *model.TableInfo) (bool, error) { +func validateAlterIndexVisibility(ctx sessionctx.Context, indexName ast.CIStr, invisible bool, tbl *model.TableInfo) (bool, error) { var idx *model.IndexInfo if idx = tbl.FindIndexByName(indexName.L); idx == nil || idx.State != model.StatePublic { return false, errors.Trace(infoschema.ErrKeyNotExists.GenWithStackByArgs(indexName.O, tbl.Name)) @@ -557,7 +556,7 @@ func onAlterIndexVisibility(jobCtx *jobContext, job *model.Job) (ver int64, _ er return ver, nil } -func setIndexVisibility(tblInfo *model.TableInfo, name pmodel.CIStr, invisible bool) { +func setIndexVisibility(tblInfo *model.TableInfo, name ast.CIStr, invisible bool) { for _, idx := range tblInfo.Indices { if idx.Name.L == name.L || (isTempIdxInfo(idx, tblInfo) && getChangingIndexOriginName(idx) == name.O) { idx.Invisible = invisible @@ -600,7 +599,7 @@ func checkPrimaryKeyNotNull(jobCtx *jobContext, w *worker, job *model.Job, dbInfo, tblInfo, nullCols, - &model.ColumnInfo{Name: pmodel.NewCIStr("")}, + &model.ColumnInfo{Name: ast.NewCIStr("")}, false, ) if err == nil { @@ -1771,8 +1770,8 @@ func checkInvisibleIndexesOnPK(tblInfo *model.TableInfo, indexInfos []*model.Ind return nil } -func checkRenameIndex(t *meta.Mutator, job *model.Job) (*model.TableInfo, pmodel.CIStr, pmodel.CIStr, error) { - var from, to pmodel.CIStr +func checkRenameIndex(t *meta.Mutator, job *model.Job) (*model.TableInfo, ast.CIStr, ast.CIStr, error) { + var from, to ast.CIStr schemaID := job.SchemaID tblInfo, err := GetTableInfoAndCancelFaultJob(t, job, schemaID) if err != nil { @@ -1798,9 +1797,9 @@ func checkRenameIndex(t *meta.Mutator, job *model.Job) (*model.TableInfo, pmodel return tblInfo, from, to, errors.Trace(err) } -func checkAlterIndexVisibility(t *meta.Mutator, job *model.Job) (*model.TableInfo, pmodel.CIStr, bool, error) { +func checkAlterIndexVisibility(t *meta.Mutator, job *model.Job) (*model.TableInfo, ast.CIStr, bool, error) { var ( - indexName pmodel.CIStr + indexName ast.CIStr invisible bool ) @@ -3079,7 +3078,7 @@ type changingIndex struct { // FindRelatedIndexesToChange finds the indexes that covering the given column. // The normal one will be overwritten by the temp one. -func FindRelatedIndexesToChange(tblInfo *model.TableInfo, colName pmodel.CIStr) []changingIndex { +func FindRelatedIndexesToChange(tblInfo *model.TableInfo, colName ast.CIStr) []changingIndex { // In multi-schema change jobs that contains several "modify column" sub-jobs, there may be temp indexes for another temp index. // To prevent reorganizing too many indexes, we should create the temp indexes that are really necessary. var normalIdxInfos, tempIdxInfos []changingIndex @@ -3119,7 +3118,7 @@ func isTempIdxInfo(idxInfo *model.IndexInfo, tblInfo *model.TableInfo) bool { return false } -func findIdxCol(idxInfo *model.IndexInfo, colName pmodel.CIStr) int { +func findIdxCol(idxInfo *model.IndexInfo, colName ast.CIStr) int { for offset, idxCol := range idxInfo.Columns { if idxCol.Name.L == colName.L { return offset @@ -3128,7 +3127,7 @@ func findIdxCol(idxInfo *model.IndexInfo, colName pmodel.CIStr) int { return -1 } -func renameIndexes(tblInfo *model.TableInfo, from, to pmodel.CIStr) { +func renameIndexes(tblInfo *model.TableInfo, from, to ast.CIStr) { for _, idx := range tblInfo.Indices { if idx.Name.L == from.L { idx.Name = to @@ -3146,7 +3145,7 @@ func renameIndexes(tblInfo *model.TableInfo, from, to pmodel.CIStr) { } } -func renameHiddenColumns(tblInfo *model.TableInfo, from, to pmodel.CIStr) { +func renameHiddenColumns(tblInfo *model.TableInfo, from, to ast.CIStr) { for _, col := range tblInfo.Columns { if col.Hidden && getExpressionIndexOriginName(col.Name) == from.O { col.Name.L = strings.Replace(col.Name.L, from.L, to.L, 1) diff --git a/pkg/ddl/index_cop_test.go b/pkg/ddl/index_cop_test.go index 87239a6121404..368824ff478e8 100644 --- a/pkg/ddl/index_cop_test.go +++ b/pkg/ddl/index_cop_test.go @@ -24,7 +24,7 @@ import ( "github.com/pingcap/tidb/pkg/ddl/copr" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/types" @@ -38,7 +38,7 @@ func TestAddIndexFetchRowsFromCoprocessor(t *testing.T) { tk.MustExec("use test") testFetchRows := func(db, tb, idx string) ([]kv.Handle, [][]types.Datum) { - tbl, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr(db), pmodel.NewCIStr(tb)) + tbl, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr(db), ast.NewCIStr(tb)) require.NoError(t, err) tblInfo := tbl.Meta() idxInfo := tblInfo.FindIndexByName(idx) diff --git a/pkg/ddl/index_modify_test.go b/pkg/ddl/index_modify_test.go index bf497f2fc86d7..999e4fe2b6994 100644 --- a/pkg/ddl/index_modify_test.go +++ b/pkg/ddl/index_modify_test.go @@ -34,7 +34,7 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/terror" sessiontypes "github.com/pingcap/tidb/pkg/session/types" "github.com/pingcap/tidb/pkg/sessionctx" @@ -1096,12 +1096,12 @@ func TestCreateTableWithVectorIndex(t *testing.T) { checkCreateTableWithVectorIdx := func(replicaCnt uint64) { tk.MustExec("create table t(a int, b vector(3), vector index((VEC_COSINE_DISTANCE(b))) USING HNSW, vector index((VEC_L2_DISTANCE(b))));") - tbl, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) require.Equal(t, replicaCnt, tbl.Meta().TiFlashReplica.Count) indexes := tbl.Meta().Indices require.Equal(t, 2, len(indexes)) - require.Equal(t, pmodel.IndexTypeHNSW, indexes[0].Tp) + require.Equal(t, ast.IndexTypeHNSW, indexes[0].Tp) require.Equal(t, model.DistanceMetricCosine, indexes[0].VectorInfo.DistanceMetric) require.Equal(t, "vector_index", tbl.Meta().Indices[0].Name.O) require.Equal(t, "vector_index_2", tbl.Meta().Indices[1].Name.O) @@ -1220,16 +1220,16 @@ func TestAddVectorIndexSimple(t *testing.T) { tk.MustExec("insert into t values (1, '[1,2.1,3.3]');") tk.MustQuery("SELECT * FROM INFORMATION_SCHEMA.KEY_COLUMN_USAGE WHERE table_name = 't'").Check(testkit.Rows()) - tbl, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) indexes := tbl.Meta().Indices require.Equal(t, 0, len(indexes)) tk.MustExec("alter table t add vector index idx((VEC_COSINE_DISTANCE(b))) USING HNSW COMMENT 'b comment';") - tbl, err = dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err = dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) indexes = tbl.Meta().Indices require.Equal(t, 1, len(indexes)) - require.Equal(t, pmodel.IndexTypeHNSW, indexes[0].Tp) + require.Equal(t, ast.IndexTypeHNSW, indexes[0].Tp) require.Equal(t, model.DistanceMetricCosine, indexes[0].VectorInfo.DistanceMetric) // test row count jobs, err := getJobsBySQL(tk.Session(), "tidb_ddl_history", "order by job_id desc limit 1") @@ -1266,7 +1266,7 @@ func TestAddVectorIndexSimple(t *testing.T) { // test rename index tk.MustExec("alter table t rename index idx to vecIdx") - tbl, err = dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err = dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) indexes1 := tbl.Meta().Indices require.Equal(t, 1, len(indexes1)) @@ -1275,7 +1275,7 @@ func TestAddVectorIndexSimple(t *testing.T) { // test drop a vector index tk.MustExec("alter table t drop index vecIdx;") - tbl, err = dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err = dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) indexes = tbl.Meta().Indices require.Equal(t, 0, len(indexes)) @@ -1289,11 +1289,11 @@ func TestAddVectorIndexSimple(t *testing.T) { // test create a vector index with same name tk.MustExec("create vector index idx on t ((VEC_COSINE_DISTANCE(b))) USING HNSW COMMENT 'b comment';") - tbl, err = dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err = dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) indexes = tbl.Meta().Indices require.Equal(t, 1, len(indexes)) - require.Equal(t, pmodel.IndexTypeHNSW, indexes[0].Tp) + require.Equal(t, ast.IndexTypeHNSW, indexes[0].Tp) require.Equal(t, model.DistanceMetricCosine, indexes[0].VectorInfo.DistanceMetric) tk.MustQuery("select * from t;").Check(testkit.Rows("1 [1,2.1,3.3]")) tk.MustQuery("show create table t").Check(testkit.Rows("t CREATE TABLE `t` (\n" + @@ -1314,16 +1314,16 @@ func TestAddVectorIndexSimple(t *testing.T) { // test anonymous index tk.MustExec("alter table t add vector index ((vec_l2_distance(b))) USING HNSW;") - tbl, err = dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err = dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) require.Equal(t, 1, len(tbl.Meta().Indices)) idx := tbl.Meta().Indices[0] require.Equal(t, "vector_index", idx.Name.O) - require.Equal(t, pmodel.IndexTypeHNSW, idx.Tp) + require.Equal(t, ast.IndexTypeHNSW, idx.Tp) require.Equal(t, model.DistanceMetricL2, idx.VectorInfo.DistanceMetric) tk.MustExec("alter table t add key vector_index_2(a);") tk.MustExec("alter table t add vector index ((VEC_COSINE_DISTANCE(b))) USING HNSW;") - tbl, err = dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err = dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) require.Equal(t, 3, len(tbl.Meta().Indices)) require.Equal(t, "vector_index_2", tbl.Meta().Indices[1].Name.O) diff --git a/pkg/ddl/modify_column.go b/pkg/ddl/modify_column.go index 09a5baf485fd0..ae7ba0641d2d6 100644 --- a/pkg/ddl/modify_column.go +++ b/pkg/ddl/modify_column.go @@ -39,7 +39,6 @@ import ( "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/charset" "github.com/pingcap/tidb/pkg/parser/format" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/variable" @@ -124,7 +123,7 @@ func (w *worker) onModifyColumn(jobCtx *jobContext, job *model.Job) (ver int64, changingCol := args.ChangingColumn if changingCol == nil { - newColName := pmodel.NewCIStr(genChangingColumnUniqueName(tblInfo, oldCol)) + newColName := ast.NewCIStr(genChangingColumnUniqueName(tblInfo, oldCol)) if mysql.HasPriKeyFlag(oldCol.GetFlag()) { job.State = model.JobStateCancelled msg := "this column has primary key flag" @@ -153,7 +152,7 @@ func (w *worker) onModifyColumn(jobCtx *jobContext, job *model.Job) (ver int64, // We create a temp index for each normal index. tmpIdx := info.IndexInfo.Clone() tmpIdxName := genChangingIndexUniqueName(tblInfo, info.IndexInfo) - setIdxIDName(tmpIdx, newIdxID, pmodel.NewCIStr(tmpIdxName)) + setIdxIDName(tmpIdx, newIdxID, ast.NewCIStr(tmpIdxName)) SetIdxColNameOffset(tmpIdx.Columns[info.Offset], changingCol) tblInfo.Indices = append(tblInfo.Indices, tmpIdx) } else { @@ -200,7 +199,7 @@ func rollbackModifyColumnJob(jobCtx *jobContext, tblInfo *model.TableInfo, job * } func getModifyColumnInfo( - t *meta.Mutator, job *model.Job, oldColName pmodel.CIStr, + t *meta.Mutator, job *model.Job, oldColName ast.CIStr, ) (*model.DBInfo, *model.TableInfo, *model.ColumnInfo, error) { dbInfo, err := checkSchemaExistAndCancelNotExistJob(t, job) if err != nil { @@ -376,7 +375,7 @@ func adjustTableInfoAfterModifyColumn( return nil } -func updateFKInfoWhenModifyColumn(tblInfo *model.TableInfo, oldCol, newCol pmodel.CIStr) { +func updateFKInfoWhenModifyColumn(tblInfo *model.TableInfo, oldCol, newCol ast.CIStr) { if oldCol.L == newCol.L { return } @@ -389,7 +388,7 @@ func updateFKInfoWhenModifyColumn(tblInfo *model.TableInfo, oldCol, newCol pmode } } -func updateTTLInfoWhenModifyColumn(tblInfo *model.TableInfo, oldCol, newCol pmodel.CIStr) { +func updateTTLInfoWhenModifyColumn(tblInfo *model.TableInfo, oldCol, newCol ast.CIStr) { if oldCol.L == newCol.L { return } @@ -669,7 +668,7 @@ func doReorgWorkForModifyColumn(w *worker, jobCtx *jobContext, job *model.Job, t } func adjustTableInfoAfterModifyColumnWithData(tblInfo *model.TableInfo, pos *ast.ColumnPosition, - oldCol, changingCol *model.ColumnInfo, newName pmodel.CIStr, changingIdxs []*model.IndexInfo) (err error) { + oldCol, changingCol *model.ColumnInfo, newName ast.CIStr, changingIdxs []*model.IndexInfo) (err error) { if pos != nil && pos.RelativeColumn != nil && oldCol.Name.L == pos.RelativeColumn.Name.L { // For cases like `modify column b after b`, it should report this error. return errors.Trace(infoschema.ErrColumnNotExists.GenWithStackByArgs(oldCol.Name, tblInfo.Name)) @@ -693,7 +692,7 @@ func adjustTableInfoAfterModifyColumnWithData(tblInfo *model.TableInfo, pos *ast return nil } -func checkModifyColumnWithGeneratedColumnsConstraint(allCols []*table.Column, oldColName pmodel.CIStr) error { +func checkModifyColumnWithGeneratedColumnsConstraint(allCols []*table.Column, oldColName ast.CIStr) error { for _, col := range allCols { if col.GeneratedExpr == nil { continue @@ -717,7 +716,7 @@ func GetModifiableColumnJob( sctx sessionctx.Context, is infoschema.InfoSchema, // WARN: is maybe nil here. ident ast.Ident, - originalColName pmodel.CIStr, + originalColName ast.CIStr, schema *model.DBInfo, t table.Table, spec *ast.AlterTableSpec, diff --git a/pkg/ddl/multi_schema_change.go b/pkg/ddl/multi_schema_change.go index cbbb60ad347c2..974459e373366 100644 --- a/pkg/ddl/multi_schema_change.go +++ b/pkg/ddl/multi_schema_change.go @@ -18,7 +18,6 @@ import ( "github.com/pingcap/tidb/pkg/meta" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/ast" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/table" @@ -205,7 +204,7 @@ func fillMultiSchemaInfo(info *model.MultiSchemaInfo, job *JobWrapper) error { col, pos := args.Col, args.Pos info.AddColumns = append(info.AddColumns, col.Name) for colName := range col.Dependences { - info.RelativeColumns = append(info.RelativeColumns, pmodel.CIStr{L: colName, O: colName}) + info.RelativeColumns = append(info.RelativeColumns, ast.CIStr{L: colName, O: colName}) } if pos != nil && pos.Tp == ast.ColumnPositionAfter { info.PositionColumns = append(info.PositionColumns, pos.RelativeColumn.Name) @@ -227,7 +226,7 @@ func fillMultiSchemaInfo(info *model.MultiSchemaInfo, job *JobWrapper) error { } for _, c := range indexArg.HiddenCols { for depColName := range c.Dependences { - info.RelativeColumns = append(info.RelativeColumns, pmodel.NewCIStr(depColName)) + info.RelativeColumns = append(info.RelativeColumns, ast.NewCIStr(depColName)) } } case model.ActionRenameIndex: @@ -273,7 +272,7 @@ func checkOperateSameColAndIdx(info *model.MultiSchemaInfo) error { modifyCols := make(map[string]struct{}) modifyIdx := make(map[string]struct{}) - checkColumns := func(colNames []pmodel.CIStr, addToModifyCols bool) error { + checkColumns := func(colNames []ast.CIStr, addToModifyCols bool) error { for _, colName := range colNames { name := colName.L if _, ok := modifyCols[name]; ok { @@ -286,7 +285,7 @@ func checkOperateSameColAndIdx(info *model.MultiSchemaInfo) error { return nil } - checkIndexes := func(idxNames []pmodel.CIStr, addToModifyIdx bool) error { + checkIndexes := func(idxNames []ast.CIStr, addToModifyIdx bool) error { for _, idxName := range idxNames { name := idxName.L if _, ok := modifyIdx[name]; ok { diff --git a/pkg/ddl/notifier/BUILD.bazel b/pkg/ddl/notifier/BUILD.bazel index 14e7b55afef95..f83e3cfe5e249 100644 --- a/pkg/ddl/notifier/BUILD.bazel +++ b/pkg/ddl/notifier/BUILD.bazel @@ -15,7 +15,7 @@ go_library( "//pkg/kv", "//pkg/meta/model", "//pkg/owner", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/sessionctx", "//pkg/util", "//pkg/util/chunk", @@ -41,7 +41,7 @@ go_test( "//pkg/ddl", "//pkg/ddl/session", "//pkg/meta/model", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/sessionctx", "//pkg/testkit", "//pkg/testkit/testfailpoint", diff --git a/pkg/ddl/notifier/events.go b/pkg/ddl/notifier/events.go index f3e6845978b3d..0f74f455f41ee 100644 --- a/pkg/ddl/notifier/events.go +++ b/pkg/ddl/notifier/events.go @@ -20,7 +20,7 @@ import ( "strings" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/util/intest" ) @@ -467,7 +467,7 @@ func (s *SchemaChangeEvent) GetDropSchemaInfo() (miniDBInfo *MiniDBInfoForSchema // MiniDBInfoForSchemaEvent is a mini version of DBInfo for DropSchemaEvent only. type MiniDBInfoForSchemaEvent struct { ID int64 `json:"id"` - Name pmodel.CIStr `json:"name"` + Name ast.CIStr `json:"name"` Tables []*MiniTableInfoForSchemaEvent `json:"tables,omitempty"` } @@ -477,7 +477,7 @@ type MiniDBInfoForSchemaEvent struct { // So please do not use this mini version in other places. type MiniTableInfoForSchemaEvent struct { ID int64 `json:"id"` - Name pmodel.CIStr `json:"name"` + Name ast.CIStr `json:"name"` Partitions []*MiniPartitionInfoForSchemaEvent `json:"partitions,omitempty"` } @@ -486,8 +486,8 @@ type MiniTableInfoForSchemaEvent struct { // DropSchemaEvent, it's more efficient to use this mini version. // So please do not use this mini version in other places. type MiniPartitionInfoForSchemaEvent struct { - ID int64 `json:"id"` - Name pmodel.CIStr `json:"name"` + ID int64 `json:"id"` + Name ast.CIStr `json:"name"` } // jsonSchemaChangeEvent is used by SchemaChangeEvent when needed to (un)marshal data, diff --git a/pkg/ddl/notifier/events_test.go b/pkg/ddl/notifier/events_test.go index 599c95e63bf73..fd412723a3aee 100644 --- a/pkg/ddl/notifier/events_test.go +++ b/pkg/ddl/notifier/events_test.go @@ -18,7 +18,7 @@ import ( "testing" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/stretchr/testify/require" ) @@ -29,7 +29,7 @@ func TestEventString(t *testing.T) { Tp: model.ActionAddColumn, TableInfo: &model.TableInfo{ ID: 1, - Name: pmodel.NewCIStr("Table1"), + Name: ast.NewCIStr("Table1"), }, AddedPartInfo: &model.PartitionInfo{ Definitions: []model.PartitionDefinition{ @@ -39,7 +39,7 @@ func TestEventString(t *testing.T) { }, OldTableInfo: &model.TableInfo{ ID: 4, - Name: pmodel.NewCIStr("Table2"), + Name: ast.NewCIStr("Table2"), }, DroppedPartInfo: &model.PartitionInfo{ Definitions: []model.PartitionDefinition{ @@ -48,12 +48,12 @@ func TestEventString(t *testing.T) { }, }, Columns: []*model.ColumnInfo{ - {ID: 7, Name: pmodel.NewCIStr("Column1")}, - {ID: 8, Name: pmodel.NewCIStr("Column2")}, + {ID: 7, Name: ast.NewCIStr("Column1")}, + {ID: 8, Name: ast.NewCIStr("Column2")}, }, Indexes: []*model.IndexInfo{ - {ID: 9, Name: pmodel.NewCIStr("Index1")}, - {ID: 10, Name: pmodel.NewCIStr("Index2")}, + {ID: 9, Name: ast.NewCIStr("Index1")}, + {ID: 10, Name: ast.NewCIStr("Index2")}, }, }, } diff --git a/pkg/ddl/notifier/store_test.go b/pkg/ddl/notifier/store_test.go index 69e13945cd345..3aee8879ebc70 100644 --- a/pkg/ddl/notifier/store_test.go +++ b/pkg/ddl/notifier/store_test.go @@ -19,7 +19,7 @@ import ( "testing" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/chunk" "github.com/stretchr/testify/require" @@ -30,12 +30,12 @@ func TestLeftoverWhenUnmarshal(t *testing.T) { changesReused := []*SchemaChange{ {event: &SchemaChangeEvent{inner: &jsonSchemaChangeEvent{ TableInfo: &model.TableInfo{ - Name: pmodel.NewCIStr("old"), - Columns: []*model.ColumnInfo{{Name: pmodel.NewCIStr("c1")}}, + Name: ast.NewCIStr("old"), + Columns: []*model.ColumnInfo{{Name: ast.NewCIStr("c1")}}, Indices: []*model.IndexInfo{ - {Name: pmodel.NewCIStr("i1")}, - {Name: pmodel.NewCIStr("i2")}, - {Name: pmodel.NewCIStr("i3")}, + {Name: ast.NewCIStr("i1")}, + {Name: ast.NewCIStr("i2")}, + {Name: ast.NewCIStr("i3")}, }, }, }}}, @@ -46,16 +46,16 @@ func TestLeftoverWhenUnmarshal(t *testing.T) { } newTableInfo := &model.TableInfo{ - Name: pmodel.NewCIStr("new"), + Name: ast.NewCIStr("new"), Columns: []*model.ColumnInfo{ - {Name: pmodel.NewCIStr("c2")}, - {Name: pmodel.NewCIStr("c3")}, + {Name: ast.NewCIStr("c2")}, + {Name: ast.NewCIStr("c3")}, }, Indices: []*model.IndexInfo{ - {Name: pmodel.NewCIStr("i4")}, + {Name: ast.NewCIStr("i4")}, }, Constraints: []*model.ConstraintInfo{ - {Name: pmodel.NewCIStr("c1")}, + {Name: ast.NewCIStr("c1")}, }, } diff --git a/pkg/ddl/notifier/testkit_test.go b/pkg/ddl/notifier/testkit_test.go index c3993d771365c..c1ab504008f69 100644 --- a/pkg/ddl/notifier/testkit_test.go +++ b/pkg/ddl/notifier/testkit_test.go @@ -31,7 +31,7 @@ import ( "github.com/pingcap/tidb/pkg/ddl/notifier" sess "github.com/pingcap/tidb/pkg/ddl/session" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/testkit/testfailpoint" @@ -50,10 +50,10 @@ func TestPublishToTableStore(t *testing.T) { ctx := context.Background() s := notifier.OpenTableStore("mysql", ddl.NotifierTableName) se := sess.NewSession(tk.Session()) - event1 := notifier.NewCreateTableEvent(&model.TableInfo{ID: 1000, Name: pmodel.NewCIStr("t1")}) + event1 := notifier.NewCreateTableEvent(&model.TableInfo{ID: 1000, Name: ast.NewCIStr("t1")}) err := notifier.PubSchemeChangeToStore(ctx, se, 1, -1, event1, s) require.NoError(t, err) - event2 := notifier.NewDropTableEvent(&model.TableInfo{ID: 1001, Name: pmodel.NewCIStr("t2")}) + event2 := notifier.NewDropTableEvent(&model.TableInfo{ID: 1001, Name: ast.NewCIStr("t2")}) err = notifier.PubSchemeChangeToStore(ctx, se, 2, -1, event2, s) require.NoError(t, err) changes := make([]*notifier.SchemaChange, 8) @@ -118,13 +118,13 @@ func TestBasicPubSub(t *testing.T) { tk2 := testkit.NewTestKit(t, store) se := sess.NewSession(tk2.Session()) ctx := context.Background() - event1 := notifier.NewCreateTableEvent(&model.TableInfo{ID: 1000, Name: pmodel.NewCIStr("t1")}) + event1 := notifier.NewCreateTableEvent(&model.TableInfo{ID: 1000, Name: ast.NewCIStr("t1")}) err := notifier.PubSchemeChangeToStore(ctx, se, 1, -1, event1, s) require.NoError(t, err) - event2 := notifier.NewDropTableEvent(&model.TableInfo{ID: 1001, Name: pmodel.NewCIStr("t2#special-char?in'name")}) + event2 := notifier.NewDropTableEvent(&model.TableInfo{ID: 1001, Name: ast.NewCIStr("t2#special-char?in'name")}) err = notifier.PubSchemeChangeToStore(ctx, se, 2, -1, event2, s) require.NoError(t, err) - event3 := notifier.NewDropTableEvent(&model.TableInfo{ID: 1002, Name: pmodel.NewCIStr("t3")}) + event3 := notifier.NewDropTableEvent(&model.TableInfo{ID: 1002, Name: ast.NewCIStr("t3")}) err = notifier.PubSchemeChangeToStore(ctx, se, 3, -1, event3, s) require.NoError(t, err) @@ -196,13 +196,13 @@ func TestDeliverOrderAndCleanup(t *testing.T) { tk2 := testkit.NewTestKit(t, store) se := sess.NewSession(tk2.Session()) ctx := context.Background() - event1 := notifier.NewCreateTableEvent(&model.TableInfo{ID: 1000, Name: pmodel.NewCIStr("t1")}) + event1 := notifier.NewCreateTableEvent(&model.TableInfo{ID: 1000, Name: ast.NewCIStr("t1")}) err := notifier.PubSchemeChangeToStore(ctx, se, 1, -1, event1, s) require.NoError(t, err) - event2 := notifier.NewCreateTableEvent(&model.TableInfo{ID: 1001, Name: pmodel.NewCIStr("t2")}) + event2 := notifier.NewCreateTableEvent(&model.TableInfo{ID: 1001, Name: ast.NewCIStr("t2")}) err = notifier.PubSchemeChangeToStore(ctx, se, 2, -1, event2, s) require.NoError(t, err) - event3 := notifier.NewCreateTableEvent(&model.TableInfo{ID: 1002, Name: pmodel.NewCIStr("t3")}) + event3 := notifier.NewCreateTableEvent(&model.TableInfo{ID: 1002, Name: ast.NewCIStr("t3")}) err = notifier.PubSchemeChangeToStore(ctx, se, 3, -1, event3, s) require.NoError(t, err) @@ -362,7 +362,7 @@ func Test2OwnerForAShortTime(t *testing.T) { tk2 := testkit.NewTestKit(t, store) se := sess.NewSession(tk2.Session()) ctx := context.Background() - event1 := notifier.NewCreateTableEvent(&model.TableInfo{ID: 1000, Name: pmodel.NewCIStr("t1")}) + event1 := notifier.NewCreateTableEvent(&model.TableInfo{ID: 1000, Name: ast.NewCIStr("t1")}) err := notifier.PubSchemeChangeToStore(ctx, se, 1, -1, event1, s) require.NoError(t, err) diff --git a/pkg/ddl/partition.go b/pkg/ddl/partition.go index 3ecc57f89daf5..0600caee75c7b 100644 --- a/pkg/ddl/partition.go +++ b/pkg/ddl/partition.go @@ -44,7 +44,6 @@ import ( "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/charset" "github.com/pingcap/tidb/pkg/parser/format" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/opcode" field_types "github.com/pingcap/tidb/pkg/parser/types" @@ -370,7 +369,7 @@ func removePartitionAddingDefinitionsFromTableInfo(tblInfo *model.TableInfo) ([] // (needs reorganize partition instead). func checkAddPartitionValue(meta *model.TableInfo, part *model.PartitionInfo) error { switch meta.Partition.Type { - case pmodel.PartitionTypeRange: + case ast.PartitionTypeRange: if len(meta.Partition.Columns) == 0 { newDefs, oldDefs := part.Definitions, meta.Partition.Definitions rangeValue := oldDefs[len(oldDefs)-1].LessThan[0] @@ -401,7 +400,7 @@ func checkAddPartitionValue(meta *model.TableInfo, part *model.PartitionInfo) er currentRangeValue = nextRangeValue } } - case pmodel.PartitionTypeList: + case ast.PartitionTypeList: if meta.Partition.GetDefaultListPartition() != -1 { return dbterror.ErrGeneralUnsupportedDDL.GenWithStackByArgs("ADD List partition, already contains DEFAULT partition. Please use REORGANIZE PARTITION instead") } @@ -509,15 +508,15 @@ func buildTablePartitionInfo(ctx *metabuild.Context, s *ast.PartitionOptions, tb var enable bool switch s.Tp { - case pmodel.PartitionTypeRange: + case ast.PartitionTypeRange: enable = true - case pmodel.PartitionTypeList: + case ast.PartitionTypeList: enable = true err := checkListPartitions(s.Definitions) if err != nil { return err } - case pmodel.PartitionTypeHash, pmodel.PartitionTypeKey: + case ast.PartitionTypeHash, ast.PartitionTypeKey: // Partition by hash and key is enabled by default. if s.Sub != nil { // Subpartitioning only allowed with Range or List @@ -527,10 +526,10 @@ func buildTablePartitionInfo(ctx *metabuild.Context, s *ast.PartitionOptions, tb if s.Linear { ctx.AppendWarning(dbterror.ErrUnsupportedCreatePartition.FastGen(fmt.Sprintf("LINEAR %s is not supported, using non-linear %s instead", s.Tp.String(), s.Tp.String()))) } - if s.Tp == pmodel.PartitionTypeHash || len(s.ColumnNames) != 0 { + if s.Tp == ast.PartitionTypeHash || len(s.ColumnNames) != 0 { enable = true } - if s.Tp == pmodel.PartitionTypeKey && len(s.ColumnNames) == 0 { + if s.Tp == ast.PartitionTypeKey && len(s.ColumnNames) == 0 { enable = true } } @@ -562,11 +561,11 @@ func buildTablePartitionInfo(ctx *metabuild.Context, s *ast.PartitionOptions, tb } pi.Expr = buf.String() } else if s.ColumnNames != nil { - pi.Columns = make([]pmodel.CIStr, 0, len(s.ColumnNames)) + pi.Columns = make([]ast.CIStr, 0, len(s.ColumnNames)) for _, cn := range s.ColumnNames { pi.Columns = append(pi.Columns, cn.Name) } - if pi.Type == pmodel.PartitionTypeKey && len(s.ColumnNames) == 0 { + if pi.Type == ast.PartitionTypeKey && len(s.ColumnNames) == 0 { if tbInfo.PKIsHandle { pi.Columns = append(pi.Columns, tbInfo.GetPkName()) pi.IsEmptyColumns = true @@ -737,10 +736,10 @@ func isValidKeyPartitionColType(fieldType types.FieldType) bool { } } -func isColTypeAllowedAsPartitioningCol(partType pmodel.PartitionType, fieldType types.FieldType) bool { +func isColTypeAllowedAsPartitioningCol(partType ast.PartitionType, fieldType types.FieldType) bool { // For key partition, the permitted partition field types can be all field types except // BLOB, JSON, Geometry - if partType == pmodel.PartitionTypeKey { + if partType == ast.PartitionTypeKey { return isValidKeyPartitionColType(fieldType) } // The permitted data types are shown in the following list: @@ -763,7 +762,7 @@ func isColTypeAllowedAsPartitioningCol(partType pmodel.PartitionType, fieldType // will return nil if error occurs, i.e. not an INTERVAL partitioned table func getPartitionIntervalFromTable(ctx expression.BuildContext, tbInfo *model.TableInfo) *ast.PartitionInterval { if tbInfo.Partition == nil || - tbInfo.Partition.Type != pmodel.PartitionTypeRange { + tbInfo.Partition.Type != ast.PartitionTypeRange { return nil } if len(tbInfo.Partition.Columns) > 1 { @@ -883,7 +882,7 @@ func getPartitionIntervalFromTable(ctx expression.BuildContext, tbInfo *model.Ta } partitionMethod := ast.PartitionMethod{ - Tp: pmodel.PartitionTypeRange, + Tp: ast.PartitionTypeRange, Interval: &interval, } partOption := &ast.PartitionOptions{PartitionMethod: partitionMethod} @@ -1017,7 +1016,7 @@ func generatePartitionDefinitionsFromInterval(ctx expression.BuildContext, partO if partOptions.Interval == nil { return nil } - if tbInfo.Partition.Type != pmodel.PartitionTypeRange { + if tbInfo.Partition.Type != ast.PartitionTypeRange { return dbterror.ErrGeneralUnsupportedDDL.GenWithStackByArgs("INTERVAL partitioning, only allowed on RANGE partitioning") } if len(partOptions.ColumnNames) > 1 || len(tbInfo.Partition.Columns) > 1 { @@ -1092,7 +1091,7 @@ func generatePartitionDefinitionsFromInterval(ctx expression.BuildContext, partO partExpr = ast.NewValueExpr(minv, "", "") } partOptions.Definitions = append(partOptions.Definitions, &ast.PartitionDefinition{ - Name: pmodel.NewCIStr("P_NULL"), + Name: ast.NewCIStr("P_NULL"), Clause: &ast.PartitionDefinitionClauseLessThan{ Exprs: []ast.ExprNode{partExpr}, }, @@ -1106,7 +1105,7 @@ func generatePartitionDefinitionsFromInterval(ctx expression.BuildContext, partO if partOptions.Interval.MaxValPart { partOptions.Definitions = append(partOptions.Definitions, &ast.PartitionDefinition{ - Name: pmodel.NewCIStr("P_MAXVALUE"), + Name: ast.NewCIStr("P_MAXVALUE"), Clause: &ast.PartitionDefinitionClauseLessThan{ Exprs: []ast.ExprNode{&ast.MaxValueExpr{}}, }, @@ -1292,7 +1291,7 @@ func GeneratePartDefsFromInterval(ctx expression.BuildContext, tp ast.AlterTable } } else { currExpr = &ast.FuncCallExpr{ - FnName: pmodel.NewCIStr("DATE_ADD"), + FnName: ast.NewCIStr("DATE_ADD"), Args: []ast.ExprNode{ startExpr, currExpr, @@ -1351,7 +1350,7 @@ func GeneratePartDefsFromInterval(ctx expression.BuildContext, tp ast.AlterTable } } partDefs = append(partDefs, &ast.PartitionDefinition{ - Name: pmodel.NewCIStr(partName), + Name: ast.NewCIStr(partName), Clause: &ast.PartitionDefinitionClauseLessThan{ Exprs: []ast.ExprNode{currExpr}, }, @@ -1375,7 +1374,7 @@ func GeneratePartDefsFromInterval(ctx expression.BuildContext, tp ast.AlterTable // buildPartitionDefinitionsInfo build partition definitions info without assign partition id. tbInfo will be constant func buildPartitionDefinitionsInfo(ctx expression.BuildContext, defs []*ast.PartitionDefinition, tbInfo *model.TableInfo, numParts uint64) (partitions []model.PartitionDefinition, err error) { switch tbInfo.Partition.Type { - case pmodel.PartitionTypeNone: + case ast.PartitionTypeNone: if len(defs) != 1 { return nil, dbterror.ErrUnsupportedPartitionType } @@ -1383,11 +1382,11 @@ func buildPartitionDefinitionsInfo(ctx expression.BuildContext, defs []*ast.Part if comment, set := defs[0].Comment(); set { partitions[0].Comment = comment } - case pmodel.PartitionTypeRange: + case ast.PartitionTypeRange: partitions, err = buildRangePartitionDefinitions(ctx, defs, tbInfo) - case pmodel.PartitionTypeHash, pmodel.PartitionTypeKey: + case ast.PartitionTypeHash, ast.PartitionTypeKey: partitions, err = buildHashPartitionDefinitions(defs, tbInfo, numParts) - case pmodel.PartitionTypeList: + case ast.PartitionTypeList: partitions, err = buildListPartitionDefinitions(ctx, defs, tbInfo) default: err = dbterror.ErrUnsupportedPartitionType @@ -1410,7 +1409,7 @@ func setPartitionPlacementFromOptions(partition *model.PartitionDefinition, opti for _, opt := range options { if opt.Tp == ast.TableOptionPlacementPolicy { partition.PlacementPolicyRef = &model.PolicyRefInfo{ - Name: pmodel.NewCIStr(opt.StrValue), + Name: ast.NewCIStr(opt.StrValue), } } } @@ -1458,7 +1457,7 @@ func buildHashPartitionDefinitions(defs []*ast.PartitionDefinition, tbInfo *mode } } else { // Use the default - definitions[i].Name = pmodel.NewCIStr(fmt.Sprintf("p%d", i)) + definitions[i].Name = ast.NewCIStr(fmt.Sprintf("p%d", i)) } } return definitions, nil @@ -1472,7 +1471,7 @@ func buildListPartitionDefinitions(ctx expression.BuildContext, defs []*ast.Part return nil, dbterror.ErrWrongPartitionName.GenWithStack("partition column name cannot be found") } for _, def := range defs { - if err := def.Clause.Validate(pmodel.PartitionTypeList, len(tbInfo.Partition.Columns)); err != nil { + if err := def.Clause.Validate(ast.PartitionTypeList, len(tbInfo.Partition.Columns)); err != nil { return nil, err } clause := def.Clause.(*ast.PartitionDefinitionClauseIn) @@ -1567,7 +1566,7 @@ func buildRangePartitionDefinitions(ctx expression.BuildContext, defs []*ast.Par return nil, dbterror.ErrWrongPartitionName.GenWithStack("partition column name cannot be found") } for _, def := range defs { - if err := def.Clause.Validate(pmodel.PartitionTypeRange, len(tbInfo.Partition.Columns)); err != nil { + if err := def.Clause.Validate(ast.PartitionTypeRange, len(tbInfo.Partition.Columns)); err != nil { return nil, err } clause := def.Clause.(*ast.PartitionDefinitionClauseLessThan) @@ -1742,7 +1741,7 @@ func checkAndOverridePartitionID(newTableInfo, oldTableInfo *model.TableInfo) er return dbterror.ErrRepairTableFail.GenWithStackByArgs("Partition type should be the same") } // Check whether partitionType is hash partition. - if newTableInfo.Partition.Type == pmodel.PartitionTypeHash { + if newTableInfo.Partition.Type == ast.PartitionTypeHash { if newTableInfo.Partition.Num != oldTableInfo.Partition.Num { return dbterror.ErrRepairTableFail.GenWithStackByArgs("Hash partition num should be the same") } @@ -2010,7 +2009,7 @@ func getRangeValue(ctx expression.BuildContext, str string, unsigned bool) (any, // CheckDropTablePartition checks if the partition exists and does not allow deleting the last existing partition in the table. func CheckDropTablePartition(meta *model.TableInfo, partLowerNames []string) error { pi := meta.Partition - if pi.Type != pmodel.PartitionTypeRange && pi.Type != pmodel.PartitionTypeList { + if pi.Type != ast.PartitionTypeRange && pi.Type != ast.PartitionTypeList { return dbterror.ErrOnlyOnRangeListPartition.GenWithStackByArgs("DROP") } @@ -2148,15 +2147,15 @@ func (w *worker) rollbackLikeDropPartition(jobCtx *jobContext, job *model.Job) ( job.State = model.JobStateCancelled return ver, err } - if partInfo.Type != pmodel.PartitionTypeNone { + if partInfo.Type != ast.PartitionTypeNone { // ALTER TABLE ... PARTITION BY // Also remove anything with the new table id if partInfo.NewTableID != 0 { physicalTableIDs = append(physicalTableIDs, partInfo.NewTableID) } // Reset if it was normal table before - if tblInfo.Partition.Type == pmodel.PartitionTypeNone || - tblInfo.Partition.DDLType == pmodel.PartitionTypeNone { + if tblInfo.Partition.Type == ast.PartitionTypeNone || + tblInfo.Partition.DDLType == ast.PartitionTypeNone { tblInfo.Partition = nil } } @@ -4310,7 +4309,7 @@ func checkExchangePartitionRecordValidation( pi := pt.Partition switch pi.Type { - case pmodel.PartitionTypeHash: + case ast.PartitionTypeHash: if pi.Num == 1 { checkNt = false } else { @@ -4327,7 +4326,7 @@ func checkExchangePartitionRecordValidation( paramList = append(paramList, pi.Num, index) } } - case pmodel.PartitionTypeRange: + case ast.PartitionTypeRange: // Table has only one partition and has the maximum value if len(pi.Definitions) == 1 && strings.EqualFold(pi.Definitions[index].LessThan[0], partitionMaxValue) { checkNt = false @@ -4343,7 +4342,7 @@ func checkExchangePartitionRecordValidation( paramList = append(paramList, params...) } } - case pmodel.PartitionTypeList: + case ast.PartitionTypeList: if len(pi.Columns) == 0 { conds := buildCheckSQLConditionForListPartition(pi, index) buf.WriteString(conds) @@ -4618,7 +4617,7 @@ func checkPartitionKeysConstraint(pi *model.PartitionInfo, indexColumns []*model partCols []*model.ColumnInfo err error ) - if pi.Type == pmodel.PartitionTypeNone { + if pi.Type == ast.PartitionTypeNone { return true, nil } // The expr will be an empty string if the partition is defined by: @@ -5020,8 +5019,8 @@ func AppendPartitionInfo(partitionInfo *model.PartitionInfo, buf *bytes.Buffer, // This also solves the issue with comments within comments that would happen for // PLACEMENT POLICY options. defaultPartitionDefinitions := true - if partitionInfo.Type == pmodel.PartitionTypeHash || - partitionInfo.Type == pmodel.PartitionTypeKey { + if partitionInfo.Type == ast.PartitionTypeHash || + partitionInfo.Type == ast.PartitionTypeKey { for i, def := range partitionInfo.Definitions { if def.Name.O != fmt.Sprintf("p%d", i) { defaultPartitionDefinitions = false @@ -5034,7 +5033,7 @@ func AppendPartitionInfo(partitionInfo *model.PartitionInfo, buf *bytes.Buffer, } if defaultPartitionDefinitions { - if partitionInfo.Type == pmodel.PartitionTypeHash { + if partitionInfo.Type == ast.PartitionTypeHash { fmt.Fprintf(buf, "\nPARTITION BY HASH (%s) PARTITIONS %d", partitionInfo.Expr, partitionInfo.Num) } else { buf.WriteString("\nPARTITION BY KEY (") @@ -5050,7 +5049,7 @@ func AppendPartitionInfo(partitionInfo *model.PartitionInfo, buf *bytes.Buffer, // partitionInfo.Type == model.PartitionTypeRange || partitionInfo.Type == model.PartitionTypeList // || partitionInfo.Type == model.PartitionTypeKey // Notice that MySQL uses two spaces between LIST and COLUMNS... - if partitionInfo.Type == pmodel.PartitionTypeKey { + if partitionInfo.Type == ast.PartitionTypeKey { fmt.Fprintf(buf, "\nPARTITION BY %s (", partitionInfo.Type.String()) } else { fmt.Fprintf(buf, "\nPARTITION BY %s COLUMNS(", partitionInfo.Type.String()) @@ -5075,13 +5074,13 @@ func AppendPartitionDefs(partitionInfo *model.PartitionInfo, buf *bytes.Buffer, } fmt.Fprintf(buf, "PARTITION %s", stringutil.Escape(def.Name.O, sqlMode)) // PartitionTypeHash and PartitionTypeKey do not have any VALUES definition - if partitionInfo.Type == pmodel.PartitionTypeRange { + if partitionInfo.Type == ast.PartitionTypeRange { lessThans := make([]string, len(def.LessThan)) for idx, v := range def.LessThan { lessThans[idx] = hexIfNonPrint(v) } fmt.Fprintf(buf, " VALUES LESS THAN (%s)", strings.Join(lessThans, ",")) - } else if partitionInfo.Type == pmodel.PartitionTypeList { + } else if partitionInfo.Type == ast.PartitionTypeList { if len(def.InValues) == 0 { fmt.Fprintf(buf, " DEFAULT") } else if len(def.InValues) == 1 && @@ -5162,14 +5161,14 @@ func checkPartitionDefinitionConstraints(ctx expression.BuildContext, tbInfo *mo } switch tbInfo.Partition.Type { - case pmodel.PartitionTypeRange: + case ast.PartitionTypeRange: failpoint.Inject("CheckPartitionByRangeErr", func() { panic("mockCheckPartitionByRangeErr") }) err = checkPartitionByRange(ctx, tbInfo) - case pmodel.PartitionTypeHash, pmodel.PartitionTypeKey: + case ast.PartitionTypeHash, ast.PartitionTypeKey: err = checkPartitionByHash(tbInfo) - case pmodel.PartitionTypeList: + case ast.PartitionTypeList: err = checkPartitionByList(ctx, tbInfo) } return errors.Trace(err) diff --git a/pkg/ddl/partition_test.go b/pkg/ddl/partition_test.go index c06f454fa454b..f57fc6aacb31c 100644 --- a/pkg/ddl/partition_test.go +++ b/pkg/ddl/partition_test.go @@ -22,7 +22,7 @@ import ( "github.com/pingcap/tidb/pkg/ddl" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "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/testkit" @@ -51,11 +51,11 @@ func TestDropAndTruncatePartition(t *testing.T) { func buildTableInfoWithPartition(t *testing.T, store kv.Storage) (*model.TableInfo, []int64) { tbl := &model.TableInfo{ - Name: pmodel.NewCIStr("t"), + Name: ast.NewCIStr("t"), } tbl.MaxColumnID++ col := &model.ColumnInfo{ - Name: pmodel.NewCIStr("c"), + Name: ast.NewCIStr("c"), Offset: 0, State: model.StatePublic, FieldType: *types.NewFieldType(mysql.TypeLong), @@ -71,33 +71,33 @@ func buildTableInfoWithPartition(t *testing.T, store kv.Storage) (*model.TableIn partIDs, err := genGlobalIDs(store, 5) require.NoError(t, err) partInfo := &model.PartitionInfo{ - Type: pmodel.PartitionTypeRange, + Type: ast.PartitionTypeRange, Expr: tbl.Columns[0].Name.L, Enable: true, Definitions: []model.PartitionDefinition{ { ID: partIDs[0], - Name: pmodel.NewCIStr("p0"), + Name: ast.NewCIStr("p0"), LessThan: []string{"100"}, }, { ID: partIDs[1], - Name: pmodel.NewCIStr("p1"), + Name: ast.NewCIStr("p1"), LessThan: []string{"200"}, }, { ID: partIDs[2], - Name: pmodel.NewCIStr("p2"), + Name: ast.NewCIStr("p2"), LessThan: []string{"300"}, }, { ID: partIDs[3], - Name: pmodel.NewCIStr("p3"), + Name: ast.NewCIStr("p3"), LessThan: []string{"400"}, }, { ID: partIDs[4], - Name: pmodel.NewCIStr("p4"), + Name: ast.NewCIStr("p4"), LessThan: []string{"500"}, }, }, @@ -241,7 +241,7 @@ func TestReorganizePartitionRollback(t *testing.T) { " PARTITION `p3` VALUES LESS THAN (8000000),\n" + " PARTITION `p4` VALUES LESS THAN (10000000),\n" + " PARTITION `p5` VALUES LESS THAN (MAXVALUE))")) - tbl, err := do.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t1")) + tbl, err := do.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t1")) require.NoError(t, err) require.NotNil(t, tbl.Meta().Partition) require.Nil(t, tbl.Meta().Partition.AddingDefinitions) diff --git a/pkg/ddl/placement/BUILD.bazel b/pkg/ddl/placement/BUILD.bazel index 56938bf82e106..1b80cb234711e 100644 --- a/pkg/ddl/placement/BUILD.bazel +++ b/pkg/ddl/placement/BUILD.bazel @@ -41,7 +41,7 @@ go_test( "//pkg/kv", "//pkg/meta", "//pkg/meta/model", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/store/mockstore", "//pkg/tablecodec", "//pkg/util/codec", diff --git a/pkg/ddl/placement/meta_bundle_test.go b/pkg/ddl/placement/meta_bundle_test.go index f261bc396b4a1..44639ba73a915 100644 --- a/pkg/ddl/placement/meta_bundle_test.go +++ b/pkg/ddl/placement/meta_bundle_test.go @@ -25,7 +25,7 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/store/mockstore" "github.com/pingcap/tidb/pkg/tablecodec" "github.com/pingcap/tidb/pkg/util/codec" @@ -47,7 +47,7 @@ func createMetaBundleSuite() *metaBundleSuite { s := new(metaBundleSuite) s.policy1 = &model.PolicyInfo{ ID: 11, - Name: pmodel.NewCIStr("p1"), + Name: ast.NewCIStr("p1"), PlacementSettings: &model.PlacementSettings{ PrimaryRegion: "r1", Regions: "r1,r2", @@ -56,7 +56,7 @@ func createMetaBundleSuite() *metaBundleSuite { } s.policy2 = &model.PolicyInfo{ ID: 12, - Name: pmodel.NewCIStr("p2"), + Name: ast.NewCIStr("p2"), PlacementSettings: &model.PlacementSettings{ PrimaryRegion: "r2", Regions: "r1,r2", @@ -65,7 +65,7 @@ func createMetaBundleSuite() *metaBundleSuite { } s.policy3 = &model.PolicyInfo{ ID: 13, - Name: pmodel.NewCIStr("p3"), + Name: ast.NewCIStr("p3"), PlacementSettings: &model.PlacementSettings{ LeaderConstraints: "[+region=bj]", }, @@ -73,58 +73,58 @@ func createMetaBundleSuite() *metaBundleSuite { } s.tbl1 = &model.TableInfo{ ID: 101, - Name: pmodel.NewCIStr("t1"), + Name: ast.NewCIStr("t1"), PlacementPolicyRef: &model.PolicyRefInfo{ ID: 11, - Name: pmodel.NewCIStr("p1"), + Name: ast.NewCIStr("p1"), }, Partition: &model.PartitionInfo{ Definitions: []model.PartitionDefinition{ { ID: 1000, - Name: pmodel.NewCIStr("par0"), + Name: ast.NewCIStr("par0"), }, { ID: 1001, - Name: pmodel.NewCIStr("par1"), - PlacementPolicyRef: &model.PolicyRefInfo{ID: 12, Name: pmodel.NewCIStr("p2")}, + Name: ast.NewCIStr("par1"), + PlacementPolicyRef: &model.PolicyRefInfo{ID: 12, Name: ast.NewCIStr("p2")}, }, { ID: 1002, - Name: pmodel.NewCIStr("par2"), + Name: ast.NewCIStr("par2"), }, }, }, } s.tbl2 = &model.TableInfo{ ID: 102, - Name: pmodel.NewCIStr("t2"), + Name: ast.NewCIStr("t2"), Partition: &model.PartitionInfo{ Definitions: []model.PartitionDefinition{ { ID: 1000, - Name: pmodel.NewCIStr("par0"), - PlacementPolicyRef: &model.PolicyRefInfo{ID: 11, Name: pmodel.NewCIStr("p1")}, + Name: ast.NewCIStr("par0"), + PlacementPolicyRef: &model.PolicyRefInfo{ID: 11, Name: ast.NewCIStr("p1")}, }, { ID: 1001, - Name: pmodel.NewCIStr("par1"), + Name: ast.NewCIStr("par1"), }, { ID: 1002, - Name: pmodel.NewCIStr("par2"), + Name: ast.NewCIStr("par2"), }, }, }, } s.tbl3 = &model.TableInfo{ ID: 103, - Name: pmodel.NewCIStr("t3"), - PlacementPolicyRef: &model.PolicyRefInfo{ID: 13, Name: pmodel.NewCIStr("p3")}, + Name: ast.NewCIStr("t3"), + PlacementPolicyRef: &model.PolicyRefInfo{ID: 13, Name: ast.NewCIStr("p3")}, } s.tbl4 = &model.TableInfo{ ID: 104, - Name: pmodel.NewCIStr("t4"), + Name: ast.NewCIStr("t4"), } return s } diff --git a/pkg/ddl/placement_policy.go b/pkg/ddl/placement_policy.go index deb93dad26211..c758586afcd87 100644 --- a/pkg/ddl/placement_policy.go +++ b/pkg/ddl/placement_policy.go @@ -27,7 +27,6 @@ import ( "github.com/pingcap/tidb/pkg/meta" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/ast" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/sessiontxn" @@ -119,7 +118,7 @@ func getPolicyInfo(t *meta.Mutator, policyID int64) (*model.PolicyInfo, error) { return policy, nil } -func getPlacementPolicyByName(infoCache *infoschema.InfoCache, t *meta.Mutator, policyName pmodel.CIStr) (*model.PolicyInfo, error) { +func getPlacementPolicyByName(infoCache *infoschema.InfoCache, t *meta.Mutator, policyName ast.CIStr) (*model.PolicyInfo, error) { currVer, err := t.GetSchemaVersion() if err != nil { return nil, err @@ -506,7 +505,7 @@ func GetRangePlacementPolicyName(ctx context.Context, rangeBundleID string) (str return "", nil } -func buildPolicyInfo(name pmodel.CIStr, options []*ast.PlacementOption) (*model.PolicyInfo, error) { +func buildPolicyInfo(name ast.CIStr, options []*ast.PlacementOption) (*model.PolicyInfo, error) { policyInfo := &model.PolicyInfo{PlacementSettings: &model.PlacementSettings{}} policyInfo.Name = name for _, opt := range options { diff --git a/pkg/ddl/placement_policy_ddl_test.go b/pkg/ddl/placement_policy_ddl_test.go index a6d9661d3ee99..ded9710e3af59 100644 --- a/pkg/ddl/placement_policy_ddl_test.go +++ b/pkg/ddl/placement_policy_ddl_test.go @@ -23,7 +23,7 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/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/testkit" @@ -33,7 +33,7 @@ import ( func testPlacementPolicyInfo(t *testing.T, store kv.Storage, name string, settings *model.PlacementSettings) *model.PolicyInfo { policy := &model.PolicyInfo{ - Name: pmodel.NewCIStr(name), + Name: ast.NewCIStr(name), PlacementSettings: settings, } genIDs, err := genGlobalIDs(store, 1) @@ -161,12 +161,12 @@ func testTableInfoWithPartition(t *testing.T, store kv.Storage, name string, num require.NoError(t, err) pid := genIDs[0] tblInfo.Partition = &model.PartitionInfo{ - Type: pmodel.PartitionTypeRange, + Type: ast.PartitionTypeRange, Expr: tblInfo.Columns[0].Name.L, Enable: true, Definitions: []model.PartitionDefinition{{ ID: pid, - Name: pmodel.NewCIStr("p0"), + Name: ast.NewCIStr("p0"), LessThan: []string{"maxvalue"}, }}, } diff --git a/pkg/ddl/placement_policy_test.go b/pkg/ddl/placement_policy_test.go index 2987d8f121dc5..889920ca22d3e 100644 --- a/pkg/ddl/placement_policy_test.go +++ b/pkg/ddl/placement_policy_test.go @@ -33,7 +33,7 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + ast "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/store/gcworker" "github.com/pingcap/tidb/pkg/testkit" @@ -81,7 +81,7 @@ func (c *bundleCheck) check(t *testing.T, is infoschema.InfoSchema) { } func checkExistTableBundlesInPD(t *testing.T, do *domain.Domain, dbName string, tbName string) { - tblInfo, err := do.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr(dbName), pmodel.NewCIStr(tbName)) + tblInfo, err := do.InfoSchema().TableByName(context.Background(), ast.NewCIStr(dbName), ast.NewCIStr(tbName)) require.NoError(t, err) ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnDDL) @@ -278,27 +278,27 @@ PARTITION p1 VALUES LESS THAN (1000)) `) defer tk.MustExec("drop table if exists tp") - oldPolicy, ok := dom.InfoSchema().PolicyByName(pmodel.NewCIStr("p")) + oldPolicy, ok := dom.InfoSchema().PolicyByName(ast.NewCIStr("p")) oldPolicy = oldPolicy.Clone() require.True(t, ok) // create a non exist policy for _, onExist := range []ddl.OnExist{ddl.OnExistReplace, ddl.OnExistIgnore, ddl.OnExistError} { newPolicy := oldPolicy.Clone() - newPolicy.Name = pmodel.NewCIStr("p2") + newPolicy.Name = ast.NewCIStr("p2") newPolicy.Followers = 2 newPolicy.LearnerConstraints = "[+zone=z2]" tk.Session().SetValue(sessionctx.QueryString, "skip") err := dom.DDLExecutor().CreatePlacementPolicyWithInfo(tk.Session(), newPolicy.Clone(), onExist) require.NoError(t, err) // old policy should not be changed - found, ok := dom.InfoSchema().PolicyByName(pmodel.NewCIStr("p")) + found, ok := dom.InfoSchema().PolicyByName(ast.NewCIStr("p")) require.True(t, ok) checkPolicyEquals(t, oldPolicy, found) checkExistTableBundlesInPD(t, dom, "test", "tp") // new created policy - found, ok = dom.InfoSchema().PolicyByName(pmodel.NewCIStr("p2")) + found, ok = dom.InfoSchema().PolicyByName(ast.NewCIStr("p2")) require.True(t, ok) // ID of the created policy should be reassigned require.NotEqual(t, newPolicy.ID, found.ID) @@ -314,7 +314,7 @@ PARTITION p1 VALUES LESS THAN (1000)) err := dom.DDLExecutor().CreatePlacementPolicyWithInfo(tk.Session(), newPolicy.Clone(), ddl.OnExistError) require.Error(t, err) require.True(t, infoschema.ErrPlacementPolicyExists.Equal(err)) - found, ok := dom.InfoSchema().PolicyByName(pmodel.NewCIStr("p")) + found, ok := dom.InfoSchema().PolicyByName(ast.NewCIStr("p")) require.True(t, ok) checkPolicyEquals(t, oldPolicy, found) checkExistTableBundlesInPD(t, dom, "test", "tp") @@ -325,7 +325,7 @@ PARTITION p1 VALUES LESS THAN (1000)) tk.Session().SetValue(sessionctx.QueryString, "skip") err = dom.DDLExecutor().CreatePlacementPolicyWithInfo(tk.Session(), newPolicy.Clone(), ddl.OnExistIgnore) require.NoError(t, err) - found, ok = dom.InfoSchema().PolicyByName(pmodel.NewCIStr("p")) + found, ok = dom.InfoSchema().PolicyByName(ast.NewCIStr("p")) require.True(t, ok) checkPolicyEquals(t, oldPolicy, found) checkExistTableBundlesInPD(t, dom, "test", "tp") @@ -338,7 +338,7 @@ PARTITION p1 VALUES LESS THAN (1000)) tk.Session().SetValue(sessionctx.QueryString, "skip") err = dom.DDLExecutor().CreatePlacementPolicyWithInfo(tk.Session(), newPolicy.Clone(), ddl.OnExistReplace) require.NoError(t, err) - found, ok = dom.InfoSchema().PolicyByName(pmodel.NewCIStr("p")) + found, ok = dom.InfoSchema().PolicyByName(ast.NewCIStr("p")) require.True(t, ok) // when replace a policy the old policy's id should not be changed newPolicy.ID = oldPolicy.ID @@ -391,7 +391,7 @@ func testGetPolicyByNameFromIS(t *testing.T, ctx sessionctx.Context, policy stri // Make sure the table schema is the new schema. err := dom.Reload() require.NoError(t, err) - po, ok := dom.InfoSchema().PolicyByName(pmodel.NewCIStr(policy)) + po, ok := dom.InfoSchema().PolicyByName(ast.NewCIStr(policy)) require.Equal(t, true, ok) return po } @@ -558,7 +558,7 @@ func TestAlterPlacementPolicy(t *testing.T) { );`) defer tk.MustExec("drop table if exists tp") - policy, ok := dom.InfoSchema().PolicyByName(pmodel.NewCIStr("x")) + policy, ok := dom.InfoSchema().PolicyByName(ast.NewCIStr("x")) require.True(t, ok) // test for normal cases @@ -728,7 +728,7 @@ func TestCreateTableWithPlacementPolicy(t *testing.T) { } func getClonedTable(dom *domain.Domain, dbName string, tableName string) (*model.TableInfo, error) { - tbl, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr(dbName), pmodel.NewCIStr(tableName)) + tbl, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr(dbName), ast.NewCIStr(tableName)) if err != nil { return nil, err } @@ -741,7 +741,7 @@ func getClonedTable(dom *domain.Domain, dbName string, tableName string) (*model } func getClonedDatabase(dom *domain.Domain, dbName string) (*model.DBInfo, bool) { - db, ok := dom.InfoSchema().SchemaByName(pmodel.NewCIStr(dbName)) + db, ok := dom.InfoSchema().SchemaByName(ast.NewCIStr(dbName)) if !ok { return nil, ok } @@ -770,7 +770,7 @@ func TestCreateTableWithInfoPlacement(t *testing.T) { tbl, err := getClonedTable(dom, "test", "t1") require.NoError(t, err) - policy, ok := dom.InfoSchema().PolicyByName(pmodel.NewCIStr("p1")) + policy, ok := dom.InfoSchema().PolicyByName(ast.NewCIStr("p1")) require.True(t, ok) require.Equal(t, policy.ID, tbl.PlacementPolicyRef.ID) @@ -778,7 +778,7 @@ func TestCreateTableWithInfoPlacement(t *testing.T) { tk.MustExec("drop placement policy p1") tk.MustExec("create placement policy p1 followers=2") tk.Session().SetValue(sessionctx.QueryString, "skip") - require.Nil(t, dom.DDLExecutor().CreateTableWithInfo(tk.Session(), pmodel.NewCIStr("test2"), tbl, nil, ddl.WithOnExist(ddl.OnExistError))) + require.Nil(t, dom.DDLExecutor().CreateTableWithInfo(tk.Session(), ast.NewCIStr("test2"), tbl, nil, ddl.WithOnExist(ddl.OnExistError))) tk.MustQuery("show create table t1").Check(testkit.Rows("t1 CREATE TABLE `t1` (\n" + " `a` int(11) DEFAULT NULL\n" + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin")) @@ -790,16 +790,16 @@ func TestCreateTableWithInfoPlacement(t *testing.T) { // The ref id for new table should be the new policy id tbl2, err := getClonedTable(dom, "test2", "t1") require.NoError(t, err) - policy2, ok := dom.InfoSchema().PolicyByName(pmodel.NewCIStr("p1")) + policy2, ok := dom.InfoSchema().PolicyByName(ast.NewCIStr("p1")) require.True(t, ok) require.Equal(t, policy2.ID, tbl2.PlacementPolicyRef.ID) require.True(t, policy2.ID != policy.ID) // Test policy not exists - tbl2.Name = pmodel.NewCIStr("t3") - tbl2.PlacementPolicyRef.Name = pmodel.NewCIStr("pxx") + tbl2.Name = ast.NewCIStr("t3") + tbl2.PlacementPolicyRef.Name = ast.NewCIStr("pxx") tk.Session().SetValue(sessionctx.QueryString, "skip") - err = dom.DDLExecutor().CreateTableWithInfo(tk.Session(), pmodel.NewCIStr("test2"), tbl2, nil, ddl.WithOnExist(ddl.OnExistError)) + err = dom.DDLExecutor().CreateTableWithInfo(tk.Session(), ast.NewCIStr("test2"), tbl2, nil, ddl.WithOnExist(ddl.OnExistError)) require.Equal(t, "[schema:8239]Unknown placement policy 'pxx'", err.Error()) } @@ -820,12 +820,12 @@ func TestCreateSchemaWithInfoPlacement(t *testing.T) { db, ok := getClonedDatabase(dom, "test2") require.True(t, ok) - policy, ok := dom.InfoSchema().PolicyByName(pmodel.NewCIStr("p1")) + policy, ok := dom.InfoSchema().PolicyByName(ast.NewCIStr("p1")) require.True(t, ok) require.Equal(t, policy.ID, db.PlacementPolicyRef.ID) db2 := db.Clone() - db2.Name = pmodel.NewCIStr("test3") + db2.Name = ast.NewCIStr("test3") tk.MustExec("alter database test2 placement policy='default'") tk.MustExec("drop placement policy p1") tk.MustExec("create placement policy p1 followers=2") @@ -838,14 +838,14 @@ func TestCreateSchemaWithInfoPlacement(t *testing.T) { // The ref id for new table should be the new policy id db2, ok = getClonedDatabase(dom, "test3") require.True(t, ok) - policy2, ok := dom.InfoSchema().PolicyByName(pmodel.NewCIStr("p1")) + policy2, ok := dom.InfoSchema().PolicyByName(ast.NewCIStr("p1")) require.True(t, ok) require.Equal(t, policy2.ID, db2.PlacementPolicyRef.ID) require.True(t, policy2.ID != policy.ID) // Test policy not exists - db2.Name = pmodel.NewCIStr("test4") - db2.PlacementPolicyRef.Name = pmodel.NewCIStr("p2") + db2.Name = ast.NewCIStr("test4") + db2.PlacementPolicyRef.Name = ast.NewCIStr("p2") tk.Session().SetValue(sessionctx.QueryString, "skip") err := dom.DDLExecutor().CreateSchemaWithInfo(tk.Session(), db2, ddl.OnExistError) require.Equal(t, "[schema:8239]Unknown placement policy 'p2'", err.Error()) @@ -967,7 +967,7 @@ func testGetPolicyByName(t *testing.T, ctx sessionctx.Context, name string, must // Make sure the table schema is the new schema. err := dom.Reload() require.NoError(t, err) - po, ok := dom.InfoSchema().PolicyByName(pmodel.NewCIStr(name)) + po, ok := dom.InfoSchema().PolicyByName(ast.NewCIStr(name)) if mustExist { require.Equal(t, true, ok) } @@ -1106,7 +1106,7 @@ func TestAlterTablePartitionWithPlacementPolicy(t *testing.T) { tk.MustQuery("SELECT TABLE_CATALOG, TABLE_SCHEMA, TABLE_NAME, PARTITION_NAME, TIDB_PLACEMENT_POLICY_NAME FROM information_schema.Partitions WHERE TABLE_SCHEMA='test' AND TABLE_NAME = 't1' AND PARTITION_NAME = 'p0'").Check(testkit.Rows(`def test t1 p0 x`)) checkExistTableBundlesInPD(t, dom, "test", "t1") - policyX, ok := dom.InfoSchema().PolicyByName(pmodel.NewCIStr("x")) + policyX, ok := dom.InfoSchema().PolicyByName(ast.NewCIStr("x")) require.True(t, ok) ptDef := testGetPartitionDefinitionsByName(t, tk.Session(), "test", "t1", "p0") require.NotNil(t, ptDef) @@ -1120,7 +1120,7 @@ func testGetPartitionDefinitionsByName(t *testing.T, ctx sessionctx.Context, db // Make sure the table schema is the new schema. err := dom.Reload() require.NoError(t, err) - tbl, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr(db), pmodel.NewCIStr(table)) + tbl, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr(db), ast.NewCIStr(table)) require.NoError(t, err) require.NotNil(t, tbl) var ptDef model.PartitionDefinition @@ -1254,7 +1254,7 @@ func TestDatabasePlacement(t *testing.T) { tk.MustExec("create placement policy p2 primary_region='r2' regions='r1,r2'") defer tk.MustExec("drop placement policy p2") - policy1, ok := dom.InfoSchema().PolicyByName(pmodel.NewCIStr("p1")) + policy1, ok := dom.InfoSchema().PolicyByName(ast.NewCIStr("p1")) require.True(t, ok) tk.MustExec(`create database db2`) @@ -1263,7 +1263,7 @@ func TestDatabasePlacement(t *testing.T) { "db2 CREATE DATABASE `db2` /*!40100 DEFAULT CHARACTER SET utf8mb4 */", )) - policy2, ok := dom.InfoSchema().PolicyByName(pmodel.NewCIStr("p2")) + policy2, ok := dom.InfoSchema().PolicyByName(ast.NewCIStr("p2")) require.True(t, ok) // alter with policy @@ -1272,7 +1272,7 @@ func TestDatabasePlacement(t *testing.T) { "db2 CREATE DATABASE `db2` /*!40100 DEFAULT CHARACTER SET utf8mb4 */ /*T![placement] PLACEMENT POLICY=`p1` */", )) - db, ok := dom.InfoSchema().SchemaByName(pmodel.NewCIStr("db2")) + db, ok := dom.InfoSchema().SchemaByName(ast.NewCIStr("db2")) require.True(t, ok) require.Equal(t, policy1.ID, db.PlacementPolicyRef.ID) @@ -1281,7 +1281,7 @@ func TestDatabasePlacement(t *testing.T) { "db2 CREATE DATABASE `db2` /*!40100 DEFAULT CHARACTER SET utf8mb4 */ /*T![placement] PLACEMENT POLICY=`p2` */", )) - db, ok = dom.InfoSchema().SchemaByName(pmodel.NewCIStr("db2")) + db, ok = dom.InfoSchema().SchemaByName(ast.NewCIStr("db2")) require.True(t, ok) require.Equal(t, policy2.ID, db.PlacementPolicyRef.ID) @@ -1291,7 +1291,7 @@ func TestDatabasePlacement(t *testing.T) { "db2 CREATE DATABASE `db2` /*!40100 DEFAULT CHARACTER SET utf8mb4 */", )) - db, ok = dom.InfoSchema().SchemaByName(pmodel.NewCIStr("db2")) + db, ok = dom.InfoSchema().SchemaByName(ast.NewCIStr("db2")) require.True(t, ok) require.Nil(t, db.PlacementPolicyRef) @@ -1345,7 +1345,7 @@ func TestDropDatabaseGCPlacement(t *testing.T) { )`) is := dom.InfoSchema() - tt, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tt, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tk.MustExec("drop database db2") @@ -1402,7 +1402,7 @@ func TestDropTableGCPlacement(t *testing.T) { defer tk.MustExec("drop table if exists t2") is := dom.InfoSchema() - t1, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t1")) + t1, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t1")) require.NoError(t, err) tk.MustExec("drop table t2") @@ -1437,7 +1437,7 @@ func TestAlterTablePlacement(t *testing.T) { tk.MustExec("create placement policy p1 primary_region='r1' regions='r1'") defer tk.MustExec("drop placement policy p1") - policy, ok := dom.InfoSchema().PolicyByName(pmodel.NewCIStr("p1")) + policy, ok := dom.InfoSchema().PolicyByName(ast.NewCIStr("p1")) require.True(t, ok) tk.MustExec(`CREATE TABLE tp (id INT) PARTITION BY RANGE (id) ( @@ -1464,7 +1464,7 @@ func TestAlterTablePlacement(t *testing.T) { "(PARTITION `p0` VALUES LESS THAN (100),\n" + " PARTITION `p1` VALUES LESS THAN (1000))")) - tb, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("tp")) + tb, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("tp")) require.NoError(t, err) require.Equal(t, policy.ID, tb.Meta().PlacementPolicyRef.ID) checkExistTableBundlesInPD(t, dom, "test", "tp") @@ -1537,9 +1537,9 @@ func TestDropTablePartitionGCPlacement(t *testing.T) { defer tk.MustExec("drop table if exists t2") is := dom.InfoSchema() - t1, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t1")) + t1, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t1")) require.NoError(t, err) - t2, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t2")) + t2, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t2")) require.NoError(t, err) tk.MustExec("alter table t2 drop partition p0") @@ -1600,7 +1600,7 @@ func TestAlterTablePartitionPlacement(t *testing.T) { tk.MustExec("create placement policy p1 primary_region='r1' regions='r1'") defer tk.MustExec("drop placement policy p1") - policy, ok := dom.InfoSchema().PolicyByName(pmodel.NewCIStr("p1")) + policy, ok := dom.InfoSchema().PolicyByName(ast.NewCIStr("p1")) require.True(t, ok) tk.MustExec(`CREATE TABLE tp (id INT) placement policy p0 PARTITION BY RANGE (id) ( @@ -1627,7 +1627,7 @@ func TestAlterTablePartitionPlacement(t *testing.T) { "(PARTITION `p0` VALUES LESS THAN (100) /*T![placement] PLACEMENT POLICY=`p1` */,\n" + " PARTITION `p1` VALUES LESS THAN (1000))")) - tb, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("tp")) + tb, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("tp")) require.NoError(t, err) require.Equal(t, policy.ID, tb.Meta().Partition.Definitions[0].PlacementPolicyRef.ID) checkExistTableBundlesInPD(t, dom, "test", "tp") @@ -1698,7 +1698,7 @@ func TestAddPartitionWithPlacement(t *testing.T) { tk.MustExec("create placement policy p2 primary_region='r2' regions='r2'") defer tk.MustExec("drop placement policy p2") - policy2, ok := dom.InfoSchema().PolicyByName(pmodel.NewCIStr("p2")) + policy2, ok := dom.InfoSchema().PolicyByName(ast.NewCIStr("p2")) require.True(t, ok) tk.MustExec(`CREATE TABLE tp (id INT) PLACEMENT POLICY p1 PARTITION BY RANGE (id) ( @@ -1733,7 +1733,7 @@ func TestAddPartitionWithPlacement(t *testing.T) { " PARTITION `p4` VALUES LESS THAN (1000000))")) checkExistTableBundlesInPD(t, dom, "test", "tp") - tb, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("tp")) + tb, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("tp")) require.NoError(t, err) require.Equal(t, policy2.ID, tb.Meta().Partition.Definitions[2].PlacementPolicyRef.ID) @@ -1781,10 +1781,10 @@ func TestTruncateTableWithPlacement(t *testing.T) { tk.MustExec("create placement policy p2 primary_region='r2' regions='r2'") defer tk.MustExec("drop placement policy p2") - policy1, ok := dom.InfoSchema().PolicyByName(pmodel.NewCIStr("p1")) + policy1, ok := dom.InfoSchema().PolicyByName(ast.NewCIStr("p1")) require.True(t, ok) - policy2, ok := dom.InfoSchema().PolicyByName(pmodel.NewCIStr("p2")) + policy2, ok := dom.InfoSchema().PolicyByName(ast.NewCIStr("p2")) require.True(t, ok) tk.MustExec(`CREATE TABLE t1 (id INT) placement policy p1`) @@ -1796,7 +1796,7 @@ func TestTruncateTableWithPlacement(t *testing.T) { " `id` int(11) DEFAULT NULL\n" + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![placement] PLACEMENT POLICY=`p1` */")) - t1, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t1")) + t1, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t1")) require.NoError(t, err) checkExistTableBundlesInPD(t, dom, "test", "t1") @@ -1805,7 +1805,7 @@ func TestTruncateTableWithPlacement(t *testing.T) { "t1 CREATE TABLE `t1` (\n" + " `id` int(11) DEFAULT NULL\n" + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![placement] PLACEMENT POLICY=`p1` */")) - newT1, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t1")) + newT1, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t1")) require.NoError(t, err) require.True(t, newT1.Meta().ID != t1.Meta().ID) checkExistTableBundlesInPD(t, dom, "test", "t1") @@ -1819,7 +1819,7 @@ func TestTruncateTableWithPlacement(t *testing.T) { );`) defer tk.MustExec("drop table tp") - tp, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("tp")) + tp, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("tp")) require.NoError(t, err) require.Equal(t, policy1.ID, tp.Meta().PlacementPolicyRef.ID) require.Equal(t, policy2.ID, tp.Meta().Partition.Definitions[1].PlacementPolicyRef.ID) @@ -1834,7 +1834,7 @@ func TestTruncateTableWithPlacement(t *testing.T) { checkExistTableBundlesInPD(t, dom, "test", "tp") tk.MustExec("TRUNCATE TABLE tp") - newTp, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("tp")) + newTp, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("tp")) require.NoError(t, err) require.True(t, newTp.Meta().ID != tp.Meta().ID) require.Equal(t, policy1.ID, newTp.Meta().PlacementPolicyRef.ID) @@ -1912,13 +1912,13 @@ func TestTruncateTablePartitionWithPlacement(t *testing.T) { tk.MustExec("create placement policy p3 primary_region='r3' regions='r3'") defer tk.MustExec("drop placement policy p3") - policy1, ok := dom.InfoSchema().PolicyByName(pmodel.NewCIStr("p1")) + policy1, ok := dom.InfoSchema().PolicyByName(ast.NewCIStr("p1")) require.True(t, ok) - policy2, ok := dom.InfoSchema().PolicyByName(pmodel.NewCIStr("p2")) + policy2, ok := dom.InfoSchema().PolicyByName(ast.NewCIStr("p2")) require.True(t, ok) - policy3, ok := dom.InfoSchema().PolicyByName(pmodel.NewCIStr("p3")) + policy3, ok := dom.InfoSchema().PolicyByName(ast.NewCIStr("p3")) require.True(t, ok) // test for partitioned table @@ -1930,7 +1930,7 @@ func TestTruncateTablePartitionWithPlacement(t *testing.T) { );`) defer tk.MustExec("drop table tp") - tp, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("tp")) + tp, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("tp")) require.NoError(t, err) checkOldPartitions := make([]model.PartitionDefinition, 0, 2) @@ -1945,7 +1945,7 @@ func TestTruncateTablePartitionWithPlacement(t *testing.T) { } tk.MustExec("ALTER TABLE tp TRUNCATE partition p1,p3") - newTp, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("tp")) + newTp, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("tp")) require.NoError(t, err) require.Equal(t, tp.Meta().ID, newTp.Meta().ID) require.Equal(t, policy1.ID, newTp.Meta().PlacementPolicyRef.ID) @@ -1974,7 +1974,7 @@ func TestTruncateTablePartitionWithPlacement(t *testing.T) { // add new partition will not override bundle waiting for GC tk.MustExec("alter table tp add partition (partition p4 values less than(1000000))") - newTp2, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("tp")) + newTp2, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("tp")) require.NoError(t, err) require.Equal(t, 5, len(newTp2.Meta().Partition.Definitions)) checkWaitingGCPartitionBundlesInPD(t, dom, checkOldPartitions) @@ -2036,7 +2036,7 @@ func TestDropTableWithPlacement(t *testing.T) { );`) defer tk.MustExec("drop table if exists tp") - tp, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("tp")) + tp, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("tp")) require.NoError(t, err) checkExistTableBundlesInPD(t, dom, "test", "tp") tk.MustExec("drop table tp") @@ -2082,10 +2082,10 @@ func TestDropPartitionWithPlacement(t *testing.T) { tk.MustExec("create placement policy p3 primary_region='r3' regions='r3'") defer tk.MustExec("drop placement policy p3") - policy1, ok := dom.InfoSchema().PolicyByName(pmodel.NewCIStr("p1")) + policy1, ok := dom.InfoSchema().PolicyByName(ast.NewCIStr("p1")) require.True(t, ok) - policy3, ok := dom.InfoSchema().PolicyByName(pmodel.NewCIStr("p3")) + policy3, ok := dom.InfoSchema().PolicyByName(ast.NewCIStr("p3")) require.True(t, ok) // test for partitioned table @@ -2097,7 +2097,7 @@ func TestDropPartitionWithPlacement(t *testing.T) { );`) defer tk.MustExec("drop table tp") - tp, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("tp")) + tp, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("tp")) require.NoError(t, err) checkOldPartitions := make([]model.PartitionDefinition, 0, 2) @@ -2112,7 +2112,7 @@ func TestDropPartitionWithPlacement(t *testing.T) { } tk.MustExec("ALTER TABLE tp DROP partition p1,p3") - newTp, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("tp")) + newTp, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("tp")) require.NoError(t, err) require.Equal(t, tp.Meta().ID, newTp.Meta().ID) require.Equal(t, policy1.ID, newTp.Meta().PlacementPolicyRef.ID) @@ -2126,7 +2126,7 @@ func TestDropPartitionWithPlacement(t *testing.T) { // add new partition will not override bundle waiting for GC tk.MustExec("alter table tp add partition (partition p4 values less than(1000000))") - newTp2, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("tp")) + newTp2, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("tp")) require.NoError(t, err) require.Equal(t, 3, len(newTp2.Meta().Partition.Definitions)) checkWaitingGCPartitionBundlesInPD(t, dom, checkOldPartitions) @@ -2160,14 +2160,14 @@ func TestExchangePartitionWithPlacement(t *testing.T) { tk.MustExec("create placement policy pp2 primary_region='r2' regions='r2'") tk.MustExec("create placement policy pp3 primary_region='r3' regions='r3'") - policy1, ok := dom.InfoSchema().PolicyByName(pmodel.NewCIStr("pp1")) + policy1, ok := dom.InfoSchema().PolicyByName(ast.NewCIStr("pp1")) require.True(t, ok) tk.MustExec(`CREATE TABLE t1 (id INT) placement policy pp1`) tk.MustExec(`CREATE TABLE t2 (id INT)`) tk.MustExec(`CREATE TABLE t3 (id INT) placement policy pp3`) - t1, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t1")) + t1, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t1")) require.NoError(t, err) t1ID := t1.Meta().ID @@ -2177,7 +2177,7 @@ func TestExchangePartitionWithPlacement(t *testing.T) { PARTITION p3 VALUES LESS THAN (10000) )`) - tp, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("tp")) + tp, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("tp")) require.NoError(t, err) tpID := tp.Meta().ID par0ID := tp.Meta().Partition.Definitions[0].ID @@ -2196,12 +2196,12 @@ func TestExchangePartitionWithPlacement(t *testing.T) { "(PARTITION `p1` VALUES LESS THAN (100) /*T![placement] PLACEMENT POLICY=`pp1` */,\n" + " PARTITION `p2` VALUES LESS THAN (1000) /*T![placement] PLACEMENT POLICY=`pp2` */,\n" + " PARTITION `p3` VALUES LESS THAN (10000))")) - tp, err = dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("tp")) + tp, err = dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("tp")) require.NoError(t, err) require.Equal(t, tpID, tp.Meta().ID) require.Equal(t, t1ID, tp.Meta().Partition.Definitions[0].ID) require.NotNil(t, tp.Meta().Partition.Definitions[0].PlacementPolicyRef) - t1, err = dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t1")) + t1, err = dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t1")) require.NoError(t, err) require.Equal(t, par0ID, t1.Meta().ID) require.Equal(t, policy1.ID, t1.Meta().PlacementPolicyRef.ID) diff --git a/pkg/ddl/placement_sql_test.go b/pkg/ddl/placement_sql_test.go index 522351c17b309..130998b7409ce 100644 --- a/pkg/ddl/placement_sql_test.go +++ b/pkg/ddl/placement_sql_test.go @@ -25,7 +25,7 @@ import ( "github.com/pingcap/tidb/pkg/domain/infosync" mysql "github.com/pingcap/tidb/pkg/errno" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/testkit" "github.com/stretchr/testify/require" @@ -56,7 +56,7 @@ PARTITION BY RANGE (c) ( is := dom.InfoSchema() - tb, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t1")) + tb, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t1")) require.NoError(t, err) partDefs := tb.Meta().GetPartitionInfo().Definitions @@ -188,7 +188,7 @@ func TestCreateSchemaWithPlacement(t *testing.T) { tk.MustQuery("SELECT TABLE_CATALOG, TABLE_SCHEMA, TABLE_NAME, TIDB_PLACEMENT_POLICY_NAME FROM information_schema.Tables WHERE TABLE_SCHEMA='SchemaPolicyPlacementTest' AND TABLE_NAME = 'UsePolicy'").Check(testkit.Rows(`def SchemaPolicyPlacementTest UsePolicy PolicyTableTest`)) is := dom.InfoSchema() - db, ok := is.SchemaByName(pmodel.NewCIStr("SchemaPolicyPlacementTest")) + db, ok := is.SchemaByName(ast.NewCIStr("SchemaPolicyPlacementTest")) require.True(t, ok) require.NotNil(t, db.PlacementPolicyRef) require.Equal(t, "PolicySchemaTest", db.PlacementPolicyRef.Name.O) @@ -311,7 +311,7 @@ func TestPlacementMode(t *testing.T) { defer tk.MustExec("drop table if exists t2") tk.MustQuery("show warnings").Check(testkit.Rows()) - existPolicy, ok := dom.InfoSchema().PolicyByName(pmodel.NewCIStr("p1")) + existPolicy, ok := dom.InfoSchema().PolicyByName(ast.NewCIStr("p1")) require.True(t, ok) // invalid values @@ -349,7 +349,7 @@ func TestPlacementMode(t *testing.T) { // create placement policy in ignore mode (policy name not exists) newPolicy = existPolicy.Clone() - newPolicy.Name = pmodel.NewCIStr("p3") + newPolicy.Name = ast.NewCIStr("p3") newPolicy.Followers = 8 tk.Session().SetValue(sessionctx.QueryString, "skip") err = dom.DDLExecutor().CreatePlacementPolicyWithInfo(tk.Session(), newPolicy, ddl.OnExistError) @@ -534,9 +534,9 @@ func TestPlacementMode(t *testing.T) { tbl, err := getClonedTableFromDomain("test", "t1", dom) require.NoError(t, err) require.NotNil(t, tbl.PlacementPolicyRef) - tbl.Name = pmodel.NewCIStr("t2") + tbl.Name = ast.NewCIStr("t2") tk.Session().SetValue(sessionctx.QueryString, "skip") - err = dom.DDLExecutor().CreateTableWithInfo(tk.Session(), pmodel.NewCIStr("test"), tbl, nil, ddl.WithOnExist(ddl.OnExistError)) + err = dom.DDLExecutor().CreateTableWithInfo(tk.Session(), ast.NewCIStr("test"), tbl, nil, ddl.WithOnExist(ddl.OnExistError)) require.NoError(t, err) tk.MustQuery("show create table t2").Check(testkit.Rows("t2 CREATE TABLE `t2` (\n" + " `id` int(11) DEFAULT NULL\n" + @@ -547,10 +547,10 @@ func TestPlacementMode(t *testing.T) { tbl, err = getClonedTableFromDomain("test", "t1", dom) require.NoError(t, err) require.NotNil(t, tbl.PlacementPolicyRef) - tbl.Name = pmodel.NewCIStr("t2") - tbl.PlacementPolicyRef.Name = pmodel.NewCIStr("pxx") + tbl.Name = ast.NewCIStr("t2") + tbl.PlacementPolicyRef.Name = ast.NewCIStr("pxx") tk.Session().SetValue(sessionctx.QueryString, "skip") - err = dom.DDLExecutor().CreateTableWithInfo(tk.Session(), pmodel.NewCIStr("test"), tbl, nil, ddl.WithOnExist(ddl.OnExistError)) + err = dom.DDLExecutor().CreateTableWithInfo(tk.Session(), ast.NewCIStr("test"), tbl, nil, ddl.WithOnExist(ddl.OnExistError)) require.NoError(t, err) tk.MustQuery("show create table t2").Check(testkit.Rows("t2 CREATE TABLE `t2` (\n" + " `id` int(11) DEFAULT NULL\n" + @@ -561,7 +561,7 @@ func TestPlacementMode(t *testing.T) { db1, ok := getClonedDatabaseFromDomain("db1", dom) require.True(t, ok) require.NotNil(t, db1.PlacementPolicyRef) - db1.Name = pmodel.NewCIStr("db2") + db1.Name = ast.NewCIStr("db2") tk.Session().SetValue(sessionctx.QueryString, "skip") err = dom.DDLExecutor().CreateSchemaWithInfo(tk.Session(), db1, ddl.OnExistError) require.NoError(t, err) @@ -572,8 +572,8 @@ func TestPlacementMode(t *testing.T) { db1, ok = getClonedDatabaseFromDomain("db1", dom) require.True(t, ok) require.NotNil(t, db1.PlacementPolicyRef) - db1.Name = pmodel.NewCIStr("db2") - db1.PlacementPolicyRef.Name = pmodel.NewCIStr("pxx") + db1.Name = ast.NewCIStr("db2") + db1.PlacementPolicyRef.Name = ast.NewCIStr("pxx") tk.Session().SetValue(sessionctx.QueryString, "skip") err = dom.DDLExecutor().CreateSchemaWithInfo(tk.Session(), db1, ddl.OnExistError) require.NoError(t, err) @@ -581,7 +581,7 @@ func TestPlacementMode(t *testing.T) { } func checkTiflashReplicaSet(t *testing.T, do *domain.Domain, db, tb string, cnt uint64) { - tbl, err := do.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr(db), pmodel.NewCIStr(tb)) + tbl, err := do.InfoSchema().TableByName(context.Background(), ast.NewCIStr(db), ast.NewCIStr(tb)) require.NoError(t, err) tiflashReplica := tbl.Meta().TiFlashReplica @@ -728,7 +728,7 @@ func TestPlacementTiflashCheck(t *testing.T) { } func getClonedTableFromDomain(dbName string, tableName string, dom *domain.Domain) (*model.TableInfo, error) { - tbl, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr(dbName), pmodel.NewCIStr(tableName)) + tbl, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr(dbName), ast.NewCIStr(tableName)) if err != nil { return nil, err } @@ -741,7 +741,7 @@ func getClonedTableFromDomain(dbName string, tableName string, dom *domain.Domai } func getClonedDatabaseFromDomain(dbName string, dom *domain.Domain) (*model.DBInfo, bool) { - db, ok := dom.InfoSchema().SchemaByName(pmodel.NewCIStr(dbName)) + db, ok := dom.InfoSchema().SchemaByName(ast.NewCIStr(dbName)) if !ok { return nil, ok } diff --git a/pkg/ddl/primary_key_handle_test.go b/pkg/ddl/primary_key_handle_test.go index ae3d9571f99cd..d86e0db9f9c08 100644 --- a/pkg/ddl/primary_key_handle_test.go +++ b/pkg/ddl/primary_key_handle_test.go @@ -24,7 +24,7 @@ import ( "github.com/pingcap/tidb/pkg/ddl" "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/kv" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/store/mockstore" "github.com/pingcap/tidb/pkg/table" @@ -75,7 +75,7 @@ func TestMultiRegionGetTableEndHandle(t *testing.T) { // Get table ID for split. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) // Split the table. @@ -100,7 +100,7 @@ func TestGetTableEndHandle(t *testing.T) { tk.MustExec("create table t(a bigint PRIMARY KEY, b int)") is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) // test empty table @@ -130,7 +130,7 @@ func TestGetTableEndHandle(t *testing.T) { tk.MustExec(sql[:len(sql)-1]) is = dom.InfoSchema() - tbl, err = is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t1")) + tbl, err = is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t1")) require.NoError(t, err) checkTableMaxHandle(t, tbl, store, false, kv.IntHandle(999)) @@ -138,7 +138,7 @@ func TestGetTableEndHandle(t *testing.T) { tk.MustExec("create table t2(a varchar(255))") is = dom.InfoSchema() - tbl, err = is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t2")) + tbl, err = is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t2")) require.NoError(t, err) checkTableMaxHandle(t, tbl, store, true, nil) @@ -202,7 +202,7 @@ func TestMultiRegionGetTableEndCommonHandle(t *testing.T) { // Get table ID for split. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) // Split the table. @@ -227,7 +227,7 @@ func TestGetTableEndCommonHandle(t *testing.T) { tk.MustExec("create table t1(a varchar(15), b bigint, c int, primary key (a(2), b))") is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) // test empty table @@ -243,7 +243,7 @@ func TestGetTableEndCommonHandle(t *testing.T) { checkTableMaxHandle(t, tbl, store, false, testutil.MustNewCommonHandle(t, "abchzzzzzzzz", 1)) // Test MaxTableRowID with prefixed primary key. - tbl, err = is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t1")) + tbl, err = is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t1")) require.NoError(t, err) checkTableMaxHandle(t, tbl, store, true, nil) tk.MustExec("insert into t1 values('abccccc', 1, 10)") @@ -262,44 +262,44 @@ func TestCreateClusteredIndex(t *testing.T) { tk.MustExec("CREATE TABLE t3 (a int, b int, c int, primary key (a, b))") tk.MustExec("CREATE TABLE t4 (a int, b int, c int)") is := domain.GetDomain(tk.Session()).InfoSchema() - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t1")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t1")) require.NoError(t, err) require.True(t, tbl.Meta().PKIsHandle) require.False(t, tbl.Meta().IsCommonHandle) - tbl, err = is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t2")) + tbl, err = is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t2")) require.NoError(t, err) require.True(t, tbl.Meta().IsCommonHandle) - tbl, err = is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t3")) + tbl, err = is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t3")) require.NoError(t, err) require.True(t, tbl.Meta().IsCommonHandle) - tbl, err = is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t4")) + tbl, err = is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t4")) require.NoError(t, err) require.False(t, tbl.Meta().IsCommonHandle) tk.MustExec("CREATE TABLE t5 (a varchar(255) primary key nonclustered, b int)") tk.MustExec("CREATE TABLE t6 (a int, b int, c int, primary key (a, b) nonclustered)") is = domain.GetDomain(tk.Session()).InfoSchema() - tbl, err = is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t5")) + tbl, err = is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t5")) require.NoError(t, err) require.False(t, tbl.Meta().IsCommonHandle) - tbl, err = is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t6")) + tbl, err = is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t6")) require.NoError(t, err) require.False(t, tbl.Meta().IsCommonHandle) tk.MustExec("CREATE TABLE t21 like t2") tk.MustExec("CREATE TABLE t31 like t3") is = domain.GetDomain(tk.Session()).InfoSchema() - tbl, err = is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t21")) + tbl, err = is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t21")) require.NoError(t, err) require.True(t, tbl.Meta().IsCommonHandle) - tbl, err = is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t31")) + tbl, err = is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t31")) require.NoError(t, err) require.True(t, tbl.Meta().IsCommonHandle) tk.Session().GetSessionVars().EnableClusteredIndex = variable.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(), model.NewCIStr("test"), model.NewCIStr("t7")) + tbl, err = is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t7")) require.NoError(t, err) require.False(t, tbl.Meta().IsCommonHandle) } diff --git a/pkg/ddl/resource_group.go b/pkg/ddl/resource_group.go index a1b0f87df1790..fdc48ab6e6f61 100644 --- a/pkg/ddl/resource_group.go +++ b/pkg/ddl/resource_group.go @@ -29,7 +29,6 @@ import ( "github.com/pingcap/tidb/pkg/meta" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/ast" - pmodel "github.com/pingcap/tidb/pkg/parser/model" rg "github.com/pingcap/tidb/pkg/resourcegroup" "github.com/pingcap/tidb/pkg/util/dbterror" kvutil "github.com/tikv/client-go/v2/util" @@ -268,7 +267,7 @@ func SetDirectResourceGroupRUSecondOption(resourceGroupSettings *model.ResourceG func SetDirectResourceGroupRunawayOption(resourceGroupSettings *model.ResourceGroupSettings, opt *ast.ResourceGroupRunawayOption) error { settings := resourceGroupSettings.Runaway switch opt.Tp { - case pmodel.RunawayRule: + case ast.RunawayRule: switch opt.RuleOption.Tp { case ast.RunawayRuleExecElapsed: // because execute time won't be too long, we use `time` pkg which does not support to parse unit 'd'. @@ -282,10 +281,10 @@ func SetDirectResourceGroupRunawayOption(resourceGroupSettings *model.ResourceGr case ast.RunawayRuleRequestUnit: settings.RequestUnit = opt.RuleOption.RequestUnit } - case pmodel.RunawayAction: + case ast.RunawayAction: settings.Action = opt.ActionOption.Type settings.SwitchGroupName = opt.ActionOption.SwitchGroupName.String() - case pmodel.RunawayWatch: + case ast.RunawayWatch: settings.WatchType = opt.WatchOption.Type if dur := opt.WatchOption.Duration; len(dur) > 0 { dur, err := time.ParseDuration(dur) diff --git a/pkg/ddl/resourcegroup/BUILD.bazel b/pkg/ddl/resourcegroup/BUILD.bazel index 90fc95e3aa38d..9eddc1cd19ca9 100644 --- a/pkg/ddl/resourcegroup/BUILD.bazel +++ b/pkg/ddl/resourcegroup/BUILD.bazel @@ -10,7 +10,7 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/meta/model", - "//pkg/parser/model", + "//pkg/parser/ast", "@com_github_pingcap_errors//:errors", "@com_github_pingcap_kvproto//pkg/resource_manager", ], diff --git a/pkg/ddl/resourcegroup/group.go b/pkg/ddl/resourcegroup/group.go index 7c1d0d69faf16..8f895fdc9093c 100644 --- a/pkg/ddl/resourcegroup/group.go +++ b/pkg/ddl/resourcegroup/group.go @@ -17,7 +17,7 @@ package resourcegroup import ( rmpb "github.com/pingcap/kvproto/pkg/resource_manager" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" ) // MaxGroupNameLength is max length of the name of a resource group @@ -49,17 +49,17 @@ func NewGroupFromOptions(groupName string, options *model.ResourceGroupSettings) runaway.Rule.ProcessedKeys = options.Runaway.ProcessedKeys runaway.Rule.RequestUnit = options.Runaway.RequestUnit // Update the action settings. - if options.Runaway.Action == pmodel.RunawayActionNone { + if options.Runaway.Action == ast.RunawayActionNone { return nil, ErrUnknownResourceGroupRunawayAction } runaway.Action = rmpb.RunawayAction(options.Runaway.Action) - if options.Runaway.Action == pmodel.RunawayActionSwitchGroup && len(options.Runaway.SwitchGroupName) == 0 { + if options.Runaway.Action == ast.RunawayActionSwitchGroup && len(options.Runaway.SwitchGroupName) == 0 { return nil, ErrUnknownResourceGroupRunawaySwitchGroupName } // TODO: validate the switch group name to ensure it exists. runaway.SwitchGroupName = options.Runaway.SwitchGroupName // Update the watch settings. - if options.Runaway.WatchType != pmodel.WatchNone { + if options.Runaway.WatchType != ast.WatchNone { runaway.Watch = &rmpb.RunawayWatch{} runaway.Watch.Type = rmpb.RunawayWatchType(options.Runaway.WatchType) runaway.Watch.LastingDurationMs = options.Runaway.WatchDurationMs diff --git a/pkg/ddl/schema_test.go b/pkg/ddl/schema_test.go index 7132fabe8c956..dad37e25115ed 100644 --- a/pkg/ddl/schema_test.go +++ b/pkg/ddl/schema_test.go @@ -28,7 +28,7 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/sessionctx" @@ -81,7 +81,7 @@ func testCheckTableState(t *testing.T, store kv.Storage, dbInfo *model.DBInfo, t // testTableInfo creates a test table with num int columns and with no index. func testTableInfo(store kv.Storage, name string, num int) (*model.TableInfo, error) { tblInfo := &model.TableInfo{ - Name: pmodel.NewCIStr(name), + Name: ast.NewCIStr(name), } genIDs, err := genGlobalIDs(store, 1) @@ -93,7 +93,7 @@ func testTableInfo(store kv.Storage, name string, num int) (*model.TableInfo, er cols := make([]*model.ColumnInfo, num) for i := range cols { col := &model.ColumnInfo{ - Name: pmodel.NewCIStr(fmt.Sprintf("c%d", i+1)), + Name: ast.NewCIStr(fmt.Sprintf("c%d", i+1)), Offset: i, DefaultValue: i + 1, State: model.StatePublic, @@ -124,7 +124,7 @@ func genGlobalIDs(store kv.Storage, count int) ([]int64, error) { func testSchemaInfo(store kv.Storage, name string) (*model.DBInfo, error) { dbInfo := &model.DBInfo{ - Name: pmodel.NewCIStr(name), + Name: ast.NewCIStr(name), } genIDs, err := genGlobalIDs(store, 1) diff --git a/pkg/ddl/schematracker/BUILD.bazel b/pkg/ddl/schematracker/BUILD.bazel index d2e5f5fdf8f47..5eda9f6bf3987 100644 --- a/pkg/ddl/schematracker/BUILD.bazel +++ b/pkg/ddl/schematracker/BUILD.bazel @@ -22,7 +22,6 @@ go_library( "//pkg/owner", "//pkg/parser/ast", "//pkg/parser/charset", - "//pkg/parser/model", "//pkg/parser/mysql", "//pkg/parser/types", "//pkg/sessionctx", @@ -56,7 +55,6 @@ go_test( "//pkg/meta/model", "//pkg/parser", "//pkg/parser/ast", - "//pkg/parser/model", "//pkg/planner/core/resolve", "//pkg/sessionctx", "//pkg/util/chunk", diff --git a/pkg/ddl/schematracker/checker.go b/pkg/ddl/schematracker/checker.go index 7068d4bce229b..936f874b4f831 100644 --- a/pkg/ddl/schematracker/checker.go +++ b/pkg/ddl/schematracker/checker.go @@ -33,7 +33,6 @@ import ( "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/owner" "github.com/pingcap/tidb/pkg/parser/ast" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/variable" @@ -89,7 +88,7 @@ func (d *Checker) CreateTestDB(ctx sessionctx.Context) { d.tracker.CreateTestDB(ctx) } -func (d *Checker) checkDBInfo(ctx sessionctx.Context, dbName pmodel.CIStr) { +func (d *Checker) checkDBInfo(ctx sessionctx.Context, dbName ast.CIStr) { if d.closed.Load() { return } @@ -122,7 +121,7 @@ func (d *Checker) checkDBInfo(ctx sessionctx.Context, dbName pmodel.CIStr) { } } -func (d *Checker) checkTableInfo(ctx sessionctx.Context, dbName, tableName pmodel.CIStr) { +func (d *Checker) checkTableInfo(ctx sessionctx.Context, dbName, tableName ast.CIStr) { if d.closed.Load() { return } @@ -484,13 +483,13 @@ func (d *Checker) CreateSchemaWithInfo(ctx sessionctx.Context, info *model.DBInf } // CreateTableWithInfo implements the DDL interface. -func (*Checker) CreateTableWithInfo(_ sessionctx.Context, _ pmodel.CIStr, _ *model.TableInfo, _ []model.InvolvingSchemaInfo, _ ...ddl.CreateTableOption) error { +func (*Checker) CreateTableWithInfo(_ sessionctx.Context, _ ast.CIStr, _ *model.TableInfo, _ []model.InvolvingSchemaInfo, _ ...ddl.CreateTableOption) error { //TODO implement me panic("implement me") } // BatchCreateTableWithInfo implements the DDL interface. -func (*Checker) BatchCreateTableWithInfo(_ sessionctx.Context, _ pmodel.CIStr, _ []*model.TableInfo, _ ...ddl.CreateTableOption) error { +func (*Checker) BatchCreateTableWithInfo(_ sessionctx.Context, _ ast.CIStr, _ []*model.TableInfo, _ ...ddl.CreateTableOption) error { //TODO implement me panic("implement me") } diff --git a/pkg/ddl/schematracker/dm_tracker.go b/pkg/ddl/schematracker/dm_tracker.go index b0e3d164cb724..e029e7495d2cf 100644 --- a/pkg/ddl/schematracker/dm_tracker.go +++ b/pkg/ddl/schematracker/dm_tracker.go @@ -29,7 +29,6 @@ import ( "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/charset" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" field_types "github.com/pingcap/tidb/pkg/parser/types" "github.com/pingcap/tidb/pkg/sessionctx" @@ -94,7 +93,7 @@ func (d *SchemaTracker) CreateSchema(ctx sessionctx.Context, stmt *ast.CreateDat // CreateTestDB creates the `test` database, which is the default behavior of TiDB. func (d *SchemaTracker) CreateTestDB(ctx sessionctx.Context) { _ = d.CreateSchema(ctx, &ast.CreateDatabaseStmt{ - Name: pmodel.NewCIStr("test"), + Name: ast.NewCIStr("test"), }) } @@ -232,7 +231,7 @@ func (d *SchemaTracker) CreateTable(ctx sessionctx.Context, s *ast.CreateTableSt // CreateTableWithInfo implements the DDL interface. func (d *SchemaTracker) CreateTableWithInfo( _ sessionctx.Context, - dbName pmodel.CIStr, + dbName ast.CIStr, info *model.TableInfo, _ []model.InvolvingSchemaInfo, cs ...ddl.CreateTableOption, @@ -365,11 +364,11 @@ func (d *SchemaTracker) DropView(_ sessionctx.Context, stmt *ast.DropTableStmt) // CreateIndex implements the DDL interface. func (d *SchemaTracker) CreateIndex(ctx sessionctx.Context, stmt *ast.CreateIndexStmt) error { ident := ast.Ident{Schema: stmt.Table.Schema, Name: stmt.Table.Name} - return d.createIndex(ctx, ident, stmt.KeyType, pmodel.NewCIStr(stmt.IndexName), + return d.createIndex(ctx, ident, stmt.KeyType, ast.NewCIStr(stmt.IndexName), stmt.IndexPartSpecifications, stmt.IndexOption, stmt.IfNotExists) } -func (d *SchemaTracker) putTableIfNoError(err error, dbName pmodel.CIStr, tbInfo *model.TableInfo) { +func (d *SchemaTracker) putTableIfNoError(err error, dbName ast.CIStr, tbInfo *model.TableInfo) { if err != nil { return } @@ -381,7 +380,7 @@ func (d *SchemaTracker) createIndex( ctx sessionctx.Context, ti ast.Ident, keyType ast.IndexKeyType, - indexName pmodel.CIStr, + indexName ast.CIStr, indexPartSpecifications []*ast.IndexPartSpecification, indexOption *ast.IndexOption, ifNotExists bool, @@ -398,11 +397,11 @@ func (d *SchemaTracker) createIndex( // Deal with anonymous index. if len(indexName.L) == 0 { - colName := pmodel.NewCIStr("expression_index") + colName := ast.NewCIStr("expression_index") if indexPartSpecifications[0].Column != nil { colName = indexPartSpecifications[0].Column.Name } - indexName = ddl.GetName4AnonymousIndex(t, colName, pmodel.NewCIStr("")) + indexName = ddl.GetName4AnonymousIndex(t, colName, ast.NewCIStr("")) } if indexInfo := tblInfo.FindIndexByName(indexName.L); indexInfo != nil { @@ -445,7 +444,7 @@ func (d *SchemaTracker) createIndex( // DropIndex implements the DDL interface. func (d *SchemaTracker) DropIndex(ctx sessionctx.Context, stmt *ast.DropIndexStmt) error { ti := ast.Ident{Schema: stmt.Table.Schema, Name: stmt.Table.Name} - err := d.dropIndex(ctx, ti, pmodel.NewCIStr(stmt.IndexName), stmt.IfExists) + err := d.dropIndex(ctx, ti, ast.NewCIStr(stmt.IndexName), stmt.IfExists) if (infoschema.ErrDatabaseNotExists.Equal(err) || infoschema.ErrTableNotExists.Equal(err)) && stmt.IfExists { err = nil } @@ -453,7 +452,7 @@ func (d *SchemaTracker) DropIndex(ctx sessionctx.Context, stmt *ast.DropIndexStm } // dropIndex is shared by DropIndex and AlterTable. -func (d *SchemaTracker) dropIndex(_ sessionctx.Context, ti ast.Ident, indexName pmodel.CIStr, ifExists bool) (err error) { +func (d *SchemaTracker) dropIndex(_ sessionctx.Context, ti ast.Ident, indexName ast.CIStr, ifExists bool) (err error) { tblInfo, err := d.TableClonedByName(ti.Schema, ti.Name) if err != nil { return infoschema.ErrTableNotExists.GenWithStackByArgs(ti.Schema, ti.Name) @@ -691,7 +690,7 @@ func (d *SchemaTracker) handleModifyColumn( ctx context.Context, sctx sessionctx.Context, ident ast.Ident, - originalColName pmodel.CIStr, + originalColName ast.CIStr, spec *ast.AlterTableSpec, ) (err error) { tblInfo, err := d.TableClonedByName(ident.Schema, ident.Name) @@ -835,7 +834,7 @@ func (d *SchemaTracker) dropTablePartitions(_ sessionctx.Context, ident ast.Iden func (d *SchemaTracker) createPrimaryKey( ctx sessionctx.Context, ti ast.Ident, - indexName pmodel.CIStr, + indexName ast.CIStr, indexPartSpecifications []*ast.IndexPartSpecification, indexOption *ast.IndexOption, ) (err error) { @@ -846,7 +845,7 @@ func (d *SchemaTracker) createPrimaryKey( defer d.putTableIfNoError(err, ti.Schema, tblInfo) - indexName = pmodel.NewCIStr(mysql.PrimaryKeyName) + indexName = ast.NewCIStr(mysql.PrimaryKeyName) if indexInfo := tblInfo.FindIndexByName(indexName.L); indexInfo != nil || // If the table's PKIsHandle is true, it also means that this table has a primary key. tblInfo.PKIsHandle { @@ -923,9 +922,9 @@ func (d *SchemaTracker) AlterTable(ctx context.Context, sctx sessionctx.Context, case ast.AlterTableDropColumn: err = d.dropColumn(sctx, ident, spec) case ast.AlterTableDropIndex: - err = d.dropIndex(sctx, ident, pmodel.NewCIStr(spec.Name), spec.IfExists) + err = d.dropIndex(sctx, ident, ast.NewCIStr(spec.Name), spec.IfExists) case ast.AlterTableDropPrimaryKey: - err = d.dropIndex(sctx, ident, pmodel.NewCIStr(mysql.PrimaryKeyName), spec.IfExists) + err = d.dropIndex(sctx, ident, ast.NewCIStr(mysql.PrimaryKeyName), spec.IfExists) case ast.AlterTableRenameIndex: err = d.renameIndex(sctx, ident, spec) case ast.AlterTableDropPartition: @@ -934,13 +933,13 @@ func (d *SchemaTracker) AlterTable(ctx context.Context, sctx sessionctx.Context, constr := spec.Constraint switch spec.Constraint.Tp { case ast.ConstraintKey, ast.ConstraintIndex: - err = d.createIndex(sctx, ident, ast.IndexKeyTypeNone, pmodel.NewCIStr(constr.Name), + err = d.createIndex(sctx, ident, ast.IndexKeyTypeNone, ast.NewCIStr(constr.Name), spec.Constraint.Keys, constr.Option, constr.IfNotExists) case ast.ConstraintUniq, ast.ConstraintUniqIndex, ast.ConstraintUniqKey: - err = d.createIndex(sctx, ident, ast.IndexKeyTypeUnique, pmodel.NewCIStr(constr.Name), + err = d.createIndex(sctx, ident, ast.IndexKeyTypeUnique, ast.NewCIStr(constr.Name), spec.Constraint.Keys, constr.Option, false) // IfNotExists should be not applied case ast.ConstraintPrimaryKey: - err = d.createPrimaryKey(sctx, ident, pmodel.NewCIStr(constr.Name), spec.Constraint.Keys, constr.Option) + err = d.createPrimaryKey(sctx, ident, ast.NewCIStr(constr.Name), spec.Constraint.Keys, constr.Option) case ast.ConstraintForeignKey, ast.ConstraintFulltext, ast.ConstraintCheck: @@ -1180,7 +1179,7 @@ func (*SchemaTracker) AlterResourceGroup(_ sessionctx.Context, _ *ast.AlterResou } // BatchCreateTableWithInfo implements the DDL interface, it will call CreateTableWithInfo for each table. -func (d *SchemaTracker) BatchCreateTableWithInfo(ctx sessionctx.Context, schema pmodel.CIStr, info []*model.TableInfo, cs ...ddl.CreateTableOption) error { +func (d *SchemaTracker) BatchCreateTableWithInfo(ctx sessionctx.Context, schema ast.CIStr, info []*model.TableInfo, cs ...ddl.CreateTableOption) error { for _, tableInfo := range info { if err := d.CreateTableWithInfo(ctx, schema, tableInfo, nil, cs...); err != nil { return err diff --git a/pkg/ddl/schematracker/dm_tracker_test.go b/pkg/ddl/schematracker/dm_tracker_test.go index 9ab0e28f90a67..f82961a0bd86e 100644 --- a/pkg/ddl/schematracker/dm_tracker_test.go +++ b/pkg/ddl/schematracker/dm_tracker_test.go @@ -31,7 +31,6 @@ import ( "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser" "github.com/pingcap/tidb/pkg/parser/ast" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/planner/core/resolve" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/util/chunk" @@ -105,7 +104,7 @@ func execAlter(t *testing.T, tracker schematracker.SchemaTracker, sql string) { } func mustTableByName(t *testing.T, tracker schematracker.SchemaTracker, schema, table string) *model.TableInfo { - tblInfo, err := tracker.TableByName(context.Background(), pmodel.NewCIStr(schema), pmodel.NewCIStr(table)) + tblInfo, err := tracker.TableByName(context.Background(), ast.NewCIStr(schema), ast.NewCIStr(table)) require.NoError(t, err) return tblInfo } @@ -202,7 +201,7 @@ func TestIndexLength(t *testing.T) { ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin" checkShowCreateTable(t, tblInfo, expected) - err := tracker.DeleteTable(pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + err := tracker.DeleteTable(ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) sql = "create table test.t(a text, b text charset ascii, c blob);" diff --git a/pkg/ddl/schematracker/info_store.go b/pkg/ddl/schematracker/info_store.go index 52454640b8af6..e1ee5efcf5c5f 100644 --- a/pkg/ddl/schematracker/info_store.go +++ b/pkg/ddl/schematracker/info_store.go @@ -19,7 +19,7 @@ import ( "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/table/tables" ) @@ -41,7 +41,7 @@ func NewInfoStore(lowerCaseTableNames int) *InfoStore { } } -func (i *InfoStore) ciStr2Key(name pmodel.CIStr) string { +func (i *InfoStore) ciStr2Key(name ast.CIStr) string { if i.lowerCaseTableNames == 0 { return name.O } @@ -49,7 +49,7 @@ func (i *InfoStore) ciStr2Key(name pmodel.CIStr) string { } // SchemaByName returns the DBInfo of given name. nil if not found. -func (i *InfoStore) SchemaByName(name pmodel.CIStr) *model.DBInfo { +func (i *InfoStore) SchemaByName(name ast.CIStr) *model.DBInfo { key := i.ciStr2Key(name) return i.dbs[key] } @@ -64,7 +64,7 @@ func (i *InfoStore) PutSchema(dbInfo *model.DBInfo) { } // DeleteSchema deletes the schema from InfoSchema. Returns true when the schema exists, false otherwise. -func (i *InfoStore) DeleteSchema(name pmodel.CIStr) bool { +func (i *InfoStore) DeleteSchema(name ast.CIStr) bool { key := i.ciStr2Key(name) _, ok := i.dbs[key] if !ok { @@ -76,7 +76,7 @@ func (i *InfoStore) DeleteSchema(name pmodel.CIStr) bool { } // TableByName returns the TableInfo. It will also return the error like an infoschema. -func (i *InfoStore) TableByName(_ context.Context, schema, table pmodel.CIStr) (*model.TableInfo, error) { +func (i *InfoStore) TableByName(_ context.Context, schema, table ast.CIStr) (*model.TableInfo, error) { schemaKey := i.ciStr2Key(schema) tables, ok := i.tables[schemaKey] if !ok { @@ -92,7 +92,7 @@ func (i *InfoStore) TableByName(_ context.Context, schema, table pmodel.CIStr) ( } // TableClonedByName is like TableByName, plus it will clone the TableInfo. -func (i *InfoStore) TableClonedByName(schema, table pmodel.CIStr) (*model.TableInfo, error) { +func (i *InfoStore) TableClonedByName(schema, table ast.CIStr) (*model.TableInfo, error) { tbl, err := i.TableByName(context.Background(), schema, table) if err != nil { return nil, err @@ -101,7 +101,7 @@ func (i *InfoStore) TableClonedByName(schema, table pmodel.CIStr) (*model.TableI } // PutTable puts a TableInfo, it will overwrite the old one. If the schema doesn't exist, it will return ErrDatabaseNotExists. -func (i *InfoStore) PutTable(schemaName pmodel.CIStr, tblInfo *model.TableInfo) error { +func (i *InfoStore) PutTable(schemaName ast.CIStr, tblInfo *model.TableInfo) error { schemaKey := i.ciStr2Key(schemaName) tables, ok := i.tables[schemaKey] if !ok { @@ -114,7 +114,7 @@ func (i *InfoStore) PutTable(schemaName pmodel.CIStr, tblInfo *model.TableInfo) // DeleteTable deletes the TableInfo, it will return ErrDatabaseNotExists or ErrTableNotExists when schema or table does // not exist. -func (i *InfoStore) DeleteTable(schema, table pmodel.CIStr) error { +func (i *InfoStore) DeleteTable(schema, table ast.CIStr) error { schemaKey := i.ciStr2Key(schema) tables, ok := i.tables[schemaKey] if !ok { @@ -140,7 +140,7 @@ func (i *InfoStore) AllSchemaNames() []string { } // AllTableNamesOfSchema return all table names of a schema. -func (i *InfoStore) AllTableNamesOfSchema(schema pmodel.CIStr) ([]string, error) { +func (i *InfoStore) AllTableNamesOfSchema(schema ast.CIStr) ([]string, error) { schemaKey := i.ciStr2Key(schema) tables, ok := i.tables[schemaKey] if !ok { @@ -161,19 +161,19 @@ type InfoStoreAdaptor struct { } // SchemaByName implements the InfoSchema interface. -func (i InfoStoreAdaptor) SchemaByName(schema pmodel.CIStr) (*model.DBInfo, bool) { +func (i InfoStoreAdaptor) SchemaByName(schema ast.CIStr) (*model.DBInfo, bool) { dbInfo := i.inner.SchemaByName(schema) return dbInfo, dbInfo != nil } // TableExists implements the InfoSchema interface. -func (i InfoStoreAdaptor) TableExists(schema, table pmodel.CIStr) bool { +func (i InfoStoreAdaptor) TableExists(schema, table ast.CIStr) bool { tableInfo, _ := i.inner.TableByName(context.Background(), schema, table) return tableInfo != nil } // TableByName implements the InfoSchema interface. -func (i InfoStoreAdaptor) TableByName(ctx context.Context, schema, table pmodel.CIStr) (t table.Table, err error) { +func (i InfoStoreAdaptor) TableByName(ctx context.Context, schema, table ast.CIStr) (t table.Table, err error) { tableInfo, err := i.inner.TableByName(ctx, schema, table) if err != nil { return nil, err @@ -182,6 +182,6 @@ func (i InfoStoreAdaptor) TableByName(ctx context.Context, schema, table pmodel. } // TableInfoByName implements the InfoSchema interface. -func (i InfoStoreAdaptor) TableInfoByName(schema, table pmodel.CIStr) (*model.TableInfo, error) { +func (i InfoStoreAdaptor) TableInfoByName(schema, table ast.CIStr) (*model.TableInfo, error) { return i.inner.TableByName(context.Background(), schema, table) } diff --git a/pkg/ddl/schematracker/info_store_test.go b/pkg/ddl/schematracker/info_store_test.go index 7a8be23f21cd6..a6edb120e9ef7 100644 --- a/pkg/ddl/schematracker/info_store_test.go +++ b/pkg/ddl/schematracker/info_store_test.go @@ -21,15 +21,15 @@ import ( "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/stretchr/testify/require" ) func TestInfoStoreLowerCaseTableNames(t *testing.T) { - dbName := pmodel.NewCIStr("DBName") - lowerDBName := pmodel.NewCIStr("dbname") - tableName := pmodel.NewCIStr("TableName") - lowerTableName := pmodel.NewCIStr("tablename") + dbName := ast.NewCIStr("DBName") + lowerDBName := ast.NewCIStr("dbname") + tableName := ast.NewCIStr("TableName") + lowerTableName := ast.NewCIStr("tablename") dbInfo := &model.DBInfo{Name: dbName} tableInfo := &model.TableInfo{Name: tableName} @@ -58,7 +58,7 @@ func TestInfoStoreLowerCaseTableNames(t *testing.T) { schemaNames := is.AllSchemaNames() require.Equal(t, []string{dbName.O}, schemaNames) - _, err = is.AllTableNamesOfSchema(pmodel.NewCIStr("wrong-db")) + _, err = is.AllTableNamesOfSchema(ast.NewCIStr("wrong-db")) require.Error(t, err) tableNames, err := is.AllTableNamesOfSchema(dbName) require.NoError(t, err) @@ -86,7 +86,7 @@ func TestInfoStoreLowerCaseTableNames(t *testing.T) { schemaNames = is.AllSchemaNames() require.Equal(t, []string{dbName.L}, schemaNames) - _, err = is.AllTableNamesOfSchema(pmodel.NewCIStr("wrong-db")) + _, err = is.AllTableNamesOfSchema(ast.NewCIStr("wrong-db")) require.Error(t, err) tableNames, err = is.AllTableNamesOfSchema(dbName) require.NoError(t, err) @@ -95,10 +95,10 @@ func TestInfoStoreLowerCaseTableNames(t *testing.T) { func TestInfoStoreDeleteTables(t *testing.T) { is := NewInfoStore(0) - dbName1 := pmodel.NewCIStr("DBName1") - dbName2 := pmodel.NewCIStr("DBName2") - tableName1 := pmodel.NewCIStr("TableName1") - tableName2 := pmodel.NewCIStr("TableName2") + dbName1 := ast.NewCIStr("DBName1") + dbName2 := ast.NewCIStr("DBName2") + tableName1 := ast.NewCIStr("TableName1") + tableName2 := ast.NewCIStr("TableName2") dbInfo1 := &model.DBInfo{Name: dbName1} dbInfo2 := &model.DBInfo{Name: dbName2} tableInfo1 := &model.TableInfo{Name: tableName1} diff --git a/pkg/ddl/table.go b/pkg/ddl/table.go index e4306fa437193..1b00f53862631 100644 --- a/pkg/ddl/table.go +++ b/pkg/ddl/table.go @@ -33,8 +33,8 @@ import ( "github.com/pingcap/tidb/pkg/meta" "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/parser/charset" - pmodel "github.com/pingcap/tidb/pkg/parser/model" field_types "github.com/pingcap/tidb/pkg/parser/types" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/table" @@ -896,7 +896,7 @@ func checkAndRenameTables(t *meta.Mutator, job *model.Job, tblInfo *model.TableI func adjustForeignKeyChildTableInfoAfterRenameTable( infoCache *infoschema.InfoCache, t *meta.Mutator, job *model.Job, fkh *foreignKeyHelper, tblInfo *model.TableInfo, - oldSchemaName, oldTableName, newTableName pmodel.CIStr, newSchemaID int64) error { + oldSchemaName, oldTableName, newTableName ast.CIStr, newSchemaID int64) error { if !variable.EnableForeignKey.Load() || newTableName.L == oldTableName.L { return nil } @@ -1277,7 +1277,7 @@ func checkTableNotExistsFromInfoSchema(is infoschema.InfoSchema, schemaID int64, if !ok { return infoschema.ErrDatabaseNotExists.GenWithStackByArgs("") } - if is.TableExists(schema.Name, pmodel.NewCIStr(tableName)) { + if is.TableExists(schema.Name, ast.NewCIStr(tableName)) { return infoschema.ErrTableExists.GenWithStackByArgs(tableName) } return nil diff --git a/pkg/ddl/table_lock.go b/pkg/ddl/table_lock.go index f056c2c1abdcc..e1ebf1ad5f7a5 100644 --- a/pkg/ddl/table_lock.go +++ b/pkg/ddl/table_lock.go @@ -18,7 +18,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/util/dbterror" ) @@ -125,8 +125,8 @@ func lockTable(tbInfo *model.TableInfo, idx int, args *model.LockTablesArgs) err if tbInfo.Lock.State == model.TableLockStatePreLock { return nil } - if (tbInfo.Lock.Tp == pmodel.TableLockRead && args.LockTables[idx].Tp == pmodel.TableLockRead) || - (tbInfo.Lock.Tp == pmodel.TableLockReadOnly && args.LockTables[idx].Tp == pmodel.TableLockReadOnly) { + if (tbInfo.Lock.Tp == ast.TableLockRead && args.LockTables[idx].Tp == ast.TableLockRead) || + (tbInfo.Lock.Tp == ast.TableLockReadOnly && args.LockTables[idx].Tp == ast.TableLockReadOnly) { sessionIndex := findSessionInfoIndex(tbInfo.Lock.Sessions, args.SessionInfo) // repeat lock. if sessionIndex >= 0 { @@ -142,15 +142,15 @@ func lockTable(tbInfo *model.TableInfo, idx int, args *model.LockTablesArgs) err } // checkTableLocked uses to check whether table was locked. -func checkTableLocked(tbInfo *model.TableInfo, lockTp pmodel.TableLockType, sessionInfo model.SessionInfo) error { +func checkTableLocked(tbInfo *model.TableInfo, lockTp ast.TableLockType, sessionInfo model.SessionInfo) error { if !tbInfo.IsLocked() { return nil } if tbInfo.Lock.State == model.TableLockStatePreLock { return nil } - if (tbInfo.Lock.Tp == pmodel.TableLockRead && lockTp == pmodel.TableLockRead) || - (tbInfo.Lock.Tp == pmodel.TableLockReadOnly && lockTp == pmodel.TableLockReadOnly) { + if (tbInfo.Lock.Tp == ast.TableLockRead && lockTp == ast.TableLockRead) || + (tbInfo.Lock.Tp == ast.TableLockReadOnly && lockTp == ast.TableLockReadOnly) { return nil } sessionIndex := findSessionInfoIndex(tbInfo.Lock.Sessions, sessionInfo) @@ -160,7 +160,7 @@ func checkTableLocked(tbInfo *model.TableInfo, lockTp pmodel.TableLockType, sess return nil } // If no other session locked this table, and it is not a read only lock (session unrelated). - if len(tbInfo.Lock.Sessions) == 1 && tbInfo.Lock.Tp != pmodel.TableLockReadOnly { + if len(tbInfo.Lock.Sessions) == 1 && tbInfo.Lock.Tp != ast.TableLockReadOnly { return nil } } diff --git a/pkg/ddl/table_split_test.go b/pkg/ddl/table_split_test.go index b47b6939988f0..d54f01103db4e 100644 --- a/pkg/ddl/table_split_test.go +++ b/pkg/ddl/table_split_test.go @@ -21,7 +21,7 @@ import ( "time" "github.com/pingcap/tidb/pkg/ddl" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/session" "github.com/pingcap/tidb/pkg/store/mockstore" "github.com/pingcap/tidb/pkg/tablecodec" @@ -62,18 +62,18 @@ func TestTableSplit(t *testing.T) { atomic.StoreUint32(&ddl.EnableSplitTableRegion, 0) infoSchema := dom.InfoSchema() require.NotNil(t, infoSchema) - tbl, err := infoSchema.TableByName(context.Background(), model.NewCIStr("mysql"), model.NewCIStr("tidb")) + tbl, err := infoSchema.TableByName(context.Background(), ast.NewCIStr("mysql"), ast.NewCIStr("tidb")) require.NoError(t, err) checkRegionStartWithTableID(t, tbl.Meta().ID, store.(kvStore)) - tbl, err = infoSchema.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t_part")) + tbl, err = infoSchema.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t_part")) require.NoError(t, err) pi := tbl.Meta().GetPartitionInfo() require.NotNil(t, pi) for _, def := range pi.Definitions { checkRegionStartWithTableID(t, def.ID, store.(kvStore)) } - tbl, err = infoSchema.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t_part_2")) + tbl, err = infoSchema.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t_part_2")) require.NoError(t, err) pi = tbl.Meta().GetPartitionInfo() require.NotNil(t, pi) diff --git a/pkg/ddl/table_test.go b/pkg/ddl/table_test.go index 0e94d623f51db..47cfb2c73e883 100644 --- a/pkg/ddl/table_test.go +++ b/pkg/ddl/table_test.go @@ -30,7 +30,6 @@ import ( "github.com/pingcap/tidb/pkg/meta/autoid" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/ast" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/table" @@ -47,8 +46,8 @@ func testRenameTable( ctx sessionctx.Context, d ddl.ExecutorForTest, newSchemaID, oldSchemaID int64, - oldSchemaName pmodel.CIStr, - newSchemaName pmodel.CIStr, + oldSchemaName ast.CIStr, + newSchemaName ast.CIStr, tblInfo *model.TableInfo, ) *model.Job { job := &model.Job{ @@ -81,8 +80,8 @@ func testRenameTable( func testRenameTables(t *testing.T, ctx sessionctx.Context, d ddl.ExecutorForTest, oldSchemaIDs, newSchemaIDs []int64, - newTableNames []pmodel.CIStr, oldTableIDs []int64, - oldSchemaNames, oldTableNames []pmodel.CIStr) *model.Job { + newTableNames []ast.CIStr, oldTableIDs []int64, + oldSchemaNames, oldTableNames []ast.CIStr) *model.Job { job := &model.Job{ Version: model.GetJobVerInUse(), Type: model.ActionRenameTables, @@ -114,9 +113,9 @@ func testLockTable( d ddl.ExecutorForTest, uuid string, newSchemaID int64, - schemaName pmodel.CIStr, + schemaName ast.CIStr, tblInfo *model.TableInfo, - lockTp pmodel.TableLockType, + lockTp ast.TableLockType, ) *model.Job { args := &model.LockTablesArgs{ LockTables: []model.TableLockTpInfo{{SchemaID: newSchemaID, TableID: tblInfo.ID, Tp: lockTp}}, @@ -144,7 +143,7 @@ func testLockTable( return job } -func checkTableLockedTest(t *testing.T, store kv.Storage, dbInfo *model.DBInfo, tblInfo *model.TableInfo, serverID string, sessionID uint64, lockTp pmodel.TableLockType) { +func checkTableLockedTest(t *testing.T, store kv.Storage, dbInfo *model.DBInfo, tblInfo *model.TableInfo, serverID string, sessionID uint64, lockTp ast.TableLockType) { ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnDDL) err := kv.RunInNewTxn(ctx, store, false, func(ctx context.Context, txn kv.Transaction) error { tt := meta.NewMutator(txn) @@ -271,10 +270,10 @@ func TestTable(t *testing.T) { testCheckTableState(t, store, dbInfo1, tblInfo, model.StatePublic) testCheckJobDone(t, store, job.ID, true) - job = testLockTable(t, ctx, de, d.GetID(), dbInfo1.ID, dbInfo1.Name, tblInfo, pmodel.TableLockWrite) + job = testLockTable(t, ctx, de, d.GetID(), dbInfo1.ID, dbInfo1.Name, tblInfo, ast.TableLockWrite) testCheckTableState(t, store, dbInfo1, tblInfo, model.StatePublic) testCheckJobDone(t, store, job.ID, true) - checkTableLockedTest(t, store, dbInfo1, tblInfo, d.GetID(), ctx.GetSessionVars().ConnectionID, pmodel.TableLockWrite) + checkTableLockedTest(t, store, dbInfo1, tblInfo, d.GetID(), ctx.GetSessionVars().ConnectionID, ast.TableLockWrite) // for alter cache table job = testAlterCacheTable(t, ctx, de, dbInfo1.ID, dbInfo1.Name, tblInfo) testCheckTableState(t, store, dbInfo1, tblInfo, model.StatePublic) @@ -402,7 +401,7 @@ func testAlterCacheTable( ctx sessionctx.Context, d ddl.ExecutorForTest, newSchemaID int64, - newSchemaName pmodel.CIStr, + newSchemaName ast.CIStr, tblInfo *model.TableInfo, ) *model.Job { job := &model.Job{ @@ -429,7 +428,7 @@ func testAlterNoCacheTable( ctx sessionctx.Context, d ddl.ExecutorForTest, newSchemaID int64, - newSchemaName pmodel.CIStr, + newSchemaName ast.CIStr, tblInfo *model.TableInfo, ) *model.Job { job := &model.Job{ @@ -478,10 +477,10 @@ func TestRenameTables(t *testing.T) { job := testRenameTables(t, ctx, de, []int64{dbInfo.ID, dbInfo.ID}, []int64{dbInfo.ID, dbInfo.ID}, - []pmodel.CIStr{newTblInfos[0].Name, newTblInfos[1].Name}, + []ast.CIStr{newTblInfos[0].Name, newTblInfos[1].Name}, []int64{tblInfos[0].ID, tblInfos[1].ID}, - []pmodel.CIStr{dbInfo.Name, dbInfo.Name}, - []pmodel.CIStr{tblInfos[0].Name, tblInfos[1].Name}) + []ast.CIStr{dbInfo.Name, dbInfo.Name}, + []ast.CIStr{tblInfos[0].Name, tblInfos[1].Name}) historyJob, err := ddl.GetHistoryJobByID(testkit.NewTestKit(t, store).Session(), job.ID) require.NoError(t, err) @@ -510,7 +509,7 @@ func TestCreateTables(t *testing.T) { args.Tables = append(args.Tables, &model.CreateTableArgs{ TableInfo: &model.TableInfo{ ID: genIDs[i], - Name: pmodel.NewCIStr(fmt.Sprintf("s%d", i+1)), + Name: ast.NewCIStr(fmt.Sprintf("s%d", i+1)), }, }) } @@ -779,7 +778,7 @@ func TestDropTableAccessibleInInfoSchema(t *testing.T) { testfailpoint.EnableCall(t, "github.com/pingcap/tidb/pkg/ddl/beforeRunOneJobStep", func(job *model.Job) { if job.Type == model.ActionDropTable && job.TableName == "t" { if job.SchemaState == model.StateDeleteOnly || job.SchemaState == model.StateWriteOnly { - _, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + _, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) errs = append(errs, err) } } diff --git a/pkg/ddl/tests/fail/BUILD.bazel b/pkg/ddl/tests/fail/BUILD.bazel index ffd6263510607..f62a676b23f49 100644 --- a/pkg/ddl/tests/fail/BUILD.bazel +++ b/pkg/ddl/tests/fail/BUILD.bazel @@ -19,7 +19,7 @@ go_test( "//pkg/ddl/util", "//pkg/domain", "//pkg/kv", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/session", "//pkg/sessionctx/variable", "//pkg/store/mockstore", diff --git a/pkg/ddl/tests/fail/fail_db_test.go b/pkg/ddl/tests/fail/fail_db_test.go index a8dc18190451b..dd1265e3e3c7b 100644 --- a/pkg/ddl/tests/fail/fail_db_test.go +++ b/pkg/ddl/tests/fail/fail_db_test.go @@ -30,7 +30,7 @@ import ( ddlutil "github.com/pingcap/tidb/pkg/ddl/util" "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/kv" - "github.com/pingcap/tidb/pkg/parser/model" + "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/store/mockstore" @@ -209,7 +209,7 @@ func TestAddIndexFailed(t *testing.T) { // Get table ID for split. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test_add_index_failed"), model.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test_add_index_failed"), ast.NewCIStr("t")) require.NoError(t, err) tblID := tbl.Meta().ID @@ -376,8 +376,8 @@ func testAddIndexWorkerNum(t *testing.T, s *failedSuite, test func(*testkit.Test } is := s.dom.InfoSchema() - schemaName := model.NewCIStr("test_db") - tableName := model.NewCIStr("test_add_index") + schemaName := ast.NewCIStr("test_db") + tableName := ast.NewCIStr("test_add_index") tbl, err := is.TableByName(context.Background(), schemaName, tableName) require.NoError(t, err) @@ -478,7 +478,7 @@ func TestModifyColumn(t *testing.T) { tk.MustExec("alter table t change column b bb mediumint first") is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) cols := tbl.Meta().Columns colsStr := "" diff --git a/pkg/ddl/tests/fk/BUILD.bazel b/pkg/ddl/tests/fk/BUILD.bazel index de6741571cc12..cb7d650c4a079 100644 --- a/pkg/ddl/tests/fk/BUILD.bazel +++ b/pkg/ddl/tests/fk/BUILD.bazel @@ -17,8 +17,8 @@ go_test( "//pkg/meta", "//pkg/meta/autoid", "//pkg/meta/model", + "//pkg/parser/ast", "//pkg/parser/auth", - "//pkg/parser/model", "//pkg/sessiontxn", "//pkg/testkit", "//pkg/testkit/testsetup", diff --git a/pkg/ddl/tests/fk/foreign_key_test.go b/pkg/ddl/tests/fk/foreign_key_test.go index 4ad788540c35f..fe29c03414b0d 100644 --- a/pkg/ddl/tests/fk/foreign_key_test.go +++ b/pkg/ddl/tests/fk/foreign_key_test.go @@ -26,8 +26,8 @@ import ( "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/meta" "github.com/pingcap/tidb/pkg/meta/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/auth" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/sessiontxn" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/util/dbterror" @@ -52,21 +52,21 @@ func TestCreateTableWithForeignKeyMetaInfo(t *testing.T) { tb1ReferredFKs := getTableInfoReferredForeignKeys(t, dom, "test", "t1") require.Equal(t, 1, len(tb1ReferredFKs)) require.Equal(t, model.ReferredFKInfo{ - Cols: []pmodel.CIStr{pmodel.NewCIStr("id")}, - ChildSchema: pmodel.NewCIStr("test2"), - ChildTable: pmodel.NewCIStr("t2"), - ChildFKName: pmodel.NewCIStr("fk_b"), + Cols: []ast.CIStr{ast.NewCIStr("id")}, + ChildSchema: ast.NewCIStr("test2"), + ChildTable: ast.NewCIStr("t2"), + ChildFKName: ast.NewCIStr("fk_b"), }, *tb1ReferredFKs[0]) tb2ReferredFKs := getTableInfoReferredForeignKeys(t, dom, "test2", "t2") require.Equal(t, 0, len(tb2ReferredFKs)) require.Equal(t, 1, len(tb2Info.ForeignKeys)) require.Equal(t, model.FKInfo{ ID: 1, - Name: pmodel.NewCIStr("fk_b"), - RefSchema: pmodel.NewCIStr("test"), - RefTable: pmodel.NewCIStr("t1"), - RefCols: []pmodel.CIStr{pmodel.NewCIStr("id")}, - Cols: []pmodel.CIStr{pmodel.NewCIStr("b")}, + Name: ast.NewCIStr("fk_b"), + RefSchema: ast.NewCIStr("test"), + RefTable: ast.NewCIStr("t1"), + RefCols: []ast.CIStr{ast.NewCIStr("id")}, + Cols: []ast.CIStr{ast.NewCIStr("b")}, OnDelete: 2, OnUpdate: 1, State: model.StatePublic, @@ -86,21 +86,21 @@ func TestCreateTableWithForeignKeyMetaInfo(t *testing.T) { tb2ReferredFKs = getTableInfoReferredForeignKeys(t, dom, "test2", "t2") require.Equal(t, 1, len(tb2ReferredFKs)) require.Equal(t, model.ReferredFKInfo{ - Cols: []pmodel.CIStr{pmodel.NewCIStr("id")}, - ChildSchema: pmodel.NewCIStr("test2"), - ChildTable: pmodel.NewCIStr("t3"), - ChildFKName: pmodel.NewCIStr("fk_b"), + Cols: []ast.CIStr{ast.NewCIStr("id")}, + ChildSchema: ast.NewCIStr("test2"), + ChildTable: ast.NewCIStr("t3"), + ChildFKName: ast.NewCIStr("fk_b"), }, *tb2ReferredFKs[0]) tb3ReferredFKs := getTableInfoReferredForeignKeys(t, dom, "test2", "t3") require.Equal(t, 0, len(tb3ReferredFKs)) require.Equal(t, 1, len(tb3Info.ForeignKeys)) require.Equal(t, model.FKInfo{ ID: 1, - Name: pmodel.NewCIStr("fk_b"), - RefSchema: pmodel.NewCIStr("test2"), - RefTable: pmodel.NewCIStr("t2"), - RefCols: []pmodel.CIStr{pmodel.NewCIStr("id")}, - Cols: []pmodel.CIStr{pmodel.NewCIStr("b")}, + Name: ast.NewCIStr("fk_b"), + RefSchema: ast.NewCIStr("test2"), + RefTable: ast.NewCIStr("t2"), + RefCols: []ast.CIStr{ast.NewCIStr("id")}, + Cols: []ast.CIStr{ast.NewCIStr("b")}, OnDelete: 4, OnUpdate: 3, State: model.StatePublic, @@ -117,18 +117,18 @@ func TestCreateTableWithForeignKeyMetaInfo(t *testing.T) { tb5ReferredFKs := getTableInfoReferredForeignKeys(t, dom, "test2", "t5") require.Equal(t, 1, len(tb5ReferredFKs)) require.Equal(t, model.ReferredFKInfo{ - Cols: []pmodel.CIStr{pmodel.NewCIStr("id")}, - ChildSchema: pmodel.NewCIStr("test2"), - ChildTable: pmodel.NewCIStr("t5"), - ChildFKName: pmodel.NewCIStr("fk_1"), + Cols: []ast.CIStr{ast.NewCIStr("id")}, + ChildSchema: ast.NewCIStr("test2"), + ChildTable: ast.NewCIStr("t5"), + ChildFKName: ast.NewCIStr("fk_1"), }, *tb5ReferredFKs[0]) require.Equal(t, model.FKInfo{ ID: 1, - Name: pmodel.NewCIStr("fk_1"), - RefSchema: pmodel.NewCIStr("test2"), - RefTable: pmodel.NewCIStr("t5"), - RefCols: []pmodel.CIStr{pmodel.NewCIStr("id")}, - Cols: []pmodel.CIStr{pmodel.NewCIStr("a")}, + Name: ast.NewCIStr("fk_1"), + RefSchema: ast.NewCIStr("test2"), + RefTable: ast.NewCIStr("t5"), + RefCols: []ast.CIStr{ast.NewCIStr("id")}, + Cols: []ast.CIStr{ast.NewCIStr("a")}, State: model.StatePublic, Version: 1, }, *tb5Info.ForeignKeys[0]) @@ -162,21 +162,21 @@ func TestCreateTableWithForeignKeyMetaInfo2(t *testing.T) { tb1ReferredFKs := getTableInfoReferredForeignKeys(t, dom, "test", "t1") require.Equal(t, 1, len(tb1ReferredFKs)) require.Equal(t, model.ReferredFKInfo{ - Cols: []pmodel.CIStr{pmodel.NewCIStr("id")}, - ChildSchema: pmodel.NewCIStr("test2"), - ChildTable: pmodel.NewCIStr("t2"), - ChildFKName: pmodel.NewCIStr("fk_b"), + Cols: []ast.CIStr{ast.NewCIStr("id")}, + ChildSchema: ast.NewCIStr("test2"), + ChildTable: ast.NewCIStr("t2"), + ChildFKName: ast.NewCIStr("fk_b"), }, *tb1ReferredFKs[0]) tb2ReferredFKs := getTableInfoReferredForeignKeys(t, dom, "test2", "t2") require.Equal(t, 0, len(tb2ReferredFKs)) require.Equal(t, 1, len(tb2Info.ForeignKeys)) require.Equal(t, model.FKInfo{ ID: 1, - Name: pmodel.NewCIStr("fk_b"), - RefSchema: pmodel.NewCIStr("test"), - RefTable: pmodel.NewCIStr("t1"), - RefCols: []pmodel.CIStr{pmodel.NewCIStr("id")}, - Cols: []pmodel.CIStr{pmodel.NewCIStr("b")}, + Name: ast.NewCIStr("fk_b"), + RefSchema: ast.NewCIStr("test"), + RefTable: ast.NewCIStr("t1"), + RefCols: []ast.CIStr{ast.NewCIStr("id")}, + Cols: []ast.CIStr{ast.NewCIStr("b")}, OnDelete: 2, OnUpdate: 1, State: model.StatePublic, @@ -194,27 +194,27 @@ func TestCreateTableWithForeignKeyMetaInfo2(t *testing.T) { tb1ReferredFKs = getTableInfoReferredForeignKeys(t, dom, "test", "t1") require.Equal(t, 2, len(tb1ReferredFKs)) require.Equal(t, model.ReferredFKInfo{ - Cols: []pmodel.CIStr{pmodel.NewCIStr("id")}, - ChildSchema: pmodel.NewCIStr("test"), - ChildTable: pmodel.NewCIStr("t3"), - ChildFKName: pmodel.NewCIStr("fk_a"), + Cols: []ast.CIStr{ast.NewCIStr("id")}, + ChildSchema: ast.NewCIStr("test"), + ChildTable: ast.NewCIStr("t3"), + ChildFKName: ast.NewCIStr("fk_a"), }, *tb1ReferredFKs[0]) require.Equal(t, model.ReferredFKInfo{ - Cols: []pmodel.CIStr{pmodel.NewCIStr("id")}, - ChildSchema: pmodel.NewCIStr("test2"), - ChildTable: pmodel.NewCIStr("t2"), - ChildFKName: pmodel.NewCIStr("fk_b"), + Cols: []ast.CIStr{ast.NewCIStr("id")}, + ChildSchema: ast.NewCIStr("test2"), + ChildTable: ast.NewCIStr("t2"), + ChildFKName: ast.NewCIStr("fk_b"), }, *tb1ReferredFKs[1]) tb3ReferredFKs := getTableInfoReferredForeignKeys(t, dom, "test", "t3") require.Equal(t, 0, len(tb3ReferredFKs)) require.Equal(t, 2, len(tb3Info.ForeignKeys)) require.Equal(t, model.FKInfo{ ID: 1, - Name: pmodel.NewCIStr("fk_a"), - RefSchema: pmodel.NewCIStr("test"), - RefTable: pmodel.NewCIStr("t1"), - RefCols: []pmodel.CIStr{pmodel.NewCIStr("id")}, - Cols: []pmodel.CIStr{pmodel.NewCIStr("a")}, + Name: ast.NewCIStr("fk_a"), + RefSchema: ast.NewCIStr("test"), + RefTable: ast.NewCIStr("t1"), + RefCols: []ast.CIStr{ast.NewCIStr("id")}, + Cols: []ast.CIStr{ast.NewCIStr("a")}, OnDelete: 2, OnUpdate: 1, State: model.StatePublic, @@ -222,11 +222,11 @@ func TestCreateTableWithForeignKeyMetaInfo2(t *testing.T) { }, *tb3Info.ForeignKeys[0]) require.Equal(t, model.FKInfo{ ID: 2, - Name: pmodel.NewCIStr("fk_a2"), - RefSchema: pmodel.NewCIStr("test2"), - RefTable: pmodel.NewCIStr("t2"), - RefCols: []pmodel.CIStr{pmodel.NewCIStr("id")}, - Cols: []pmodel.CIStr{pmodel.NewCIStr("a")}, + Name: ast.NewCIStr("fk_a2"), + RefSchema: ast.NewCIStr("test2"), + RefTable: ast.NewCIStr("t2"), + RefCols: []ast.CIStr{ast.NewCIStr("id")}, + Cols: []ast.CIStr{ast.NewCIStr("a")}, State: model.StatePublic, Version: 1, }, *tb3Info.ForeignKeys[1]) @@ -244,21 +244,21 @@ func TestCreateTableWithForeignKeyMetaInfo2(t *testing.T) { tb1ReferredFKs = getTableInfoReferredForeignKeys(t, dom, "test", "t1") require.Equal(t, 1, len(tb1ReferredFKs)) require.Equal(t, model.ReferredFKInfo{ - Cols: []pmodel.CIStr{pmodel.NewCIStr("id")}, - ChildSchema: pmodel.NewCIStr("test"), - ChildTable: pmodel.NewCIStr("t3"), - ChildFKName: pmodel.NewCIStr("fk_a"), + Cols: []ast.CIStr{ast.NewCIStr("id")}, + ChildSchema: ast.NewCIStr("test"), + ChildTable: ast.NewCIStr("t3"), + ChildFKName: ast.NewCIStr("fk_a"), }, *tb1ReferredFKs[0]) tb3ReferredFKs = getTableInfoReferredForeignKeys(t, dom, "test", "t3") require.Equal(t, 0, len(tb3ReferredFKs)) require.Equal(t, 2, len(tb3Info.ForeignKeys)) require.Equal(t, model.FKInfo{ ID: 1, - Name: pmodel.NewCIStr("fk_a"), - RefSchema: pmodel.NewCIStr("test"), - RefTable: pmodel.NewCIStr("t1"), - RefCols: []pmodel.CIStr{pmodel.NewCIStr("id")}, - Cols: []pmodel.CIStr{pmodel.NewCIStr("a")}, + Name: ast.NewCIStr("fk_a"), + RefSchema: ast.NewCIStr("test"), + RefTable: ast.NewCIStr("t1"), + RefCols: []ast.CIStr{ast.NewCIStr("id")}, + Cols: []ast.CIStr{ast.NewCIStr("a")}, OnDelete: 2, OnUpdate: 1, State: model.StatePublic, @@ -266,11 +266,11 @@ func TestCreateTableWithForeignKeyMetaInfo2(t *testing.T) { }, *tb3Info.ForeignKeys[0]) require.Equal(t, model.FKInfo{ ID: 2, - Name: pmodel.NewCIStr("fk_a2"), - RefSchema: pmodel.NewCIStr("test2"), - RefTable: pmodel.NewCIStr("t2"), - RefCols: []pmodel.CIStr{pmodel.NewCIStr("id")}, - Cols: []pmodel.CIStr{pmodel.NewCIStr("a")}, + Name: ast.NewCIStr("fk_a2"), + RefSchema: ast.NewCIStr("test2"), + RefTable: ast.NewCIStr("t2"), + RefCols: []ast.CIStr{ast.NewCIStr("id")}, + Cols: []ast.CIStr{ast.NewCIStr("a")}, State: model.StatePublic, Version: 1, }, *tb3Info.ForeignKeys[1]) @@ -366,18 +366,18 @@ func TestRenameTableWithForeignKeyMetaInfo(t *testing.T) { require.Equal(t, 1, len(tblInfo.ForeignKeys)) require.Equal(t, 1, len(tbReferredFKs)) require.Equal(t, model.ReferredFKInfo{ - Cols: []pmodel.CIStr{pmodel.NewCIStr("id")}, - ChildSchema: pmodel.NewCIStr("test2"), - ChildTable: pmodel.NewCIStr("t2"), - ChildFKName: pmodel.NewCIStr("fk"), + Cols: []ast.CIStr{ast.NewCIStr("id")}, + ChildSchema: ast.NewCIStr("test2"), + ChildTable: ast.NewCIStr("t2"), + ChildFKName: ast.NewCIStr("fk"), }, *tbReferredFKs[0]) require.Equal(t, model.FKInfo{ ID: 1, - Name: pmodel.NewCIStr("fk"), - RefSchema: pmodel.NewCIStr("test2"), - RefTable: pmodel.NewCIStr("t2"), - RefCols: []pmodel.CIStr{pmodel.NewCIStr("id")}, - Cols: []pmodel.CIStr{pmodel.NewCIStr("a")}, + Name: ast.NewCIStr("fk"), + RefSchema: ast.NewCIStr("test2"), + RefTable: ast.NewCIStr("t2"), + RefCols: []ast.CIStr{ast.NewCIStr("id")}, + Cols: []ast.CIStr{ast.NewCIStr("a")}, State: model.StatePublic, Version: 1, }, *tblInfo.ForeignKeys[0]) @@ -398,21 +398,21 @@ func TestRenameTableWithForeignKeyMetaInfo(t *testing.T) { tb1ReferredFKs := getTableInfoReferredForeignKeys(t, dom, "test", "t1") require.Equal(t, 1, len(tb1ReferredFKs)) require.Equal(t, model.ReferredFKInfo{ - Cols: []pmodel.CIStr{pmodel.NewCIStr("id")}, - ChildSchema: pmodel.NewCIStr("test2"), - ChildTable: pmodel.NewCIStr("tt2"), - ChildFKName: pmodel.NewCIStr("fk_b"), + Cols: []ast.CIStr{ast.NewCIStr("id")}, + ChildSchema: ast.NewCIStr("test2"), + ChildTable: ast.NewCIStr("tt2"), + ChildFKName: ast.NewCIStr("fk_b"), }, *tb1ReferredFKs[0]) tb2ReferredFKs := getTableInfoReferredForeignKeys(t, dom, "test2", "tt2") require.Equal(t, 0, len(tb2ReferredFKs)) require.Equal(t, 1, len(tb2Info.ForeignKeys)) require.Equal(t, model.FKInfo{ ID: 1, - Name: pmodel.NewCIStr("fk_b"), - RefSchema: pmodel.NewCIStr("test"), - RefTable: pmodel.NewCIStr("t1"), - RefCols: []pmodel.CIStr{pmodel.NewCIStr("id")}, - Cols: []pmodel.CIStr{pmodel.NewCIStr("b")}, + Name: ast.NewCIStr("fk_b"), + RefSchema: ast.NewCIStr("test"), + RefTable: ast.NewCIStr("t1"), + RefCols: []ast.CIStr{ast.NewCIStr("id")}, + Cols: []ast.CIStr{ast.NewCIStr("b")}, State: model.StatePublic, Version: 1, }, *tb2Info.ForeignKeys[0]) @@ -430,10 +430,10 @@ func TestRenameTableWithForeignKeyMetaInfo(t *testing.T) { require.Equal(t, model.ActionRenameTable, diff.Type) require.Equal(t, 0, len(diff.AffectedOpts)) require.Equal(t, model.ReferredFKInfo{ - Cols: []pmodel.CIStr{pmodel.NewCIStr("id")}, - ChildSchema: pmodel.NewCIStr("test2"), - ChildTable: pmodel.NewCIStr("tt2"), - ChildFKName: pmodel.NewCIStr("fk_b"), + Cols: []ast.CIStr{ast.NewCIStr("id")}, + ChildSchema: ast.NewCIStr("test2"), + ChildTable: ast.NewCIStr("tt2"), + ChildFKName: ast.NewCIStr("fk_b"), }, *tb1ReferredFKs[0]) tbl2Info := getTableInfo(t, dom, "test2", "tt2") tb2ReferredFKs = getTableInfoReferredForeignKeys(t, dom, "test2", "tt2") @@ -441,11 +441,11 @@ func TestRenameTableWithForeignKeyMetaInfo(t *testing.T) { require.Equal(t, 1, len(tbl2Info.ForeignKeys)) require.Equal(t, model.FKInfo{ ID: 1, - Name: pmodel.NewCIStr("fk_b"), - RefSchema: pmodel.NewCIStr("test3"), - RefTable: pmodel.NewCIStr("tt1"), - RefCols: []pmodel.CIStr{pmodel.NewCIStr("id")}, - Cols: []pmodel.CIStr{pmodel.NewCIStr("b")}, + Name: ast.NewCIStr("fk_b"), + RefSchema: ast.NewCIStr("test3"), + RefTable: ast.NewCIStr("tt1"), + RefCols: []ast.CIStr{ast.NewCIStr("id")}, + Cols: []ast.CIStr{ast.NewCIStr("b")}, State: model.StatePublic, Version: 1, }, *tbl2Info.ForeignKeys[0]) @@ -959,7 +959,7 @@ func getTableInfo(t *testing.T, dom *domain.Domain, db, tb string) *model.TableI err := dom.Reload() require.NoError(t, err) is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr(db), pmodel.NewCIStr(tb)) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr(db), ast.NewCIStr(tb)) require.NoError(t, err) _, exist := is.TableByID(context.Background(), tbl.Meta().ID) require.True(t, exist) @@ -1465,16 +1465,16 @@ func TestRenameTablesWithForeignKey(t *testing.T) { require.Equal(t, 1, len(tt1ReferredFKs)) require.Equal(t, 1, len(tt2ReferredFKs)) require.Equal(t, model.ReferredFKInfo{ - Cols: []pmodel.CIStr{pmodel.NewCIStr("id")}, - ChildSchema: pmodel.NewCIStr("test2"), - ChildTable: pmodel.NewCIStr("tt2"), - ChildFKName: pmodel.NewCIStr("fk"), + Cols: []ast.CIStr{ast.NewCIStr("id")}, + ChildSchema: ast.NewCIStr("test2"), + ChildTable: ast.NewCIStr("tt2"), + ChildFKName: ast.NewCIStr("fk"), }, *tt1ReferredFKs[0]) require.Equal(t, model.ReferredFKInfo{ - Cols: []pmodel.CIStr{pmodel.NewCIStr("id")}, - ChildSchema: pmodel.NewCIStr("test1"), - ChildTable: pmodel.NewCIStr("tt1"), - ChildFKName: pmodel.NewCIStr("fk"), + Cols: []ast.CIStr{ast.NewCIStr("id")}, + ChildSchema: ast.NewCIStr("test1"), + ChildTable: ast.NewCIStr("tt1"), + ChildFKName: ast.NewCIStr("fk"), }, *tt2ReferredFKs[0]) // check show create table information diff --git a/pkg/ddl/tests/multivaluedindex/BUILD.bazel b/pkg/ddl/tests/multivaluedindex/BUILD.bazel index bf3d650763386..826b28239ac48 100644 --- a/pkg/ddl/tests/multivaluedindex/BUILD.bazel +++ b/pkg/ddl/tests/multivaluedindex/BUILD.bazel @@ -10,7 +10,7 @@ go_test( flaky = True, deps = [ "//pkg/infoschema", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/testkit", "//pkg/testkit/testsetup", "@com_github_stretchr_testify//require", diff --git a/pkg/ddl/tests/multivaluedindex/multi_valued_index_test.go b/pkg/ddl/tests/multivaluedindex/multi_valued_index_test.go index 271532b667d33..a478d543d2d6a 100644 --- a/pkg/ddl/tests/multivaluedindex/multi_valued_index_test.go +++ b/pkg/ddl/tests/multivaluedindex/multi_valued_index_test.go @@ -19,7 +19,7 @@ import ( "testing" "github.com/pingcap/tidb/pkg/infoschema" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/testkit" "github.com/stretchr/testify/require" ) @@ -32,7 +32,7 @@ func TestCreateMultiValuedIndexHasBinaryCollation(t *testing.T) { is := tk.Session().GetDomainInfoSchema().(infoschema.InfoSchema) require.NotNil(t, is) - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) foundIndex := false diff --git a/pkg/ddl/tests/partition/BUILD.bazel b/pkg/ddl/tests/partition/BUILD.bazel index 2650bd7b3883b..25a8e3a23c3ef 100644 --- a/pkg/ddl/tests/partition/BUILD.bazel +++ b/pkg/ddl/tests/partition/BUILD.bazel @@ -26,7 +26,6 @@ go_test( "//pkg/kv", "//pkg/meta/model", "//pkg/parser/ast", - "//pkg/parser/model", "//pkg/parser/mysql", "//pkg/parser/terror", "//pkg/session", diff --git a/pkg/ddl/tests/partition/db_partition_test.go b/pkg/ddl/tests/partition/db_partition_test.go index a4b14b38cf836..399d9325ac684 100644 --- a/pkg/ddl/tests/partition/db_partition_test.go +++ b/pkg/ddl/tests/partition/db_partition_test.go @@ -35,7 +35,6 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/ast" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/session" @@ -100,11 +99,11 @@ func TestCreateTableWithPartition(t *testing.T) { );`) ctx := tk.Session() is := domain.GetDomain(ctx).InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("tp")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("tp")) require.NoError(t, err) require.NotNil(t, tbl.Meta().Partition) part := tbl.Meta().Partition - require.Equal(t, pmodel.PartitionTypeRange, part.Type) + require.Equal(t, ast.PartitionTypeRange, part.Type) require.Equal(t, "`a`", part.Expr) for _, pdef := range part.Definitions { require.Greater(t, pdef.ID, int64(0)) @@ -931,7 +930,7 @@ func TestCreateTableWithListPartition(t *testing.T) { tblInfo := tbl.Meta() require.NotNil(t, tblInfo.Partition) require.True(t, tblInfo.Partition.Enable) - require.Equal(t, pmodel.PartitionTypeList, tblInfo.Partition.Type) + require.Equal(t, ast.PartitionTypeList, tblInfo.Partition.Type) } } @@ -1141,7 +1140,7 @@ func TestCreateTableWithListColumnsPartition(t *testing.T) { tblInfo := tbl.Meta() require.NotNil(t, tblInfo.Partition) require.Equal(t, true, tblInfo.Partition.Enable) - require.True(t, tblInfo.Partition.Type == pmodel.PartitionTypeList) + require.True(t, tblInfo.Partition.Type == ast.PartitionTypeList) } } @@ -1162,10 +1161,10 @@ func TestAlterTableTruncatePartitionByList(t *testing.T) { tbl := external.GetTableByName(t, tk, "test", "t") require.NotNil(t, tbl.Meta().Partition) part := tbl.Meta().Partition - require.True(t, part.Type == pmodel.PartitionTypeList) + require.True(t, part.Type == ast.PartitionTypeList) require.Len(t, part.Definitions, 3) require.Equal(t, [][]string{{"3"}, {"4"}}, part.Definitions[1].InValues) - require.Equal(t, pmodel.NewCIStr("p1"), part.Definitions[1].Name) + require.Equal(t, ast.NewCIStr("p1"), part.Definitions[1].Name) require.False(t, part.Definitions[1].ID == oldTbl.Meta().Partition.Definitions[1].ID) sql := "alter table t truncate partition p10;" @@ -1193,10 +1192,10 @@ func TestAlterTableTruncatePartitionByListColumns(t *testing.T) { tbl := external.GetTableByName(t, tk, "test", "t") require.NotNil(t, tbl.Meta().Partition) part := tbl.Meta().Partition - require.True(t, part.Type == pmodel.PartitionTypeList) + require.True(t, part.Type == ast.PartitionTypeList) require.Len(t, part.Definitions, 3) require.Equal(t, [][]string{{"3", `'a'`}, {"4", `'b'`}}, part.Definitions[1].InValues) - require.Equal(t, pmodel.NewCIStr("p1"), part.Definitions[1].Name) + require.Equal(t, ast.NewCIStr("p1"), part.Definitions[1].Name) require.False(t, part.Definitions[1].ID == oldTbl.Meta().Partition.Definitions[1].ID) sql := "alter table t truncate partition p10;" @@ -2235,7 +2234,7 @@ func TestTruncatePartitionAndDropTable(t *testing.T) { result.Check(testkit.Rows("10")) ctx := tk.Session() is := domain.GetDomain(ctx).InfoSchema() - oldTblInfo, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t3")) + oldTblInfo, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t3")) require.NoError(t, err) // Only one partition id test is taken here. tk.MustExec("truncate table t3;") @@ -2270,7 +2269,7 @@ func TestTruncatePartitionAndDropTable(t *testing.T) { result = tk.MustQuery("select count(*) from t4; ") result.Check(testkit.Rows("10")) is = domain.GetDomain(ctx).InfoSchema() - oldTblInfo, err = is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t4")) + oldTblInfo, err = is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t4")) require.NoError(t, err) // Only one partition id test is taken here. oldPID = oldTblInfo.Meta().Partition.Definitions[1].ID @@ -2293,13 +2292,13 @@ func TestTruncatePartitionAndDropTable(t *testing.T) { partition p5 values less than (2015) );`) is = domain.GetDomain(ctx).InfoSchema() - oldTblInfo, err = is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t5")) + oldTblInfo, err = is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t5")) require.NoError(t, err) oldPID = oldTblInfo.Meta().Partition.Definitions[0].ID tk.MustExec("truncate table t5;") is = domain.GetDomain(ctx).InfoSchema() - newTblInfo, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t5")) + newTblInfo, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t5")) require.NoError(t, err) newPID := newTblInfo.Meta().Partition.Definitions[0].ID require.True(t, oldPID != newPID) @@ -2314,14 +2313,14 @@ func TestTruncatePartitionAndDropTable(t *testing.T) { partition by hash( month(signed) ) partitions 12;`) is = domain.GetDomain(ctx).InfoSchema() - oldTblInfo, err = is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("clients")) + oldTblInfo, err = is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("clients")) require.NoError(t, err) oldDefs := oldTblInfo.Meta().Partition.Definitions // Test truncate `hash partitioned table` reassigns new partitionIDs. tk.MustExec("truncate table clients;") is = domain.GetDomain(ctx).InfoSchema() - newTblInfo, err = is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("clients")) + newTblInfo, err = is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("clients")) require.NoError(t, err) newDefs := newTblInfo.Meta().Partition.Definitions for i := 0; i < len(oldDefs); i++ { @@ -2473,7 +2472,7 @@ func testPartitionAddIndex(tk *testkit.TestKit, t *testing.T, key string) { tk.MustExec("alter table partition_add_idx add index idx2 (id, hired)") ctx := tk.Session() is := domain.GetDomain(ctx).InfoSchema() - tt, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("partition_add_idx")) + tt, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("partition_add_idx")) require.NoError(t, err) var idx1 table.Index for _, idx := range tt.Indices() { @@ -3280,7 +3279,7 @@ func TestAlterLastIntervalPartition(t *testing.T) { first partition less than ('2023-01-01') last partition less than ('2023-01-03');`) ctx := tk.Session() - tbl, err := domain.GetDomain(ctx).InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := domain.GetDomain(ctx).InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) pd := tbl.Meta().Partition.Definitions require.Equal(t, 3, len(pd)) @@ -3289,7 +3288,7 @@ func TestAlterLastIntervalPartition(t *testing.T) { require.Equal(t, "'2023-01-03 00:00:00'", pd[2].LessThan[0]) tk.MustExec("alter table t last partition less than ('2024-01-04')") tk.MustExec("alter table t last partition less than ('2025-01-01 00:00:00')") - tbl, err = domain.GetDomain(ctx).InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err = domain.GetDomain(ctx).InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) pd = tbl.Meta().Partition.Definitions require.Equal(t, 732, len(pd)) @@ -3305,7 +3304,7 @@ func TestAlterLastIntervalPartition(t *testing.T) { interval (2 day) first partition less than ('2023-01-01') last partition less than ('2023-01-05');`) - tbl, err = domain.GetDomain(ctx).InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t2")) + tbl, err = domain.GetDomain(ctx).InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t2")) require.NoError(t, err) pd = tbl.Meta().Partition.Definitions require.Equal(t, 3, len(pd)) @@ -3314,7 +3313,7 @@ func TestAlterLastIntervalPartition(t *testing.T) { require.Equal(t, "'2023-01-05 00:00:00'", pd[2].LessThan[0]) tk.MustExec("alter table t2 last partition less than ('2023-01-09')") tk.MustExec("alter table t2 last partition less than ('2023-01-11 00:00:00')") - tbl, err = domain.GetDomain(ctx).InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t2")) + tbl, err = domain.GetDomain(ctx).InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t2")) require.NoError(t, err) pd = tbl.Meta().Partition.Definitions require.Equal(t, 6, len(pd)) @@ -3331,7 +3330,7 @@ func TestAlterLastIntervalPartition(t *testing.T) { interval (2 day) first partition less than ('2023-01-01 12:01:02') last partition less than ('2023-01-05 12:01:02');`) - tbl, err = domain.GetDomain(ctx).InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t3")) + tbl, err = domain.GetDomain(ctx).InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t3")) require.NoError(t, err) pd = tbl.Meta().Partition.Definitions require.Equal(t, 3, len(pd)) @@ -3339,7 +3338,7 @@ func TestAlterLastIntervalPartition(t *testing.T) { require.Equal(t, "'2023-01-03 12:01:02'", pd[1].LessThan[0]) require.Equal(t, "'2023-01-05 12:01:02'", pd[2].LessThan[0]) tk.MustExec("alter table t3 last partition less than ('2023-01-09 12:01:02')") - tbl, err = domain.GetDomain(ctx).InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t3")) + tbl, err = domain.GetDomain(ctx).InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t3")) require.NoError(t, err) pd = tbl.Meta().Partition.Definitions require.Equal(t, 5, len(pd)) @@ -3355,7 +3354,7 @@ func TestAlterLastIntervalPartition(t *testing.T) { interval (48 hour) first partition less than ('2023-01-01') last partition less than ('2023-01-05');`) - tbl, err = domain.GetDomain(ctx).InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t4")) + tbl, err = domain.GetDomain(ctx).InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t4")) require.NoError(t, err) pd = tbl.Meta().Partition.Definitions require.Equal(t, 3, len(pd)) @@ -3363,7 +3362,7 @@ func TestAlterLastIntervalPartition(t *testing.T) { require.Equal(t, "'2023-01-03 00:00:00'", pd[1].LessThan[0]) require.Equal(t, "'2023-01-05 00:00:00'", pd[2].LessThan[0]) tk.MustExec("alter table t4 last partition less than ('2023-01-09 00:00:00')") - tbl, err = domain.GetDomain(ctx).InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t4")) + tbl, err = domain.GetDomain(ctx).InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t4")) require.NoError(t, err) pd = tbl.Meta().Partition.Definitions require.Equal(t, 5, len(pd)) diff --git a/pkg/ddl/tests/partition/multi_domain_test.go b/pkg/ddl/tests/partition/multi_domain_test.go index 60d2985b19891..d85a11122a876 100644 --- a/pkg/ddl/tests/partition/multi_domain_test.go +++ b/pkg/ddl/tests/partition/multi_domain_test.go @@ -25,7 +25,7 @@ import ( "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/session" "github.com/pingcap/tidb/pkg/sessiontxn" "github.com/pingcap/tidb/pkg/store/gcworker" @@ -620,9 +620,9 @@ func TestMultiSchemaModifyColumn(t *testing.T) { // to the new type, and block writes otherwise. But then it would break the first tkO insert above... tkO.MustQuery(`select * from t where a = 11`).Check(testkit.Rows("11 12")) tkNO.MustQuery(`select * from t where a = 11`).Check(testkit.Rows("11 011.50 ")) - tblO, err := tkO.Session().GetInfoSchema().TableInfoByName(pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tblO, err := tkO.Session().GetInfoSchema().TableInfoByName(ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) - tblNO, err := tkNO.Session().GetInfoSchema().TableInfoByName(pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tblNO, err := tkNO.Session().GetInfoSchema().TableInfoByName(ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) require.Greater(t, tblO.Columns[1].ID, tblNO.Columns[1].ID) // This also means that old copies of the columns will be left in the row, until the row is updated or deleted. @@ -844,7 +844,7 @@ func runMultiSchemaTest(t *testing.T, createSQL, alterSQL string, initFn func(*t originalGlobalIndexIDs := make([]int64, 0, 1) ctx := tkO.Session() is := domain.GetDomain(ctx).InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableID := tbl.Meta().ID if tbl.Meta().Partition != nil { @@ -941,7 +941,7 @@ func runMultiSchemaTest(t *testing.T, createSQL, alterSQL string, initFn func(*t tkO.MustQuery(`select * from mysql.gc_delete_range`).Check(testkit.Rows()) ctx = tkO.Session() is = domain.GetDomain(ctx).InfoSchema() - tbl, err = is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err = is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) newTableID := tbl.Meta().ID if tableID != newTableID { @@ -1083,10 +1083,10 @@ func TestMultiSchemaTruncatePartitionWithGlobalIndex(t *testing.T) { // even if it cannot read them from the global index, due to filtering. rows := tkNO.MustQuery(`select * from t`).Sort().Rows() tkO.MustQuery(`select * from t`).Sort().Check(rows) - tblNO, err := tkNO.Session().GetInfoSchema().TableInfoByName(pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tblNO, err := tkNO.Session().GetInfoSchema().TableInfoByName(ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) require.Equal(t, model.StateNone, tblNO.Partition.DDLState) - tblO, err := tkO.Session().GetInfoSchema().TableInfoByName(pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tblO, err := tkO.Session().GetInfoSchema().TableInfoByName(ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) require.Equal(t, model.StateWriteOnly, tblO.Partition.DDLState) require.Equal(t, tblNO.Partition.Definitions[1].ID, tblO.Partition.Definitions[1].ID) @@ -1100,10 +1100,10 @@ func TestMultiSchemaTruncatePartitionWithGlobalIndex(t *testing.T) { tkNO.MustContainErrMsg(`insert into t values (1,1,"Duplicate key")`, "[kv:1062]Duplicate entry '1' for key 't.") tkO.MustContainErrMsg(`insert into t values (1,1,"Duplicate key")`, "[kv:1062]Duplicate entry '1' for key 't.") - tblNO, err := tkNO.Session().GetInfoSchema().TableInfoByName(pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tblNO, err := tkNO.Session().GetInfoSchema().TableInfoByName(ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) require.Equal(t, model.StateWriteOnly, tblNO.Partition.DDLState) - tblO, err := tkO.Session().GetInfoSchema().TableInfoByName(pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tblO, err := tkO.Session().GetInfoSchema().TableInfoByName(ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) require.Equal(t, model.StateDeleteOnly, tblO.Partition.DDLState) require.NotEqual(t, tblNO.Partition.Definitions[1].ID, tblO.Partition.Definitions[1].ID) @@ -1200,10 +1200,10 @@ func TestMultiSchemaTruncatePartitionWithGlobalIndex(t *testing.T) { tkNO.MustQuery(`select * from t`).Sort().Check(rows) rows = tkO.MustQuery(`select b from t order by b`).Rows() tkNO.MustQuery(`select b from t order by b`).Check(rows) - tblNO, err := tkNO.Session().GetInfoSchema().TableInfoByName(pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tblNO, err := tkNO.Session().GetInfoSchema().TableInfoByName(ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) require.Equal(t, model.StateDeleteOnly, tblNO.Partition.DDLState) - tblO, err := tkO.Session().GetInfoSchema().TableInfoByName(pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tblO, err := tkO.Session().GetInfoSchema().TableInfoByName(ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) require.Equal(t, model.StateDeleteReorganization, tblO.Partition.DDLState) require.Equal(t, tblNO.Partition.Definitions[1].ID, tblO.Partition.Definitions[1].ID) @@ -1239,10 +1239,10 @@ func TestMultiSchemaTruncatePartitionWithGlobalIndex(t *testing.T) { tkO.MustExec(`insert into t values (87,87,"OK")`) rows := tkNO.MustQuery(`select * from t`).Sort().Rows() tkO.MustQuery(`select * from t`).Sort().Check(rows) - tblNO, err := tkNO.Session().GetInfoSchema().TableInfoByName(pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tblNO, err := tkNO.Session().GetInfoSchema().TableInfoByName(ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) require.Equal(t, model.StateDeleteReorganization, tblNO.Partition.DDLState) - tblO, err := tkO.Session().GetInfoSchema().TableInfoByName(pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tblO, err := tkO.Session().GetInfoSchema().TableInfoByName(ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) require.Equal(t, model.StateNone, tblO.Partition.DDLState) default: @@ -1270,10 +1270,10 @@ func TestMultiSchemaTruncatePartitionWithPKGlobal(t *testing.T) { // even if it cannot read them from the global index, due to filtering. rows := tkNO.MustQuery(`select * from t`).Sort().Rows() tkO.MustQuery(`select * from t`).Sort().Check(rows) - tblNO, err := tkNO.Session().GetInfoSchema().TableInfoByName(pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tblNO, err := tkNO.Session().GetInfoSchema().TableInfoByName(ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) require.Equal(t, model.StateNone, tblNO.Partition.DDLState) - tblO, err := tkO.Session().GetInfoSchema().TableInfoByName(pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tblO, err := tkO.Session().GetInfoSchema().TableInfoByName(ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) require.Equal(t, model.StateWriteOnly, tblO.Partition.DDLState) require.Equal(t, tblNO.Partition.Definitions[1].ID, tblO.Partition.Definitions[1].ID) @@ -1303,10 +1303,10 @@ func TestMultiSchemaTruncatePartitionWithPKGlobal(t *testing.T) { tkO.MustQuery(`select * from t where b = 13`).Check(testkit.Rows("13 13 OK")) tkNO.MustQuery(`select * from t where b = 13`).Check(testkit.Rows("15 13 OK, non global unique index")) - tblNO, err := tkNO.Session().GetInfoSchema().TableInfoByName(pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tblNO, err := tkNO.Session().GetInfoSchema().TableInfoByName(ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) require.Equal(t, model.StateWriteOnly, tblNO.Partition.DDLState) - tblO, err := tkO.Session().GetInfoSchema().TableInfoByName(pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tblO, err := tkO.Session().GetInfoSchema().TableInfoByName(ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) require.Equal(t, model.StateDeleteOnly, tblO.Partition.DDLState) require.NotEqual(t, tblNO.Partition.Definitions[1].ID, tblO.Partition.Definitions[1].ID) @@ -1393,10 +1393,10 @@ func TestMultiSchemaTruncatePartitionWithPKGlobal(t *testing.T) { // and can ignore the dropped partitions entries in the Global Indexes, i.e. overwrite them! rows := tkO.MustQuery(`select * from t`).Sort().Rows() tkNO.MustQuery(`select * from t`).Sort().Check(rows) - tblNO, err := tkNO.Session().GetInfoSchema().TableInfoByName(pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tblNO, err := tkNO.Session().GetInfoSchema().TableInfoByName(ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) require.Equal(t, model.StateDeleteOnly, tblNO.Partition.DDLState) - tblO, err := tkO.Session().GetInfoSchema().TableInfoByName(pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tblO, err := tkO.Session().GetInfoSchema().TableInfoByName(ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) require.Equal(t, model.StateDeleteReorganization, tblO.Partition.DDLState) require.Equal(t, tblNO.Partition.Definitions[1].ID, tblO.Partition.Definitions[1].ID) @@ -1430,10 +1430,10 @@ func TestMultiSchemaTruncatePartitionWithPKGlobal(t *testing.T) { tkO.MustExec(`insert into t values (87,87,"OK")`) rows := tkNO.MustQuery(`select * from t`).Sort().Rows() tkO.MustQuery(`select * from t`).Sort().Check(rows) - tblNO, err := tkNO.Session().GetInfoSchema().TableInfoByName(pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tblNO, err := tkNO.Session().GetInfoSchema().TableInfoByName(ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) require.Equal(t, model.StateDeleteReorganization, tblNO.Partition.DDLState) - tblO, err := tkO.Session().GetInfoSchema().TableInfoByName(pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tblO, err := tkO.Session().GetInfoSchema().TableInfoByName(ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) require.Equal(t, model.StateNone, tblO.Partition.DDLState) require.Equal(t, tblNO.Partition.Definitions[1].ID, tblO.Partition.Definitions[1].ID) diff --git a/pkg/ddl/tests/partition/reorg_partition_test.go b/pkg/ddl/tests/partition/reorg_partition_test.go index 12c20d2ac194e..8c0dd45b15bfb 100644 --- a/pkg/ddl/tests/partition/reorg_partition_test.go +++ b/pkg/ddl/tests/partition/reorg_partition_test.go @@ -30,7 +30,7 @@ import ( "github.com/pingcap/tidb/pkg/errno" "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessiontxn" "github.com/pingcap/tidb/pkg/store/gcworker" @@ -538,9 +538,9 @@ func TestReorgPartitionConcurrent(t *testing.T) { tk.MustExec(`admin check table t`) writeOnlyInfoSchema := sessiontxn.GetTxnManager(tk.Session()).GetTxnInfoSchema() require.Equal(t, int64(1), writeOnlyInfoSchema.SchemaMetaVersion()-deleteOnlyInfoSchema.SchemaMetaVersion()) - deleteOnlyTbl, err := deleteOnlyInfoSchema.TableByName(context.Background(), pmodel.NewCIStr(schemaName), pmodel.NewCIStr("t")) + deleteOnlyTbl, err := deleteOnlyInfoSchema.TableByName(context.Background(), ast.NewCIStr(schemaName), ast.NewCIStr("t")) require.NoError(t, err) - writeOnlyTbl, err := writeOnlyInfoSchema.TableByName(context.Background(), pmodel.NewCIStr(schemaName), pmodel.NewCIStr("t")) + writeOnlyTbl, err := writeOnlyInfoSchema.TableByName(context.Background(), ast.NewCIStr(schemaName), ast.NewCIStr("t")) require.NoError(t, err) writeOnlyParts := writeOnlyTbl.Meta().Partition writeOnlyTbl.Meta().Partition = deleteOnlyTbl.Meta().Partition @@ -581,13 +581,13 @@ func TestReorgPartitionConcurrent(t *testing.T) { deleteReorgInfoSchema := sessiontxn.GetTxnManager(tk.Session()).GetTxnInfoSchema() require.Equal(t, int64(1), deleteReorgInfoSchema.SchemaMetaVersion()-writeReorgInfoSchema.SchemaMetaVersion()) tk.MustExec(`insert into t values (16, "16", 16)`) - oldTbl, err := writeReorgInfoSchema.TableByName(context.Background(), pmodel.NewCIStr(schemaName), pmodel.NewCIStr("t")) + oldTbl, err := writeReorgInfoSchema.TableByName(context.Background(), ast.NewCIStr(schemaName), ast.NewCIStr("t")) require.NoError(t, err) partDef := oldTbl.Meta().Partition.Definitions[1] require.Equal(t, "p1", partDef.Name.O) rows := getNumRowsFromPartitionDefs(t, tk, oldTbl, oldTbl.Meta().Partition.Definitions[1:2]) require.Equal(t, 5, rows) - currTbl, err := deleteReorgInfoSchema.TableByName(context.Background(), pmodel.NewCIStr(schemaName), pmodel.NewCIStr("t")) + currTbl, err := deleteReorgInfoSchema.TableByName(context.Background(), ast.NewCIStr(schemaName), ast.NewCIStr("t")) require.NoError(t, err) currPart := currTbl.Meta().Partition currTbl.Meta().Partition = oldTbl.Meta().Partition @@ -640,14 +640,14 @@ func TestReorgPartitionConcurrent(t *testing.T) { publicInfoSchema := sessiontxn.GetTxnManager(tk.Session()).GetTxnInfoSchema() require.Equal(t, int64(1), publicInfoSchema.SchemaMetaVersion()-deleteReorgInfoSchema.SchemaMetaVersion()) tk.MustExec(`insert into t values (17, "17", 17)`) - oldTbl, err = deleteReorgInfoSchema.TableByName(context.Background(), pmodel.NewCIStr(schemaName), pmodel.NewCIStr("t")) + oldTbl, err = deleteReorgInfoSchema.TableByName(context.Background(), ast.NewCIStr(schemaName), ast.NewCIStr("t")) require.NoError(t, err) partDef = oldTbl.Meta().Partition.Definitions[1] require.Equal(t, "p1a", partDef.Name.O) rows = getNumRowsFromPartitionDefs(t, tk, oldTbl, oldTbl.Meta().Partition.Definitions[1:2]) require.Equal(t, 3, rows) tk.MustQuery(`select * from t partition (p1a)`).Sort().Check(testkit.Rows("10 10 10", "12 12b 12", "14 14 14")) - currTbl, err = publicInfoSchema.TableByName(context.Background(), pmodel.NewCIStr(schemaName), pmodel.NewCIStr("t")) + currTbl, err = publicInfoSchema.TableByName(context.Background(), ast.NewCIStr(schemaName), ast.NewCIStr("t")) require.NoError(t, err) currPart = currTbl.Meta().Partition currTbl.Meta().Partition = oldTbl.Meta().Partition @@ -681,7 +681,7 @@ func TestReorgPartitionConcurrent(t *testing.T) { tk.MustExec(`admin check table t`) newInfoSchema := sessiontxn.GetTxnManager(tk.Session()).GetTxnInfoSchema() require.Equal(t, int64(1), newInfoSchema.SchemaMetaVersion()-publicInfoSchema.SchemaMetaVersion()) - oldTbl, err = publicInfoSchema.TableByName(context.Background(), pmodel.NewCIStr(schemaName), pmodel.NewCIStr("t")) + oldTbl, err = publicInfoSchema.TableByName(context.Background(), ast.NewCIStr(schemaName), ast.NewCIStr("t")) require.NoError(t, err) partDef = oldTbl.Meta().Partition.Definitions[1] require.Equal(t, "p1a", partDef.Name.O) @@ -699,7 +699,7 @@ func TestReorgPartitionConcurrent(t *testing.T) { " PARTITION `p1a` VALUES LESS THAN (15),\n" + " PARTITION `p1b` VALUES LESS THAN (20),\n" + " PARTITION `pMax` VALUES LESS THAN (MAXVALUE))")) - newTbl, err := newInfoSchema.TableByName(context.Background(), pmodel.NewCIStr(schemaName), pmodel.NewCIStr("t")) + newTbl, err := newInfoSchema.TableByName(context.Background(), ast.NewCIStr(schemaName), ast.NewCIStr("t")) require.NoError(t, err) newPart := newTbl.Meta().Partition newTbl.Meta().Partition = oldTbl.Meta().Partition @@ -798,7 +798,7 @@ func TestReorgPartitionFailConcurrent(t *testing.T) { go backgroundExec(store, schemaName, "alter table t reorganize partition p1a,p1b into (partition p1a values less than (14), partition p1b values less than (17), partition p1c values less than (20))", alterErr) wait <- true infoSchema := sessiontxn.GetTxnManager(tk.Session()).GetTxnInfoSchema() - tbl, err := infoSchema.TableByName(context.Background(), pmodel.NewCIStr(schemaName), pmodel.NewCIStr("t")) + tbl, err := infoSchema.TableByName(context.Background(), ast.NewCIStr(schemaName), ast.NewCIStr("t")) require.NoError(t, err) require.Equal(t, 0, getNumRowsFromPartitionDefs(t, tk, tbl, tbl.Meta().Partition.AddingDefinitions)) tk.MustExec(`delete from t where a = 14`) @@ -806,7 +806,7 @@ func TestReorgPartitionFailConcurrent(t *testing.T) { tk.MustExec(`admin check table t`) wait <- true wait <- true - tbl, err = infoSchema.TableByName(context.Background(), pmodel.NewCIStr(schemaName), pmodel.NewCIStr("t")) + tbl, err = infoSchema.TableByName(context.Background(), ast.NewCIStr(schemaName), ast.NewCIStr("t")) require.NoError(t, err) require.Equal(t, 5, getNumRowsFromPartitionDefs(t, tk, tbl, tbl.Meta().Partition.AddingDefinitions)) tk.MustExec(`delete from t where a = 15`) @@ -930,7 +930,7 @@ func TestReorgPartitionRollback(t *testing.T) { testfailpoint.Disable(t, "github.com/pingcap/tidb/pkg/ddl/mockUpdateVersionAndTableInfoErr") ctx := tk.Session() is := domain.GetDomain(ctx).InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr(schemaName), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr(schemaName), ast.NewCIStr("t")) require.NoError(t, err) noNewTablesAfter(t, tk, ctx, tbl, "Reorganize rollback") testfailpoint.Enable(t, "github.com/pingcap/tidb/pkg/ddl/reorgPartitionAfterDataCopy", `return(true)`) @@ -953,7 +953,7 @@ func TestReorgPartitionRollback(t *testing.T) { " PARTITION `p1` VALUES LESS THAN (20),\n" + " PARTITION `pMax` VALUES LESS THAN (MAXVALUE))")) - tbl, err = is.TableByName(context.Background(), pmodel.NewCIStr(schemaName), pmodel.NewCIStr("t")) + tbl, err = is.TableByName(context.Background(), ast.NewCIStr(schemaName), ast.NewCIStr("t")) require.NoError(t, err) noNewTablesAfter(t, tk, ctx, tbl, "Reorganize rollback") } diff --git a/pkg/ddl/tests/serial/BUILD.bazel b/pkg/ddl/tests/serial/BUILD.bazel index 0566b1e160acb..9fdfed5608c2a 100644 --- a/pkg/ddl/tests/serial/BUILD.bazel +++ b/pkg/ddl/tests/serial/BUILD.bazel @@ -22,7 +22,7 @@ go_test( "//pkg/meta", "//pkg/meta/autoid", "//pkg/meta/model", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/parser/mysql", "//pkg/session", "//pkg/sessionctx/variable", diff --git a/pkg/ddl/tests/serial/serial_test.go b/pkg/ddl/tests/serial/serial_test.go index 1455073ce632c..ec753f0d958c4 100644 --- a/pkg/ddl/tests/serial/serial_test.go +++ b/pkg/ddl/tests/serial/serial_test.go @@ -35,7 +35,7 @@ import ( "github.com/pingcap/tidb/pkg/meta" "github.com/pingcap/tidb/pkg/meta/autoid" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "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" @@ -120,7 +120,7 @@ func TestCreateTableWithLike(t *testing.T) { tk.MustQuery("select * from t1").Check(testkit.Rows("1 11")) tk.MustQuery("select * from t2").Check(testkit.Rows("1 12")) is := domain.GetDomain(tk.Session()).InfoSchema() - tbl1, err := is.TableByName(context.Background(), pmodel.NewCIStr("ctwl_db"), pmodel.NewCIStr("t1")) + tbl1, err := is.TableByName(context.Background(), ast.NewCIStr("ctwl_db"), ast.NewCIStr("t1")) require.NoError(t, err) tbl1Info := tbl1.Meta() require.Nil(t, tbl1Info.ForeignKeys) @@ -128,7 +128,7 @@ func TestCreateTableWithLike(t *testing.T) { col := tbl1Info.Columns[0] hasNotNull := mysql.HasNotNullFlag(col.GetFlag()) require.True(t, hasNotNull) - tbl2, err := is.TableByName(context.Background(), pmodel.NewCIStr("ctwl_db"), pmodel.NewCIStr("t2")) + tbl2, err := is.TableByName(context.Background(), ast.NewCIStr("ctwl_db"), ast.NewCIStr("t2")) require.NoError(t, err) tbl2Info := tbl2.Meta() require.Nil(t, tbl2Info.ForeignKeys) @@ -142,7 +142,7 @@ func TestCreateTableWithLike(t *testing.T) { tk.MustExec("insert into t1 set c2=11") tk.MustQuery("select * from t1").Check(testkit.Rows("1 11")) is = domain.GetDomain(tk.Session()).InfoSchema() - tbl1, err = is.TableByName(context.Background(), pmodel.NewCIStr("ctwl_db1"), pmodel.NewCIStr("t1")) + tbl1, err = is.TableByName(context.Background(), ast.NewCIStr("ctwl_db1"), ast.NewCIStr("t1")) require.NoError(t, err) require.Nil(t, tbl1.Meta().ForeignKeys) @@ -278,7 +278,7 @@ func TestCreateTableWithLikeAtTemporaryMode(t *testing.T) { tk.MustExec(`create global temporary table test_gv_ddl_temp like test_gv_ddl on commit delete rows;`) defer tk.MustExec("drop table if exists test_gv_ddl_temp, test_gv_ddl") is := sessiontxn.GetTxnManager(tk.Session()).GetTxnInfoSchema() - table, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("test_gv_ddl")) + table, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("test_gv_ddl")) require.NoError(t, err) testCases := []struct { generatedExprString string @@ -307,7 +307,7 @@ func TestCreateTableWithLikeAtTemporaryMode(t *testing.T) { defer tk.MustExec("drop table if exists test_foreign_key, t1") tk.MustExec("create global temporary table test_foreign_key_temp like test_foreign_key on commit delete rows") is = sessiontxn.GetTxnManager(tk.Session()).GetTxnInfoSchema() - table, err = is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("test_foreign_key_temp")) + table, err = is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("test_foreign_key_temp")) require.NoError(t, err) tableInfo := table.Meta() require.Equal(t, 0, len(tableInfo.ForeignKeys)) @@ -391,7 +391,7 @@ func TestCreateTableWithLikeAtTemporaryMode(t *testing.T) { tk.MustExec("create table foreign_key_table2 (c int,d int,foreign key (d) references foreign_key_table1 (b))") tk.MustExec("create temporary table foreign_key_tmp like foreign_key_table2") is = sessiontxn.GetTxnManager(tk.Session()).GetTxnInfoSchema() - table, err = is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("foreign_key_tmp")) + table, err = is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("foreign_key_tmp")) require.NoError(t, err) tableInfo = table.Meta() require.Equal(t, 0, len(tableInfo.ForeignKeys)) @@ -1238,7 +1238,7 @@ func TestGetReverseKey(t *testing.T) { // Get table ID for split. is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("db_get"), pmodel.NewCIStr("test_get")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("db_get"), ast.NewCIStr("test_get")) require.NoError(t, err) // Split the table. tableStart := tablecodec.GenTableRecordPrefix(tbl.Meta().ID) diff --git a/pkg/ddl/tests/tiflash/BUILD.bazel b/pkg/ddl/tests/tiflash/BUILD.bazel index c965bf3b7b88f..e9ae74b337949 100644 --- a/pkg/ddl/tests/tiflash/BUILD.bazel +++ b/pkg/ddl/tests/tiflash/BUILD.bazel @@ -19,7 +19,7 @@ go_test( "//pkg/domain/infosync", "//pkg/kv", "//pkg/meta/model", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/session", "//pkg/sessionctx", "//pkg/store/gcworker", diff --git a/pkg/ddl/tests/tiflash/ddl_tiflash_test.go b/pkg/ddl/tests/tiflash/ddl_tiflash_test.go index 52cd7e523c530..1486624f87fd6 100644 --- a/pkg/ddl/tests/tiflash/ddl_tiflash_test.go +++ b/pkg/ddl/tests/tiflash/ddl_tiflash_test.go @@ -36,7 +36,7 @@ import ( "github.com/pingcap/tidb/pkg/domain/infosync" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/session" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/store/gcworker" @@ -148,7 +148,7 @@ func (s *tiflashContext) CheckFlashback(tk *testkit.TestKit, t *testing.T) { time.Sleep(ddl.PollTiFlashInterval * 3) CheckTableAvailable(s.dom, t, 1, []string{}) - tb, err := s.dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("ddltiflash")) + tb, err := s.dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("ddltiflash")) require.NoError(t, err) require.NotNil(t, tb) if tb.Meta().Partition != nil { @@ -282,7 +282,7 @@ func TestTiFlashReplicaPartitionTableNormal(t *testing.T) { tk.MustExec("drop table if exists ddltiflash") tk.MustExec("create table ddltiflash(z int) PARTITION BY RANGE(z) (PARTITION p0 VALUES LESS THAN (10),PARTITION p1 VALUES LESS THAN (20), PARTITION p2 VALUES LESS THAN (30))") - tb, err := s.dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("ddltiflash")) + tb, err := s.dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("ddltiflash")) require.NoError(t, err) replica := tb.Meta().TiFlashReplica require.Nil(t, replica) @@ -295,7 +295,7 @@ func TestTiFlashReplicaPartitionTableNormal(t *testing.T) { // Should get schema again CheckTableAvailable(s.dom, t, 1, []string{}) - tb2, err := s.dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("ddltiflash")) + tb2, err := s.dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("ddltiflash")) require.NoError(t, err) require.NotNil(t, tb2) pi := tb2.Meta().GetPartitionInfo() @@ -335,7 +335,7 @@ func TestTiFlashReplicaPartitionTableBlock(t *testing.T) { }() tk.MustExec(fmt.Sprintf("ALTER TABLE ddltiflash ADD PARTITION (PARTITION pn VALUES LESS THAN (%v))", lessThan)) - tb, err := s.dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("ddltiflash")) + tb, err := s.dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("ddltiflash")) require.NoError(t, err) pi := tb.Meta().GetPartitionInfo() require.NotNil(t, pi) @@ -373,14 +373,14 @@ func TestTiFlashReplicaAvailable(t *testing.T) { CheckTableAvailableWithTableName(s.dom, t, 1, []string{}, "test", "ddltiflash2") s.CheckFlashback(tk, t) - tb, err := s.dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("ddltiflash")) + tb, err := s.dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("ddltiflash")) require.NoError(t, err) r, ok := s.tiflash.GetPlacementRule(infosync.MakeRuleID(tb.Meta().ID)) require.NotNil(t, r) require.True(t, ok) tk.MustExec("alter table ddltiflash set tiflash replica 0") time.Sleep(ddl.PollTiFlashInterval * RoundToBeAvailable) - tb, err = s.dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("ddltiflash")) + tb, err = s.dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("ddltiflash")) require.NoError(t, err) replica := tb.Meta().TiFlashReplica require.Nil(t, replica) @@ -483,7 +483,7 @@ func TestTiFlashFlashbackCluster(t *testing.T) { } func CheckTableAvailableWithTableName(dom *domain.Domain, t *testing.T, count uint64, labels []string, db string, table string) { - tb, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr(db), pmodel.NewCIStr(table)) + tb, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr(db), ast.NewCIStr(table)) require.NoError(t, err) replica := tb.Meta().TiFlashReplica require.NotNil(t, replica) @@ -497,7 +497,7 @@ func CheckTableAvailable(dom *domain.Domain, t *testing.T, count uint64, labels } func CheckTableNoReplica(dom *domain.Domain, t *testing.T, db string, table string) { - tb, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr(db), pmodel.NewCIStr(table)) + tb, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr(db), ast.NewCIStr(table)) require.NoError(t, err) replica := tb.Meta().TiFlashReplica require.Nil(t, replica) @@ -561,7 +561,7 @@ func TestSetPlacementRuleNormal(t *testing.T) { tk.MustExec("drop table if exists ddltiflash") tk.MustExec("create table ddltiflash(z int)") tk.MustExec("alter table ddltiflash set tiflash replica 1 location labels 'a','b'") - tb, err := s.dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("ddltiflash")) + tb, err := s.dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("ddltiflash")) require.NoError(t, err) expectRule := infosync.MakeNewRule(tb.Meta().ID, 1, []string{"a", "b"}) res := s.tiflash.CheckPlacementRule(expectRule) @@ -616,7 +616,7 @@ func TestSetPlacementRuleWithGCWorker(t *testing.T) { tk.MustExec("drop table if exists ddltiflash_gc") tk.MustExec("create table ddltiflash_gc(z int)") tk.MustExec("alter table ddltiflash_gc set tiflash replica 1 location labels 'a','b'") - tb, err := s.dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("ddltiflash_gc")) + tb, err := s.dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("ddltiflash_gc")) require.NoError(t, err) expectRule := infosync.MakeNewRule(tb.Meta().ID, 1, []string{"a", "b"}) @@ -647,7 +647,7 @@ func TestSetPlacementRuleFail(t *testing.T) { s.tiflash.PdSwitch(true) }() tk.MustExec("alter table ddltiflash set tiflash replica 1") - tb, err := s.dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("ddltiflash")) + tb, err := s.dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("ddltiflash")) require.NoError(t, err) expectRule := infosync.MakeNewRule(tb.Meta().ID, 1, []string{}) @@ -755,7 +755,7 @@ func TestTiFlashBackoff(t *testing.T) { // 1, 1.5, 2.25, 3.375, 5.5625 // (1), 1, 1, 2, 3, 5 time.Sleep(ddl.PollTiFlashInterval * 5) - tb, err := s.dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("ddltiflash")) + tb, err := s.dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("ddltiflash")) require.NoError(t, err) require.NotNil(t, tb) require.False(t, tb.Meta().TiFlashReplica.Available) @@ -764,7 +764,7 @@ func TestTiFlashBackoff(t *testing.T) { require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/ddl/PollTiFlashReplicaStatusReplaceCurAvailableValue")) time.Sleep(ddl.PollTiFlashInterval * 3) - tb, err = s.dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("ddltiflash")) + tb, err = s.dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("ddltiflash")) require.NoError(t, err) require.NotNil(t, tb) require.True(t, tb.Meta().TiFlashReplica.Available) @@ -872,7 +872,7 @@ func TestTiFlashBatchRateLimiter(t *testing.T) { check := func(expected int, total int) { cnt := 0 for i := 0; i < total; i++ { - tb, err := s.dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("tiflash_ddl_limit"), pmodel.NewCIStr(fmt.Sprintf("t%v", i))) + tb, err := s.dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("tiflash_ddl_limit"), ast.NewCIStr(fmt.Sprintf("t%v", i))) require.NoError(t, err) if tb.Meta().TiFlashReplica != nil { cnt++ @@ -974,7 +974,7 @@ func TestTiFlashProgress(t *testing.T) { tk.MustExec("create database tiflash_d") tk.MustExec("create table tiflash_d.t(z int)") tk.MustExec("alter table tiflash_d.t set tiflash replica 1") - tb, err := s.dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("tiflash_d"), pmodel.NewCIStr("t")) + tb, err := s.dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("tiflash_d"), ast.NewCIStr("t")) require.NoError(t, err) require.NotNil(t, tb) mustExist := func(tid int64) { @@ -994,13 +994,13 @@ func TestTiFlashProgress(t *testing.T) { tk.MustExec("truncate table tiflash_d.t") mustAbsent(tb.Meta().ID) - tb, _ = s.dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("tiflash_d"), pmodel.NewCIStr("t")) + tb, _ = s.dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("tiflash_d"), ast.NewCIStr("t")) infosync.UpdateTiFlashProgressCache(tb.Meta().ID, 5.0) tk.MustExec("alter table tiflash_d.t set tiflash replica 0") mustAbsent(tb.Meta().ID) tk.MustExec("alter table tiflash_d.t set tiflash replica 1") - tb, _ = s.dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("tiflash_d"), pmodel.NewCIStr("t")) + tb, _ = s.dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("tiflash_d"), ast.NewCIStr("t")) infosync.UpdateTiFlashProgressCache(tb.Meta().ID, 5.0) tk.MustExec("drop table tiflash_d.t") mustAbsent(tb.Meta().ID) @@ -1017,7 +1017,7 @@ func TestTiFlashProgressForPartitionTable(t *testing.T) { tk.MustExec("create database tiflash_d") tk.MustExec("create table tiflash_d.t(z int) PARTITION BY RANGE(z) (PARTITION p0 VALUES LESS THAN (10))") tk.MustExec("alter table tiflash_d.t set tiflash replica 1") - tb, err := s.dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("tiflash_d"), pmodel.NewCIStr("t")) + tb, err := s.dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("tiflash_d"), ast.NewCIStr("t")) require.NoError(t, err) require.NotNil(t, tb) mustExist := func(tid int64) { @@ -1037,13 +1037,13 @@ func TestTiFlashProgressForPartitionTable(t *testing.T) { tk.MustExec("truncate table tiflash_d.t") mustAbsent(tb.Meta().Partition.Definitions[0].ID) - tb, _ = s.dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("tiflash_d"), pmodel.NewCIStr("t")) + tb, _ = s.dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("tiflash_d"), ast.NewCIStr("t")) infosync.UpdateTiFlashProgressCache(tb.Meta().Partition.Definitions[0].ID, 5.0) tk.MustExec("alter table tiflash_d.t set tiflash replica 0") mustAbsent(tb.Meta().Partition.Definitions[0].ID) tk.MustExec("alter table tiflash_d.t set tiflash replica 1") - tb, _ = s.dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("tiflash_d"), pmodel.NewCIStr("t")) + tb, _ = s.dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("tiflash_d"), ast.NewCIStr("t")) infosync.UpdateTiFlashProgressCache(tb.Meta().Partition.Definitions[0].ID, 5.0) tk.MustExec("drop table tiflash_d.t") mustAbsent(tb.Meta().Partition.Definitions[0].ID) @@ -1082,7 +1082,7 @@ func TestTiFlashFailureProgressAfterAvailable(t *testing.T) { time.Sleep(ddl.PollTiFlashInterval * RoundToBeAvailable * 3) CheckTableAvailable(s.dom, t, 1, []string{}) - tb, err := s.dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("ddltiflash")) + tb, err := s.dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("ddltiflash")) require.NoError(t, err) require.NotNil(t, tb) // after available, progress should can be updated. @@ -1133,7 +1133,7 @@ func TestTiFlashProgressAfterAvailable(t *testing.T) { time.Sleep(ddl.PollTiFlashInterval * RoundToBeAvailable * 3) CheckTableAvailable(s.dom, t, 1, []string{}) - tb, err := s.dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("ddltiflash")) + tb, err := s.dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("ddltiflash")) require.NoError(t, err) require.NotNil(t, tb) // after available, progress should can be updated. @@ -1162,7 +1162,7 @@ func TestTiFlashProgressAfterAvailableForPartitionTable(t *testing.T) { time.Sleep(ddl.PollTiFlashInterval * RoundToBeAvailable * 3) CheckTableAvailable(s.dom, t, 1, []string{}) - tb, err := s.dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("ddltiflash")) + tb, err := s.dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("ddltiflash")) require.NoError(t, err) require.NotNil(t, tb) // after available, progress should can be updated. @@ -1191,7 +1191,7 @@ func TestTiFlashProgressCache(t *testing.T) { time.Sleep(ddl.PollTiFlashInterval * RoundToBeAvailable * 3) CheckTableAvailable(s.dom, t, 1, []string{}) - tb, err := s.dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("ddltiflash")) + tb, err := s.dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("ddltiflash")) require.NoError(t, err) require.NotNil(t, tb) infosync.UpdateTiFlashProgressCache(tb.Meta().ID, 0) @@ -1226,7 +1226,7 @@ func TestTiFlashProgressAvailableList(t *testing.T) { // After available, reset TiFlash sync status. for i := 0; i < tableCount; i++ { var err error - tbls[i], err = s.dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr(tableNames[i])) + tbls[i], err = s.dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr(tableNames[i])) require.NoError(t, err) require.NotNil(t, tbls[i]) s.tiflash.ResetSyncStatus(int(tbls[i].Meta().ID), false) @@ -1284,7 +1284,7 @@ func TestTiFlashAvailableAfterResetReplica(t *testing.T) { CheckTableAvailable(s.dom, t, 2, []string{}) tk.MustExec("alter table ddltiflash set tiflash replica 0") - tb, err := s.dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("ddltiflash")) + tb, err := s.dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("ddltiflash")) require.NoError(t, err) require.NotNil(t, tb) require.Nil(t, tb.Meta().TiFlashReplica) @@ -1299,7 +1299,7 @@ func TestTiFlashPartitionNotAvailable(t *testing.T) { tk.MustExec("drop table if exists ddltiflash") tk.MustExec("create table ddltiflash(z int) PARTITION BY RANGE(z) (PARTITION p0 VALUES LESS THAN (10))") - tb, err := s.dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("ddltiflash")) + tb, err := s.dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("ddltiflash")) require.NoError(t, err) require.NotNil(t, tb) @@ -1307,7 +1307,7 @@ func TestTiFlashPartitionNotAvailable(t *testing.T) { s.tiflash.ResetSyncStatus(int(tb.Meta().Partition.Definitions[0].ID), false) time.Sleep(ddl.PollTiFlashInterval * RoundToBeAvailable * 3) - tb, err = s.dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("ddltiflash")) + tb, err = s.dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("ddltiflash")) require.NoError(t, err) require.NotNil(t, tb) replica := tb.Meta().TiFlashReplica @@ -1317,7 +1317,7 @@ func TestTiFlashPartitionNotAvailable(t *testing.T) { s.tiflash.ResetSyncStatus(int(tb.Meta().Partition.Definitions[0].ID), true) time.Sleep(ddl.PollTiFlashInterval * RoundToBeAvailable * 3) - tb, err = s.dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("ddltiflash")) + tb, err = s.dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("ddltiflash")) require.NoError(t, err) require.NotNil(t, tb) replica = tb.Meta().TiFlashReplica @@ -1345,7 +1345,7 @@ func TestTiFlashAvailableAfterAddPartition(t *testing.T) { time.Sleep(ddl.PollTiFlashInterval * RoundToBeAvailable * 3) CheckTableAvailable(s.dom, t, 1, []string{}) - tb, err := s.dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("ddltiflash")) + tb, err := s.dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("ddltiflash")) require.NoError(t, err) require.NotNil(t, tb) @@ -1361,7 +1361,7 @@ func TestTiFlashAvailableAfterAddPartition(t *testing.T) { tk.MustExec("ALTER TABLE ddltiflash ADD PARTITION (PARTITION pn VALUES LESS THAN (20))") time.Sleep(ddl.PollTiFlashInterval * RoundToBeAvailable * 3) CheckTableAvailable(s.dom, t, 1, []string{}) - tb, err = s.dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("ddltiflash")) + tb, err = s.dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("ddltiflash")) require.NoError(t, err) pi := tb.Meta().GetPartitionInfo() require.NotNil(t, pi) diff --git a/pkg/ddl/testutil/BUILD.bazel b/pkg/ddl/testutil/BUILD.bazel index 0aa782f2e1a93..1b0753d9d140c 100644 --- a/pkg/ddl/testutil/BUILD.bazel +++ b/pkg/ddl/testutil/BUILD.bazel @@ -14,7 +14,7 @@ go_library( "//pkg/domain", "//pkg/kv", "//pkg/meta/model", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/session", "//pkg/session/types", "//pkg/sessiontxn", diff --git a/pkg/ddl/testutil/testutil.go b/pkg/ddl/testutil/testutil.go index 9d516c25eff99..acda723a44105 100644 --- a/pkg/ddl/testutil/testutil.go +++ b/pkg/ddl/testutil/testutil.go @@ -24,7 +24,7 @@ import ( "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/session" sessiontypes "github.com/pingcap/tidb/pkg/session/types" "github.com/pingcap/tidb/pkg/sessiontxn" @@ -72,7 +72,7 @@ func ExecMultiSQLInGoroutine(s kv.Storage, dbName string, multiSQL []string, don // ExtractAllTableHandles extracts all handles of a given table. func ExtractAllTableHandles(se sessiontypes.Session, dbName, tbName string) ([]int64, error) { dom := domain.GetDomain(se) - tbl, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr(dbName), pmodel.NewCIStr(tbName)) + tbl, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr(dbName), ast.NewCIStr(tbName)) if err != nil { return nil, err } @@ -92,7 +92,7 @@ func ExtractAllTableHandles(se sessiontypes.Session, dbName, tbName string) ([]i // FindIdxInfo is to get IndexInfo by index name. func FindIdxInfo(dom *domain.Domain, dbName, tbName, idxName string) *model.IndexInfo { - tbl, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr(dbName), pmodel.NewCIStr(tbName)) + tbl, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr(dbName), ast.NewCIStr(tbName)) if err != nil { logutil.DDLLogger().Warn("cannot find table", zap.String("dbName", dbName), zap.String("tbName", tbName)) return nil diff --git a/pkg/ddl/tiflash_replica_test.go b/pkg/ddl/tiflash_replica_test.go index 09e184a99518e..fc1327b766c28 100644 --- a/pkg/ddl/tiflash_replica_test.go +++ b/pkg/ddl/tiflash_replica_test.go @@ -31,8 +31,8 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta" "github.com/pingcap/tidb/pkg/meta/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/auth" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/server" "github.com/pingcap/tidb/pkg/tablecodec" @@ -188,7 +188,7 @@ func TestInfoSchemaForTiFlashReplica(t *testing.T) { tk.MustExec("alter table t set tiflash replica 2 location labels 'a','b';") tk.MustQuery("select TABLE_SCHEMA,TABLE_NAME,REPLICA_COUNT,LOCATION_LABELS,AVAILABLE,PROGRESS from information_schema.tiflash_replica").Check(testkit.Rows("test t 2 a,b 0 0")) dom := domain.GetDomain(tk.Session()) - tbl, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tbl.Meta().TiFlashReplica.Available = true updateTableMeta(t, store, tbl.Meta().DBID, tbl.Meta()) @@ -241,7 +241,7 @@ func TestSetTableFlashReplicaForSystemTable(t *testing.T) { for _, one := range sysTables { _, err := tk.Exec(fmt.Sprintf("alter table `%s` set tiflash replica 1", one)) if db == "MySQL" || db == "SYS" { - tbl, err1 := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr(db), pmodel.NewCIStr(one)) + tbl, err1 := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr(db), ast.NewCIStr(one)) require.NoError(t, err1) if tbl.Meta().View != nil { require.ErrorIs(t, err, dbterror.ErrWrongObject) @@ -412,7 +412,7 @@ func TestTruncateTable2(t *testing.T) { tk.MustExec("create table truncate_table (c1 int, c2 int)") tk.MustExec("insert truncate_table values (1, 1), (2, 2)") is := domain.GetDomain(tk.Session()).InfoSchema() - oldTblInfo, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("truncate_table")) + oldTblInfo, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("truncate_table")) require.NoError(t, err) oldTblID := oldTblInfo.Meta().ID @@ -422,7 +422,7 @@ func TestTruncateTable2(t *testing.T) { tk.MustQuery("select * from truncate_table").Check(testkit.Rows("3 3", "4 4")) is = domain.GetDomain(tk.Session()).InfoSchema() - newTblInfo, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("truncate_table")) + newTblInfo, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("truncate_table")) require.NoError(t, err) require.Greater(t, newTblInfo.Meta().ID, oldTblID) diff --git a/pkg/ddl/ttl.go b/pkg/ddl/ttl.go index 3b1236a98ff1f..c8ac853c73675 100644 --- a/pkg/ddl/ttl.go +++ b/pkg/ddl/ttl.go @@ -23,7 +23,6 @@ import ( "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/format" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/ttl/cache" "github.com/pingcap/tidb/pkg/types" @@ -95,7 +94,7 @@ func onTTLInfoChange(jobCtx *jobContext, job *model.Job) (ver int64, err error) // checkTTLInfoValid checks the TTL settings for a table. // The argument `isForForeignKeyCheck` is used to check the table should not be referenced by foreign key. // If `isForForeignKeyCheck` is `nil`, it will skip the foreign key check. -func checkTTLInfoValid(schema pmodel.CIStr, tblInfo *model.TableInfo, foreignKeyCheckIs infoschemactx.MetaOnlyInfoSchema) error { +func checkTTLInfoValid(schema ast.CIStr, tblInfo *model.TableInfo, foreignKeyCheckIs infoschemactx.MetaOnlyInfoSchema) error { if tblInfo.TempTableType != model.TempTableNone { return dbterror.ErrTempTableNotAllowedWithTTL } diff --git a/pkg/ddl/ttl_test.go b/pkg/ddl/ttl_test.go index b156c697b8375..323a306749914 100644 --- a/pkg/ddl/ttl_test.go +++ b/pkg/ddl/ttl_test.go @@ -19,7 +19,6 @@ import ( "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/ast" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/stretchr/testify/assert" ) @@ -46,13 +45,13 @@ func Test_getTTLInfoInOptions(t *testing.T) { []*ast.TableOption{ { Tp: ast.TableOptionTTL, - ColumnName: &ast.ColumnName{Name: pmodel.NewCIStr("test_column")}, + ColumnName: &ast.ColumnName{Name: ast.NewCIStr("test_column")}, Value: ast.NewValueExpr(5, "", ""), TimeUnitValue: &ast.TimeUnitExpr{Unit: ast.TimeUnitYear}, }, }, &model.TTLInfo{ - ColumnName: pmodel.NewCIStr("test_column"), + ColumnName: ast.NewCIStr("test_column"), IntervalExprStr: "5", IntervalTimeUnit: int(ast.TimeUnitYear), Enable: true, @@ -70,13 +69,13 @@ func Test_getTTLInfoInOptions(t *testing.T) { }, { Tp: ast.TableOptionTTL, - ColumnName: &ast.ColumnName{Name: pmodel.NewCIStr("test_column")}, + ColumnName: &ast.ColumnName{Name: ast.NewCIStr("test_column")}, Value: ast.NewValueExpr(5, "", ""), TimeUnitValue: &ast.TimeUnitExpr{Unit: ast.TimeUnitYear}, }, }, &model.TTLInfo{ - ColumnName: pmodel.NewCIStr("test_column"), + ColumnName: ast.NewCIStr("test_column"), IntervalExprStr: "5", IntervalTimeUnit: int(ast.TimeUnitYear), Enable: false, @@ -94,7 +93,7 @@ func Test_getTTLInfoInOptions(t *testing.T) { }, { Tp: ast.TableOptionTTL, - ColumnName: &ast.ColumnName{Name: pmodel.NewCIStr("test_column")}, + ColumnName: &ast.ColumnName{Name: ast.NewCIStr("test_column")}, Value: ast.NewValueExpr(5, "", ""), TimeUnitValue: &ast.TimeUnitExpr{Unit: ast.TimeUnitYear}, }, @@ -104,7 +103,7 @@ func Test_getTTLInfoInOptions(t *testing.T) { }, }, &model.TTLInfo{ - ColumnName: pmodel.NewCIStr("test_column"), + ColumnName: ast.NewCIStr("test_column"), IntervalExprStr: "5", IntervalTimeUnit: int(ast.TimeUnitYear), Enable: true, @@ -118,7 +117,7 @@ func Test_getTTLInfoInOptions(t *testing.T) { []*ast.TableOption{ { Tp: ast.TableOptionTTL, - ColumnName: &ast.ColumnName{Name: pmodel.NewCIStr("test_column")}, + ColumnName: &ast.ColumnName{Name: ast.NewCIStr("test_column")}, Value: ast.NewValueExpr(5, "", ""), TimeUnitValue: &ast.TimeUnitExpr{Unit: ast.TimeUnitYear}, }, @@ -128,7 +127,7 @@ func Test_getTTLInfoInOptions(t *testing.T) { }, }, &model.TTLInfo{ - ColumnName: pmodel.NewCIStr("test_column"), + ColumnName: ast.NewCIStr("test_column"), IntervalExprStr: "5", IntervalTimeUnit: int(ast.TimeUnitYear), Enable: true, diff --git a/pkg/disttask/importinto/BUILD.bazel b/pkg/disttask/importinto/BUILD.bazel index b9a44514945cb..52d7e0be1de21 100644 --- a/pkg/disttask/importinto/BUILD.bazel +++ b/pkg/disttask/importinto/BUILD.bazel @@ -113,7 +113,6 @@ go_test( "//pkg/meta/model", "//pkg/parser", "//pkg/parser/ast", - "//pkg/parser/model", "//pkg/session", "//pkg/testkit", "//pkg/testkit/testfailpoint", diff --git a/pkg/disttask/importinto/planner_test.go b/pkg/disttask/importinto/planner_test.go index ac09f98530e54..13d4bdfecaab4 100644 --- a/pkg/disttask/importinto/planner_test.go +++ b/pkg/disttask/importinto/planner_test.go @@ -31,7 +31,7 @@ import ( "github.com/pingcap/tidb/pkg/lightning/backend/external" "github.com/pingcap/tidb/pkg/meta/autoid" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/stretchr/testify/require" pd "github.com/tikv/pd/client" "github.com/tikv/pd/client/opt" @@ -60,7 +60,7 @@ func TestToPhysicalPlan(t *testing.T) { Plan: importer.Plan{ DBName: "db", TableInfo: &model.TableInfo{ - Name: pmodel.NewCIStr("tb"), + Name: ast.NewCIStr("tb"), }, }, Stmt: `IMPORT INTO db.tb FROM 'gs://test-load/*.csv?endpoint=xxx'`, diff --git a/pkg/disttask/importinto/scheduler_testkit_test.go b/pkg/disttask/importinto/scheduler_testkit_test.go index dff9de8d6957f..460ceff6816e2 100644 --- a/pkg/disttask/importinto/scheduler_testkit_test.go +++ b/pkg/disttask/importinto/scheduler_testkit_test.go @@ -30,7 +30,7 @@ import ( "github.com/pingcap/tidb/pkg/executor/importer" "github.com/pingcap/tidb/pkg/lightning/backend/external" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/testkit/testfailpoint" "github.com/stretchr/testify/require" @@ -63,7 +63,7 @@ func TestSchedulerExtLocalSort(t *testing.T) { Plan: importer.Plan{ DBName: "test", TableInfo: &model.TableInfo{ - Name: pmodel.NewCIStr("t"), + Name: ast.NewCIStr("t"), }, DisableTiKVImportMode: true, }, @@ -200,7 +200,7 @@ func TestSchedulerExtGlobalSort(t *testing.T) { Format: "csv", DBName: "test", TableInfo: &model.TableInfo{ - Name: pmodel.NewCIStr("t"), + Name: ast.NewCIStr("t"), State: model.StatePublic, }, DisableTiKVImportMode: true, diff --git a/pkg/disttask/importinto/task_executor_testkit_test.go b/pkg/disttask/importinto/task_executor_testkit_test.go index 57c4f20450a2f..cc19c18f7260a 100644 --- a/pkg/disttask/importinto/task_executor_testkit_test.go +++ b/pkg/disttask/importinto/task_executor_testkit_test.go @@ -24,7 +24,7 @@ import ( "github.com/pingcap/tidb/pkg/disttask/importinto" "github.com/pingcap/tidb/pkg/executor/importer" "github.com/pingcap/tidb/pkg/lightning/config" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/session" "github.com/pingcap/tidb/pkg/testkit" "github.com/stretchr/testify/require" @@ -57,7 +57,7 @@ func TestPostProcessStepExecutor(t *testing.T) { dom, err := session.GetDomain(store) require.NoError(t, err) - table, err := dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + table, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) taskMeta := &importinto.TaskMeta{ Plan: importer.Plan{ diff --git a/pkg/domain/BUILD.bazel b/pkg/domain/BUILD.bazel index da7b2683981f7..8164e05d09181 100644 --- a/pkg/domain/BUILD.bazel +++ b/pkg/domain/BUILD.bazel @@ -51,7 +51,6 @@ go_library( "//pkg/owner", "//pkg/parser", "//pkg/parser/ast", - "//pkg/parser/model", "//pkg/parser/mysql", "//pkg/parser/terror", "//pkg/planner/core/metrics", @@ -152,7 +151,6 @@ go_test( "//pkg/metrics", "//pkg/parser/ast", "//pkg/parser/auth", - "//pkg/parser/model", "//pkg/parser/mysql", "//pkg/parser/terror", "//pkg/server", diff --git a/pkg/domain/domain_test.go b/pkg/domain/domain_test.go index e4a25577c20d4..3a4665c376eaa 100644 --- a/pkg/domain/domain_test.go +++ b/pkg/domain/domain_test.go @@ -33,7 +33,6 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/metrics" "github.com/pingcap/tidb/pkg/parser/ast" - "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/store/mockstore" @@ -137,7 +136,7 @@ func TestInfo(t *testing.T) { require.True(t, syncerStarted) stmt := &ast.CreateDatabaseStmt{ - Name: model.NewCIStr("aaa"), + Name: ast.NewCIStr("aaa"), // Make sure loading schema is normal. Options: []*ast.DatabaseOption{ { diff --git a/pkg/domain/extract.go b/pkg/domain/extract.go index 7f0032de93923..04147b3fa9432 100644 --- a/pkg/domain/extract.go +++ b/pkg/domain/extract.go @@ -32,7 +32,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/config" "github.com/pingcap/tidb/pkg/kv" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util" @@ -212,11 +212,11 @@ func (w *extractWorker) handleTableNames(tableNames string, record *stmtSummaryH case util.PerformanceSchemaName.L, util.InformationSchemaName.L, util.MetricSchemaName.L, "mysql": return false, nil } - exists := is.TableExists(model.NewCIStr(dbName), model.NewCIStr(tblName)) + exists := is.TableExists(ast.NewCIStr(dbName), ast.NewCIStr(tblName)) if !exists { return false, nil } - t, err := is.TableByName(w.ctx, model.NewCIStr(dbName), model.NewCIStr(tblName)) + t, err := is.TableByName(w.ctx, ast.NewCIStr(dbName), ast.NewCIStr(tblName)) if err != nil { return false, err } @@ -269,13 +269,13 @@ func (w *extractWorker) handleIsView(ctx context.Context, p *extractPlanPackage) ctx: ctx, executor: w.sctx.GetRestrictedSQLExecutor(), is: is, - curDB: model.NewCIStr(""), + curDB: ast.NewCIStr(""), names: make(map[tableNamePair]struct{}), cteNames: make(map[string]struct{}), } for v := range p.tables { if v.IsView { - v, err := is.TableByName(w.ctx, model.NewCIStr(v.DBName), model.NewCIStr(v.TableName)) + v, err := is.TableByName(w.ctx, ast.NewCIStr(v.DBName), ast.NewCIStr(v.TableName)) if err != nil { return err } diff --git a/pkg/domain/infosync/BUILD.bazel b/pkg/domain/infosync/BUILD.bazel index 571f9549c73e6..edccd8a52482f 100644 --- a/pkg/domain/infosync/BUILD.bazel +++ b/pkg/domain/infosync/BUILD.bazel @@ -71,7 +71,7 @@ go_test( "//pkg/ddl/util", "//pkg/keyspace", "//pkg/meta/model", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/testkit/testsetup", "//pkg/util", "@com_github_pingcap_failpoint//:failpoint", diff --git a/pkg/domain/infosync/info_test.go b/pkg/domain/infosync/info_test.go index 3f85c8289c147..9be9326e2ab3a 100644 --- a/pkg/domain/infosync/info_test.go +++ b/pkg/domain/infosync/info_test.go @@ -30,7 +30,7 @@ import ( "github.com/pingcap/tidb/pkg/ddl/util" "github.com/pingcap/tidb/pkg/keyspace" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/testkit/testsetup" util2 "github.com/pingcap/tidb/pkg/util" "github.com/stretchr/testify/require" @@ -259,12 +259,12 @@ func TestTiFlashManager(t *testing.T) { ConfigureTiFlashPDForPartitions(true, &[]model.PartitionDefinition{ { ID: 2, - Name: pmodel.NewCIStr("p1"), + Name: ast.NewCIStr("p1"), LessThan: []string{}, }, { ID: 3, - Name: pmodel.NewCIStr("p2"), + Name: ast.NewCIStr("p2"), LessThan: []string{}, }, }, 3, &[]string{}, 100) diff --git a/pkg/domain/plan_replayer_dump.go b/pkg/domain/plan_replayer_dump.go index 9a2f5076d16d1..5d0f1fa1b4492 100644 --- a/pkg/domain/plan_replayer_dump.go +++ b/pkg/domain/plan_replayer_dump.go @@ -31,7 +31,6 @@ import ( domain_metrics "github.com/pingcap/tidb/pkg/domain/metrics" "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/parser/ast" - "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/statistics" @@ -91,7 +90,7 @@ type tableNameExtractor struct { ctx context.Context executor sqlexec.RestrictedSQLExecutor is infoschema.InfoSchema - curDB model.CIStr + curDB ast.CIStr names map[tableNamePair]struct{} cteNames map[string]struct{} err error @@ -119,7 +118,7 @@ func (tne *tableNameExtractor) getTablesAndViews() (map[tableNamePair]struct{}, } func findFK(is infoschema.InfoSchema, dbName, tableName string, tableMap map[tableNamePair]struct{}) error { - tblInfo, err := is.TableByName(context.Background(), model.NewCIStr(dbName), model.NewCIStr(tableName)) + tblInfo, err := is.TableByName(context.Background(), ast.NewCIStr(dbName), ast.NewCIStr(tableName)) if err != nil { return err } @@ -301,7 +300,7 @@ func DumpPlanReplayerInfo(ctx context.Context, sctx sessionctx.Context, return err } // Retrieve current DB - dbName := model.NewCIStr(sessionVars.CurrentDB) + dbName := ast.NewCIStr(sessionVars.CurrentDB) do := GetDomain(sctx) // Retrieve all tables @@ -473,8 +472,8 @@ func dumpTiFlashReplica(sctx sessionctx.Context, zw *zip.Writer, pairs map[table is := GetDomain(sctx).InfoSchema() ctx := infoschema.WithRefillOption(context.Background(), false) for pair := range pairs { - dbName := model.NewCIStr(pair.DBName) - tableName := model.NewCIStr(pair.TableName) + dbName := ast.NewCIStr(pair.DBName) + tableName := ast.NewCIStr(pair.TableName) t, err := is.TableByName(ctx, dbName, tableName) if err != nil { logutil.BgLogger().Warn("failed to find table info", zap.Error(err), @@ -527,7 +526,7 @@ func dumpStatsMemStatus(zw *zip.Writer, pairs map[tableNamePair]struct{}, do *Do if pair.IsView { continue } - tbl, err := is.TableByName(ctx, model.NewCIStr(pair.DBName), model.NewCIStr(pair.TableName)) + tbl, err := is.TableByName(ctx, ast.NewCIStr(pair.DBName), ast.NewCIStr(pair.TableName)) if err != nil { return err } @@ -786,7 +785,7 @@ func dumpPlanReplayerExplain(ctx sessionctx.Context, zw *zip.Writer, task *PlanR // extractTableNames extracts table names from the given stmts. func extractTableNames(ctx context.Context, sctx sessionctx.Context, - execStmts []ast.StmtNode, curDB model.CIStr) (map[tableNamePair]struct{}, error) { + execStmts []ast.StmtNode, curDB ast.CIStr) (map[tableNamePair]struct{}, error) { tableExtractor := &tableNameExtractor{ ctx: ctx, executor: sctx.GetRestrictedSQLExecutor(), @@ -807,7 +806,7 @@ func extractTableNames(ctx context.Context, sctx sessionctx.Context, func getStatsForTable(do *Domain, pair tableNamePair, historyStatsTS uint64) (*util.JSONTable, []string, error) { is := do.InfoSchema() h := do.StatsHandle() - tbl, err := is.TableByName(context.Background(), model.NewCIStr(pair.DBName), model.NewCIStr(pair.TableName)) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr(pair.DBName), ast.NewCIStr(pair.TableName)) if err != nil { return nil, nil, err } diff --git a/pkg/domain/ru_stats.go b/pkg/domain/ru_stats.go index bb75c758e2217..9460d5ffb5c6e 100644 --- a/pkg/domain/ru_stats.go +++ b/pkg/domain/ru_stats.go @@ -24,7 +24,7 @@ import ( "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/resourcegroup/runaway" "github.com/pingcap/tidb/pkg/util" "github.com/pingcap/tidb/pkg/util/intest" @@ -188,7 +188,7 @@ func (r *RUStatsWriter) fetchResourceGroupStats(ctx context.Context) ([]meta.Gro infos := r.InfoCache.GetLatest() res := make([]meta.GroupRUStats, 0, len(groups)) for _, g := range groups { - groupInfo, exists := infos.ResourceGroupByName(model.NewCIStr(g.Name)) + groupInfo, exists := infos.ResourceGroupByName(ast.NewCIStr(g.Name)) if !exists { continue } diff --git a/pkg/domain/ru_stats_test.go b/pkg/domain/ru_stats_test.go index ce4b184aebc4d..7c34c2917e06f 100644 --- a/pkg/domain/ru_stats_test.go +++ b/pkg/domain/ru_stats_test.go @@ -23,7 +23,7 @@ import ( "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/testkit" "github.com/stretchr/testify/require" pd "github.com/tikv/pd/client" @@ -67,11 +67,11 @@ func testWriteRUStatisticsTz(t *testing.T, tz *time.Location) { infoGroups := make(map[string]*model.ResourceGroupInfo, 2) infoGroups["default"] = &model.ResourceGroupInfo{ ID: 1, - Name: pmodel.NewCIStr("default"), + Name: ast.NewCIStr("default"), } infoGroups["test"] = &model.ResourceGroupInfo{ ID: 2, - Name: pmodel.NewCIStr("test"), + Name: ast.NewCIStr("test"), } testInfo := &testInfoschema{ groups: infoGroups, @@ -133,7 +133,7 @@ type testInfoschema struct { groups map[string]*model.ResourceGroupInfo } -func (is *testInfoschema) ResourceGroupByName(name pmodel.CIStr) (*model.ResourceGroupInfo, bool) { +func (is *testInfoschema) ResourceGroupByName(name ast.CIStr) (*model.ResourceGroupInfo, bool) { g, ok := is.groups[name.L] return g, ok } diff --git a/pkg/domain/test_helper.go b/pkg/domain/test_helper.go index 74991dca1ea11..178f490f42f24 100644 --- a/pkg/domain/test_helper.go +++ b/pkg/domain/test_helper.go @@ -21,7 +21,7 @@ import ( "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/meta" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/stretchr/testify/require" ) @@ -35,7 +35,7 @@ func (do *Domain) MockInfoCacheAndLoadInfoSchema(is infoschema.InfoSchema) { // MustGetTableInfo returns the table info. Only used in unit tests. func (do *Domain) MustGetTableInfo(t *testing.T, dbName, tableName string) *model.TableInfo { - tbl, err := do.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr(dbName), pmodel.NewCIStr(tableName)) + tbl, err := do.InfoSchema().TableByName(context.Background(), ast.NewCIStr(dbName), ast.NewCIStr(tableName)) require.Nil(t, err) return tbl.Meta() } diff --git a/pkg/executor/BUILD.bazel b/pkg/executor/BUILD.bazel index ebf819b0e41dc..db46b514bda37 100644 --- a/pkg/executor/BUILD.bazel +++ b/pkg/executor/BUILD.bazel @@ -158,7 +158,6 @@ go_library( "//pkg/parser/auth", "//pkg/parser/charset", "//pkg/parser/format", - "//pkg/parser/model", "//pkg/parser/mysql", "//pkg/parser/terror", "//pkg/parser/tidb", @@ -421,7 +420,6 @@ go_test( "//pkg/parser", "//pkg/parser/ast", "//pkg/parser/auth", - "//pkg/parser/model", "//pkg/parser/mysql", "//pkg/parser/terror", "//pkg/planner", diff --git a/pkg/executor/adapter.go b/pkg/executor/adapter.go index adafafa117013..2cd3d6c6ea6ce 100644 --- a/pkg/executor/adapter.go +++ b/pkg/executor/adapter.go @@ -44,7 +44,6 @@ import ( "github.com/pingcap/tidb/pkg/metrics" "github.com/pingcap/tidb/pkg/parser" "github.com/pingcap/tidb/pkg/parser/ast" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/planner" @@ -114,7 +113,7 @@ func (a *recordSet) Fields() []*resolve.ResultField { func colNames2ResultFields(schema *expression.Schema, names []*types.FieldName, defaultDB string) []*resolve.ResultField { rfs := make([]*resolve.ResultField, 0, schema.Len()) - defaultDBCIStr := pmodel.NewCIStr(defaultDB) + defaultDBCIStr := ast.NewCIStr(defaultDB) for i := 0; i < schema.Len(); i++ { dbName := names[i].DBName if dbName.L == "" && names[i].TblName.L != "" { diff --git a/pkg/executor/admin.go b/pkg/executor/admin.go index 926c17c26b790..13d6661f12f5b 100644 --- a/pkg/executor/admin.go +++ b/pkg/executor/admin.go @@ -27,7 +27,6 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/ast" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/terror" plannercore "github.com/pingcap/tidb/pkg/planner/util" "github.com/pingcap/tidb/pkg/table" @@ -406,7 +405,7 @@ func (e *RecoverIndexExec) buildIndexedValues(row chunk.Row, idxVals []types.Dat } if e.cols == nil { - columns, _, err := expression.ColumnInfos2ColumnsAndNames(e.Ctx().GetExprCtx(), pmodel.NewCIStr("mock"), e.table.Meta().Name, e.table.Meta().Columns, e.table.Meta()) + columns, _, err := expression.ColumnInfos2ColumnsAndNames(e.Ctx().GetExprCtx(), ast.NewCIStr("mock"), e.table.Meta().Name, e.table.Meta().Columns, e.table.Meta()) if err != nil { return nil, err } diff --git a/pkg/executor/analyze_test.go b/pkg/executor/analyze_test.go index fbbf1bcc0d930..e47084a8f17f1 100644 --- a/pkg/executor/analyze_test.go +++ b/pkg/executor/analyze_test.go @@ -22,7 +22,7 @@ import ( "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/infoschema" - "github.com/pingcap/tidb/pkg/parser/model" + "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/stmtctx" @@ -77,7 +77,7 @@ func TestAnalyzeIndexExtractTopN(t *testing.T) { tk.MustExec("analyze table t") is := tk.Session().(sessionctx.Context).GetInfoSchema().(infoschema.InfoSchema) - table, err := is.TableByName(context.Background(), model.NewCIStr("test_index_extract_topn"), model.NewCIStr("t")) + table, err := is.TableByName(context.Background(), ast.NewCIStr("test_index_extract_topn"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo := table.Meta() tbl := dom.StatsHandle().GetTableStats(tableInfo) diff --git a/pkg/executor/batch_point_get.go b/pkg/executor/batch_point_get.go index 463221f98a6b5..286b6d29f69b6 100644 --- a/pkg/executor/batch_point_get.go +++ b/pkg/executor/batch_point_get.go @@ -25,7 +25,7 @@ import ( "github.com/pingcap/tidb/pkg/executor/internal/exec" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" plannercore "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/sessionctx" @@ -57,7 +57,7 @@ type BatchPointGetExec struct { planPhysIDs []int64 // If != 0 then it is a single partition under Static Prune mode. singlePartID int64 - partitionNames []pmodel.CIStr + partitionNames []ast.CIStr idxVals [][]types.Datum txn kv.Transaction lock bool @@ -109,7 +109,7 @@ func (e *BatchPointGetExec) Open(context.Context) error { lock := e.tblInfo.Lock if e.lock { batchGetter = driver.NewBufferBatchGetter(txn.GetMemBuffer(), &PessimisticLockCacheGetter{txnCtx: txnCtx}, e.snapshot) - } else if lock != nil && (lock.Tp == pmodel.TableLockRead || lock.Tp == pmodel.TableLockReadOnly) && e.Ctx().GetSessionVars().EnablePointGetCache { + } else if lock != nil && (lock.Tp == ast.TableLockRead || lock.Tp == ast.TableLockReadOnly) && e.Ctx().GetSessionVars().EnablePointGetCache { batchGetter = newCacheBatchGetter(e.Ctx(), e.tblInfo.ID, e.snapshot) } else { batchGetter = driver.NewBufferBatchGetter(txn.GetMemBuffer(), nil, e.snapshot) diff --git a/pkg/executor/benchmark_test.go b/pkg/executor/benchmark_test.go index 9000fd7b79742..8bf66afc77e6b 100644 --- a/pkg/executor/benchmark_test.go +++ b/pkg/executor/benchmark_test.go @@ -37,7 +37,6 @@ import ( "github.com/pingcap/tidb/pkg/expression/aggregation" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/ast" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/planner/core/base" @@ -2073,7 +2072,7 @@ func BenchmarkPipelinedRowNumberWindowFunctionExecution(b *testing.B) { func BenchmarkCompleteInsertErr(b *testing.B) { b.ReportAllocs() col := &model.ColumnInfo{ - Name: pmodel.NewCIStr("a"), + Name: ast.NewCIStr("a"), FieldType: *types.NewFieldType(mysql.TypeBlob), } err := types.ErrWarnDataOutOfRange @@ -2085,7 +2084,7 @@ func BenchmarkCompleteInsertErr(b *testing.B) { func BenchmarkCompleteLoadErr(b *testing.B) { b.ReportAllocs() col := &model.ColumnInfo{ - Name: pmodel.NewCIStr("a"), + Name: ast.NewCIStr("a"), } err := types.ErrDataTooLong for n := 0; n < b.N; n++ { diff --git a/pkg/executor/brie.go b/pkg/executor/brie.go index 8ad6ec5b9b2f9..07341ed5e0812 100644 --- a/pkg/executor/brie.go +++ b/pkg/executor/brie.go @@ -41,7 +41,6 @@ import ( "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/format" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/sessionctx" @@ -792,7 +791,7 @@ func (gs *tidbGlueSession) CreateDatabase(_ context.Context, schema *model.DBInf } // CreateTable implements glue.Session -func (gs *tidbGlueSession) CreateTable(_ context.Context, dbName pmodel.CIStr, table *model.TableInfo, cs ...ddl.CreateTableOption) error { +func (gs *tidbGlueSession) CreateTable(_ context.Context, dbName ast.CIStr, table *model.TableInfo, cs ...ddl.CreateTableOption) error { return BRIECreateTable(gs.se, dbName, table, "", cs...) } diff --git a/pkg/executor/brie_utils.go b/pkg/executor/brie_utils.go index 78517c7ab90e9..d8ad8fd648663 100644 --- a/pkg/executor/brie_utils.go +++ b/pkg/executor/brie_utils.go @@ -25,7 +25,7 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta/autoid" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/sessionctx" "go.uber.org/zap" @@ -88,7 +88,7 @@ func showRestoredCreateTable(sctx sessionctx.Context, tbl *model.TableInfo, brCo // BRIECreateTable creates the table with OnExistIgnore option func BRIECreateTable( sctx sessionctx.Context, - dbName pmodel.CIStr, + dbName ast.CIStr, table *model.TableInfo, brComment string, cs ...ddl.CreateTableOption, @@ -129,7 +129,7 @@ func BRIECreateTables( sctx.GetSessionVars().ForeignKeyChecks = originForeignKeyChecks }() for db, tablesInDB := range tables { - dbName := pmodel.NewCIStr(db) + dbName := ast.NewCIStr(db) queryBuilder := strings.Builder{} cloneTables := make([]*model.TableInfo, 0, len(tablesInDB)) for _, table := range tablesInDB { @@ -159,7 +159,7 @@ func BRIECreateTables( // splitBatchCreateTable provide a way to split batch into small batch when batch size is large than 6 MB. // The raft entry has limit size of 6 MB, a batch of CreateTables may hit this limitation // TODO: shall query string be set for each split batch create, it looks does not matter if we set once for all. -func splitBatchCreateTable(sctx sessionctx.Context, schema pmodel.CIStr, +func splitBatchCreateTable(sctx sessionctx.Context, schema ast.CIStr, infos []*model.TableInfo, cs ...ddl.CreateTableOption) error { var err error d := domain.GetDomain(sctx).DDLExecutor() diff --git a/pkg/executor/brie_utils_test.go b/pkg/executor/brie_utils_test.go index 023e4f73255a8..f1d4838b607ce 100644 --- a/pkg/executor/brie_utils_test.go +++ b/pkg/executor/brie_utils_test.go @@ -28,7 +28,6 @@ import ( "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser" "github.com/pingcap/tidb/pkg/parser/ast" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/testkit" "github.com/stretchr/testify/require" @@ -49,18 +48,18 @@ func TestSplitBatchCreateTableWithTableId(t *testing.T) { infos1 := []*model.TableInfo{} infos1 = append(infos1, &model.TableInfo{ ID: 124, - Name: pmodel.NewCIStr("table_id_resued1"), + Name: ast.NewCIStr("table_id_resued1"), }) infos1 = append(infos1, &model.TableInfo{ ID: 125, - Name: pmodel.NewCIStr("table_id_resued2"), + Name: ast.NewCIStr("table_id_resued2"), }) sctx := tk.Session() // keep/reused table id verification sctx.SetValue(sessionctx.QueryString, "skip") - err := executor.SplitBatchCreateTableForTest(sctx, pmodel.NewCIStr("test"), infos1, ddl.WithIDAllocated(true)) + err := executor.SplitBatchCreateTableForTest(sctx, ast.NewCIStr("test"), infos1, ddl.WithIDAllocated(true)) require.NoError(t, err) require.Equal(t, "skip", sctx.Value(sessionctx.QueryString)) @@ -85,11 +84,11 @@ func TestSplitBatchCreateTableWithTableId(t *testing.T) { infos2 := []*model.TableInfo{} infos2 = append(infos2, &model.TableInfo{ ID: 124, - Name: pmodel.NewCIStr("table_id_new"), + Name: ast.NewCIStr("table_id_new"), }) tk.Session().SetValue(sessionctx.QueryString, "skip") - err = executor.SplitBatchCreateTableForTest(sctx, pmodel.NewCIStr("test"), infos2) + err = executor.SplitBatchCreateTableForTest(sctx, ast.NewCIStr("test"), infos2) require.NoError(t, err) require.Equal(t, "skip", sctx.Value(sessionctx.QueryString)) @@ -105,7 +104,7 @@ func TestSplitBatchCreateTableWithTableId(t *testing.T) { infos3 := []*model.TableInfo{} originQueryString := sctx.Value(sessionctx.QueryString) - err = executor.SplitBatchCreateTableForTest(sctx, pmodel.NewCIStr("test"), infos3, ddl.WithIDAllocated(true)) + err = executor.SplitBatchCreateTableForTest(sctx, ast.NewCIStr("test"), infos3, ddl.WithIDAllocated(true)) require.NoError(t, err) require.Equal(t, originQueryString, sctx.Value(sessionctx.QueryString)) } @@ -125,15 +124,15 @@ func TestSplitBatchCreateTable(t *testing.T) { infos := []*model.TableInfo{} infos = append(infos, &model.TableInfo{ ID: 1234, - Name: pmodel.NewCIStr("tables_1"), + Name: ast.NewCIStr("tables_1"), }) infos = append(infos, &model.TableInfo{ ID: 1235, - Name: pmodel.NewCIStr("tables_2"), + Name: ast.NewCIStr("tables_2"), }) infos = append(infos, &model.TableInfo{ ID: 1236, - Name: pmodel.NewCIStr("tables_3"), + Name: ast.NewCIStr("tables_3"), }) sctx := tk.Session() @@ -141,7 +140,7 @@ func TestSplitBatchCreateTable(t *testing.T) { // keep/reused table id verification tk.Session().SetValue(sessionctx.QueryString, "skip") require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/ddl/RestoreBatchCreateTableEntryTooLarge", "return(1)")) - err := executor.SplitBatchCreateTableForTest(sctx, pmodel.NewCIStr("test"), infos, ddl.WithIDAllocated(true)) + err := executor.SplitBatchCreateTableForTest(sctx, ast.NewCIStr("test"), infos, ddl.WithIDAllocated(true)) require.NoError(t, err) require.Equal(t, "skip", sctx.Value(sessionctx.QueryString)) @@ -194,20 +193,20 @@ func TestSplitBatchCreateTableFailWithEntryTooLarge(t *testing.T) { infos := []*model.TableInfo{} infos = append(infos, &model.TableInfo{ - Name: pmodel.NewCIStr("tables_1"), + Name: ast.NewCIStr("tables_1"), }) infos = append(infos, &model.TableInfo{ - Name: pmodel.NewCIStr("tables_2"), + Name: ast.NewCIStr("tables_2"), }) infos = append(infos, &model.TableInfo{ - Name: pmodel.NewCIStr("tables_3"), + Name: ast.NewCIStr("tables_3"), }) sctx := tk.Session() tk.Session().SetValue(sessionctx.QueryString, "skip") require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/ddl/RestoreBatchCreateTableEntryTooLarge", "return(0)")) - err := executor.SplitBatchCreateTableForTest(sctx, pmodel.NewCIStr("test"), infos) + err := executor.SplitBatchCreateTableForTest(sctx, ast.NewCIStr("test"), infos) require.Equal(t, "skip", sctx.Value(sessionctx.QueryString)) require.True(t, kv.ErrEntryTooLarge.Equal(err)) @@ -227,7 +226,7 @@ func TestBRIECreateDatabase(t *testing.T) { originQueryString := sctx.Value(sessionctx.QueryString) schema1 := &model.DBInfo{ ID: 1230, - Name: pmodel.NewCIStr("db_1"), + Name: ast.NewCIStr("db_1"), Charset: "utf8mb4", Collate: "utf8mb4_bin", State: model.StatePublic, @@ -237,7 +236,7 @@ func TestBRIECreateDatabase(t *testing.T) { schema2 := &model.DBInfo{ ID: 1240, - Name: pmodel.NewCIStr("db_2"), + Name: ast.NewCIStr("db_2"), Charset: "utf8mb4", Collate: "utf8mb4_bin", State: model.StatePublic, @@ -270,14 +269,14 @@ func TestBRIECreateTable(t *testing.T) { sctx := tk.Session() originQueryString := sctx.Value(sessionctx.QueryString) - dbName := pmodel.NewCIStr("test") + dbName := ast.NewCIStr("test") tableInfo := mockTableInfo(t, sctx, "create table test.table_1 (a int primary key, b json, c varchar(20))") tableInfo.ID = 1230 err := executor.BRIECreateTable(sctx, dbName, tableInfo, "/* from test */") require.NoError(t, err) tableInfo.ID = 1240 - tableInfo.Name = pmodel.NewCIStr("table_2") + tableInfo.Name = ast.NewCIStr("table_2") err = executor.BRIECreateTable(sctx, dbName, tableInfo, "") require.NoError(t, err) require.Equal(t, originQueryString, sctx.Value(sessionctx.QueryString)) diff --git a/pkg/executor/builder.go b/pkg/executor/builder.go index 4ca10ebca5efe..2c31e3b1a4c97 100644 --- a/pkg/executor/builder.go +++ b/pkg/executor/builder.go @@ -59,7 +59,6 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/ast" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" plannercore "github.com/pingcap/tidb/pkg/planner/core" @@ -884,12 +883,12 @@ func (b *executorBuilder) buildShow(v *plannercore.PhysicalShow) exec.Executor { BaseExecutor: exec.NewBaseExecutor(b.ctx, v.Schema(), v.ID()), Tp: v.Tp, CountWarningsOrErrors: v.CountWarningsOrErrors, - DBName: pmodel.NewCIStr(v.DBName), + DBName: ast.NewCIStr(v.DBName), Table: v.Table, Partition: v.Partition, Column: v.Column, IndexName: v.IndexName, - ResourceGroupName: pmodel.NewCIStr(v.ResourceGroupName), + ResourceGroupName: ast.NewCIStr(v.ResourceGroupName), Flag: v.Flag, Roles: v.Roles, User: v.User, @@ -3164,7 +3163,7 @@ func (b *executorBuilder) buildAnalyze(v *plannercore.Analyze) exec.Executor { // so we need to make sure all the columns pass into it. columns, _, err := expression.ColumnInfos2ColumnsAndNames( exprCtx, - pmodel.NewCIStr(task.AnalyzeInfo.DBName), + ast.NewCIStr(task.AnalyzeInfo.DBName), task.TblInfo.Name, append(task.ColsInfo, task.SkipColsInfo...), task.TblInfo, @@ -5474,7 +5473,7 @@ func (builder *dataReaderBuilder) partitionPruning(tbl table.PartitionedTable, p func partitionPruning(ctx sessionctx.Context, tbl table.PartitionedTable, planPartInfo *plannercore.PhysPlanPartInfo) ([]table.PhysicalTable, error) { var pruningConds []expression.Expression - var partitionNames []pmodel.CIStr + var partitionNames []ast.CIStr var columns []*expression.Column var columnNames types.NameSlice if planPartInfo != nil { diff --git a/pkg/executor/chunk_size_control_test.go b/pkg/executor/chunk_size_control_test.go index 7f9f0975198db..86c5e6eeaf96b 100644 --- a/pkg/executor/chunk_size_control_test.go +++ b/pkg/executor/chunk_size_control_test.go @@ -24,7 +24,7 @@ import ( "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/kv" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/session" "github.com/pingcap/tidb/pkg/store/mockstore" "github.com/pingcap/tidb/pkg/tablecodec" @@ -99,7 +99,7 @@ func generateTableSplitKeyForInt(tid int64, splitNum []int) [][]byte { func TestLimitAndTableScan(t *testing.T) { t.Skip("not stable because coprocessor may result in goroutine leak") kit := createChunkSizeControlKit(t, "create table t (a int, primary key (a))") - tbl, err := kit.dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := kit.dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tid := tbl.Meta().ID @@ -130,7 +130,7 @@ func TestLimitAndTableScan(t *testing.T) { func TestLimitAndIndexScan(t *testing.T) { t.Skip("not stable because coprocessor may result in goroutine leak") kit := createChunkSizeControlKit(t, "create table t (a int, index idx_a(a))") - tbl, err := kit.dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := kit.dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tid := tbl.Meta().ID idx := tbl.Meta().Indices[0].ID diff --git a/pkg/executor/copr_cache_test.go b/pkg/executor/copr_cache_test.go index 2f084d856a993..cf6b8611ee5cb 100644 --- a/pkg/executor/copr_cache_test.go +++ b/pkg/executor/copr_cache_test.go @@ -22,7 +22,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb/pkg/config" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/store/mockstore" "github.com/pingcap/tidb/pkg/tablecodec" "github.com/pingcap/tidb/pkg/testkit" @@ -53,7 +53,7 @@ func TestIntegrationCopCache(t *testing.T) { tk.MustExec("use test") tk.MustExec("create table t (a int primary key)") - tblInfo, err := dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tblInfo, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tid := tblInfo.Meta().ID tk.MustExec(`insert into t values(1),(2),(3),(4),(5),(6),(7),(8),(9),(10),(11),(12)`) diff --git a/pkg/executor/ddl.go b/pkg/executor/ddl.go index a0f09abe689ad..f7a979a6b93a7 100644 --- a/pkg/executor/ddl.go +++ b/pkg/executor/ddl.go @@ -28,7 +28,6 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/ast" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/planner/core" @@ -76,7 +75,7 @@ func (e *DDLExec) toErr(err error) error { return err } -func (e *DDLExec) getLocalTemporaryTable(schema pmodel.CIStr, table pmodel.CIStr) (table.Table, bool, error) { +func (e *DDLExec) getLocalTemporaryTable(schema ast.CIStr, table ast.CIStr) (table.Table, bool, error) { tbl, err := e.Ctx().GetInfoSchema().(infoschema.InfoSchema).TableByName(context.Background(), schema, table) if infoschema.ErrTableNotExists.Equal(err) { return nil, false, nil @@ -602,7 +601,7 @@ func (e *DDLExec) executeFlashbackTable(s *ast.FlashBackTableStmt) error { return err } if len(s.NewName) != 0 { - tblInfo.Name = pmodel.NewCIStr(s.NewName) + tblInfo.Name = ast.NewCIStr(s.NewName) } // Check the table ID was not exists. is := domain.GetDomain(e.Ctx()).InfoSchema() @@ -637,7 +636,7 @@ func (e *DDLExec) executeFlashbackTable(s *ast.FlashBackTableStmt) error { func (e *DDLExec) executeFlashbackDatabase(s *ast.FlashBackDatabaseStmt) error { dbName := s.DBName if len(s.NewName) > 0 { - dbName = pmodel.NewCIStr(s.NewName) + dbName = ast.NewCIStr(s.NewName) } // Check the Schema Name was not exists. is := domain.GetDomain(e.Ctx()).InfoSchema() @@ -658,7 +657,7 @@ func (e *DDLExec) executeFlashbackDatabase(s *ast.FlashBackDatabaseStmt) error { return err } -func (e *DDLExec) getRecoverDBByName(schemaName pmodel.CIStr) (recoverSchemaInfo *model.RecoverSchemaInfo, err error) { +func (e *DDLExec) getRecoverDBByName(schemaName ast.CIStr) (recoverSchemaInfo *model.RecoverSchemaInfo, err error) { txn, err := e.Ctx().Txn(true) if err != nil { return nil, err diff --git a/pkg/executor/distsql_test.go b/pkg/executor/distsql_test.go index 7a37fcdaee7e8..3e90935cabd91 100644 --- a/pkg/executor/distsql_test.go +++ b/pkg/executor/distsql_test.go @@ -30,7 +30,7 @@ import ( "github.com/pingcap/tidb/pkg/executor" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/metrics" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/store/copr" "github.com/pingcap/tidb/pkg/store/mockstore" "github.com/pingcap/tidb/pkg/table/tables" @@ -83,7 +83,7 @@ func TestCopClientSend(t *testing.T) { // Get table ID for split. is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("copclient")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("copclient")) require.NoError(t, err) tblID := tbl.Meta().ID @@ -161,7 +161,7 @@ func TestInconsistentIndex(t *testing.T) { tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int, b int, index idx_a(a))") is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) idx := tbl.Meta().FindIndexByName("idx_a") idxOp := tables.NewIndex(tbl.Meta().ID, tbl.Meta(), idx) diff --git a/pkg/executor/foreign_key.go b/pkg/executor/foreign_key.go index 5f0c7e701bbd8..2c51f88e53183 100644 --- a/pkg/executor/foreign_key.go +++ b/pkg/executor/foreign_key.go @@ -26,7 +26,6 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/ast" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/planner" plannercore "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/planner/core/base" @@ -136,7 +135,7 @@ func buildFKCheckExecs(sctx sessionctx.Context, tbl table.Table, fkChecks []*pla } func buildFKCheckExec(sctx sessionctx.Context, tbl table.Table, fkCheck *plannercore.FKCheck) (*FKCheckExec, error) { - var cols []pmodel.CIStr + var cols []ast.CIStr if fkCheck.FK != nil { cols = fkCheck.FK.Cols } else if fkCheck.ReferredFK != nil { @@ -520,7 +519,7 @@ func (*fkValueHelper) hasNullValue(vals []types.Datum) bool { return false } -func getFKColumnsOffsets(tbInfo *model.TableInfo, cols []pmodel.CIStr) ([]int, error) { +func getFKColumnsOffsets(tbInfo *model.TableInfo, cols []ast.CIStr) ([]int, error) { colsOffsets := make([]int, len(cols)) for i, col := range cols { offset := -1 @@ -709,7 +708,7 @@ func (fkc *FKCascadeExec) onUpdateRow(sc *stmtctx.StatementContext, oldRow, newR if err != nil || len(oldVals) == 0 { return err } - if pmodel.ReferOptionType(fkc.fk.OnUpdate) == pmodel.ReferOptionSetNull { + if ast.ReferOptionType(fkc.fk.OnUpdate) == ast.ReferOptionSetNull { fkc.fkValues = append(fkc.fkValues, oldVals) return nil } @@ -751,7 +750,7 @@ func (fkc *FKCascadeExec) buildFKCascadePlan(ctx context.Context) (base.Plan, er if len(fkc.fkValues) == 0 && len(fkc.fkUpdatedValuesMap) == 0 { return nil, nil } - var indexName pmodel.CIStr + var indexName ast.CIStr if fkc.fkIdx != nil { indexName = fkc.fkIdx.Name } @@ -759,15 +758,15 @@ func (fkc *FKCascadeExec) buildFKCascadePlan(ctx context.Context) (base.Plan, er switch fkc.tp { case plannercore.FKCascadeOnDelete: fkValues := fkc.fetchOnDeleteOrUpdateFKValues() - switch pmodel.ReferOptionType(fkc.fk.OnDelete) { - case pmodel.ReferOptionCascade: + switch ast.ReferOptionType(fkc.fk.OnDelete) { + case ast.ReferOptionCascade: stmtNode = GenCascadeDeleteAST(fkc.referredFK.ChildSchema, fkc.childTable.Name, indexName, fkc.fkCols, fkValues) - case pmodel.ReferOptionSetNull: + case ast.ReferOptionSetNull: stmtNode = GenCascadeSetNullAST(fkc.referredFK.ChildSchema, fkc.childTable.Name, indexName, fkc.fkCols, fkValues) } case plannercore.FKCascadeOnUpdate: - switch pmodel.ReferOptionType(fkc.fk.OnUpdate) { - case pmodel.ReferOptionCascade: + switch ast.ReferOptionType(fkc.fk.OnUpdate) { + case ast.ReferOptionCascade: couple := fkc.fetchUpdatedValuesCouple() if couple != nil && len(couple.NewValues) != 0 { if fkc.stats != nil { @@ -775,7 +774,7 @@ func (fkc *FKCascadeExec) buildFKCascadePlan(ctx context.Context) (base.Plan, er } stmtNode = GenCascadeUpdateAST(fkc.referredFK.ChildSchema, fkc.childTable.Name, indexName, fkc.fkCols, couple) } - case pmodel.ReferOptionSetNull: + case ast.ReferOptionSetNull: fkValues := fkc.fetchOnDeleteOrUpdateFKValues() stmtNode = GenCascadeSetNullAST(fkc.referredFK.ChildSchema, fkc.childTable.Name, indexName, fkc.fkCols, fkValues) } @@ -828,7 +827,7 @@ func (fkc *FKCascadeExec) fetchUpdatedValuesCouple() *UpdatedValuesCouple { } // GenCascadeDeleteAST uses to generate cascade delete ast, export for test. -func GenCascadeDeleteAST(schema, table, idx pmodel.CIStr, cols []*model.ColumnInfo, fkValues [][]types.Datum) *ast.DeleteStmt { +func GenCascadeDeleteAST(schema, table, idx ast.CIStr, cols []*model.ColumnInfo, fkValues [][]types.Datum) *ast.DeleteStmt { deleteStmt := &ast.DeleteStmt{ TableRefs: genTableRefsAST(schema, table, idx), Where: genWhereConditionAst(cols, fkValues), @@ -837,7 +836,7 @@ func GenCascadeDeleteAST(schema, table, idx pmodel.CIStr, cols []*model.ColumnIn } // GenCascadeSetNullAST uses to generate foreign key `SET NULL` ast, export for test. -func GenCascadeSetNullAST(schema, table, idx pmodel.CIStr, cols []*model.ColumnInfo, fkValues [][]types.Datum) *ast.UpdateStmt { +func GenCascadeSetNullAST(schema, table, idx ast.CIStr, cols []*model.ColumnInfo, fkValues [][]types.Datum) *ast.UpdateStmt { newValues := make([]types.Datum, len(cols)) for i := range cols { newValues[i] = types.NewDatum(nil) @@ -850,7 +849,7 @@ func GenCascadeSetNullAST(schema, table, idx pmodel.CIStr, cols []*model.ColumnI } // GenCascadeUpdateAST uses to generate cascade update ast, export for test. -func GenCascadeUpdateAST(schema, table, idx pmodel.CIStr, cols []*model.ColumnInfo, couple *UpdatedValuesCouple) *ast.UpdateStmt { +func GenCascadeUpdateAST(schema, table, idx ast.CIStr, cols []*model.ColumnInfo, couple *UpdatedValuesCouple) *ast.UpdateStmt { list := make([]*ast.Assignment, 0, len(cols)) for i, col := range cols { v := &driver.ValueExpr{Datum: couple.NewValues[i]} @@ -869,11 +868,11 @@ func GenCascadeUpdateAST(schema, table, idx pmodel.CIStr, cols []*model.ColumnIn return updateStmt } -func genTableRefsAST(schema, table, idx pmodel.CIStr) *ast.TableRefsClause { +func genTableRefsAST(schema, table, idx ast.CIStr) *ast.TableRefsClause { tn := &ast.TableName{Schema: schema, Name: table} if idx.L != "" { tn.IndexHints = []*ast.IndexHint{{ - IndexNames: []pmodel.CIStr{idx}, + IndexNames: []ast.CIStr{idx}, HintType: ast.HintUse, HintScope: ast.HintForScan, }} diff --git a/pkg/executor/grant.go b/pkg/executor/grant.go index f941d200f2f96..b25d6a769b51c 100644 --- a/pkg/executor/grant.go +++ b/pkg/executor/grant.go @@ -26,7 +26,6 @@ import ( "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser/ast" - "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/planner/core/resolve" @@ -95,9 +94,9 @@ func (e *GrantExec) Next(ctx context.Context, _ *chunk.Chunk) error { } } } - dbNameStr := model.NewCIStr(dbName) + dbNameStr := ast.NewCIStr(dbName) schema := e.Ctx().GetInfoSchema().(infoschema.InfoSchema) - tbl, err := schema.TableByName(ctx, dbNameStr, model.NewCIStr(e.Level.TableName)) + tbl, err := schema.TableByName(ctx, dbNameStr, ast.NewCIStr(e.Level.TableName)) // Allow GRANT on non-existent table with at least create privilege, see issue #28533 #29268 if err != nil { allowed := false @@ -780,8 +779,8 @@ func getTargetSchemaAndTable(ctx context.Context, sctx sessionctx.Context, dbNam return "", nil, errors.New("miss DB name for grant privilege") } } - name := model.NewCIStr(tableName) - tbl, err := is.TableByName(ctx, model.NewCIStr(dbName), name) + name := ast.NewCIStr(tableName) + tbl, err := is.TableByName(ctx, ast.NewCIStr(dbName), name) if terror.ErrorEqual(err, infoschema.ErrTableNotExists) { return dbName, nil, err } diff --git a/pkg/executor/historical_stats_test.go b/pkg/executor/historical_stats_test.go index 73a6d78d30e98..47f49be0f81f0 100644 --- a/pkg/executor/historical_stats_test.go +++ b/pkg/executor/historical_stats_test.go @@ -23,7 +23,7 @@ import ( "time" "github.com/pingcap/failpoint" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/statistics/handle/storage" "github.com/pingcap/tidb/pkg/statistics/util" @@ -46,7 +46,7 @@ func TestRecordHistoryStatsAfterAnalyze(t *testing.T) { h := dom.StatsHandle() is := dom.InfoSchema() - tableInfo, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tableInfo, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) // 1. switch off the tidb_enable_historical_stats, and there is no records in table `mysql.stats_history` @@ -110,7 +110,7 @@ func TestRecordHistoryStatsMetaAfterAnalyze(t *testing.T) { h := dom.StatsHandle() is := dom.InfoSchema() - tableInfo, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tableInfo, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) // 1. switch off the tidb_enable_historical_stats, and there is no record in table `mysql.stats_meta_history` @@ -167,7 +167,7 @@ func TestGCHistoryStatsAfterDropTable(t *testing.T) { tk.MustExec("create table t(a int, b varchar(10), index idx(a, b))") tk.MustExec("analyze table test.t") is := dom.InfoSchema() - tableInfo, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tableInfo, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) // dump historical stats h := dom.StatsHandle() @@ -204,7 +204,7 @@ func TestAssertHistoricalStatsAfterAlterTable(t *testing.T) { tk.MustExec("create table t(a int, b varchar(10),c int, KEY `idx` (`c`))") tk.MustExec("analyze table test.t") is := dom.InfoSchema() - tableInfo, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tableInfo, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) // dump historical stats h := dom.StatsHandle() @@ -253,7 +253,7 @@ func TestGCOutdatedHistoryStats(t *testing.T) { tk.MustExec("create table t(a int, b varchar(10), index idx(a, b))") tk.MustExec("analyze table test.t") is := dom.InfoSchema() - tableInfo, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tableInfo, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) // dump historical stats h := dom.StatsHandle() @@ -328,7 +328,7 @@ PARTITION p0 VALUES LESS THAN (6) tk.MustExec("analyze table t") is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) require.NotNil(t, tbl) @@ -387,7 +387,7 @@ PARTITION p0 VALUES LESS THAN (6) // dump historical stats tk.MustExec("analyze table t") is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) require.NotNil(t, tbl) diff --git a/pkg/executor/importer/BUILD.bazel b/pkg/executor/importer/BUILD.bazel index 61e9e8fada8b4..c593c60eb23fa 100644 --- a/pkg/executor/importer/BUILD.bazel +++ b/pkg/executor/importer/BUILD.bazel @@ -126,7 +126,6 @@ go_test( "//pkg/metrics", "//pkg/parser", "//pkg/parser/ast", - "//pkg/parser/model", "//pkg/parser/mysql", "//pkg/planner/core", "//pkg/planner/core/base", diff --git a/pkg/executor/importer/chunk_process_testkit_test.go b/pkg/executor/importer/chunk_process_testkit_test.go index 5bc27816ce46e..64ef93de69140 100644 --- a/pkg/executor/importer/chunk_process_testkit_test.go +++ b/pkg/executor/importer/chunk_process_testkit_test.go @@ -35,7 +35,7 @@ import ( "github.com/pingcap/tidb/pkg/lightning/mydump" verify "github.com/pingcap/tidb/pkg/lightning/verification" tidbmetrics "github.com/pingcap/tidb/pkg/metrics" - "github.com/pingcap/tidb/pkg/parser/model" + "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/testkit" @@ -67,7 +67,7 @@ func TestFileChunkProcess(t *testing.T) { tk.MustExec(stmt) do, err := session.GetDomain(store) require.NoError(t, err) - table, err := do.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + table, err := do.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) fieldMappings := make([]*importer.FieldMapping, 0, len(table.VisibleCols())) diff --git a/pkg/executor/importer/importer_testkit_test.go b/pkg/executor/importer/importer_testkit_test.go index 01b00acced6da..6d8ff16109128 100644 --- a/pkg/executor/importer/importer_testkit_test.go +++ b/pkg/executor/importer/importer_testkit_test.go @@ -38,7 +38,6 @@ import ( "github.com/pingcap/tidb/pkg/meta/autoid" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/ast" - pmodel "github.com/pingcap/tidb/pkg/parser/model" plannercore "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/core/resolve" @@ -67,7 +66,7 @@ func TestVerifyChecksum(t *testing.T) { plan := &importer.Plan{ DBName: "db", TableInfo: &model.TableInfo{ - Name: pmodel.NewCIStr("tb"), + Name: ast.NewCIStr("tb"), }, Checksum: config.OpLevelRequired, DistSQLScanConcurrency: 50, @@ -92,7 +91,7 @@ func TestVerifyChecksum(t *testing.T) { plan2 := &importer.Plan{ DBName: "db", TableInfo: &model.TableInfo{ - Name: pmodel.NewCIStr("tb2"), + Name: ast.NewCIStr("tb2"), }, Checksum: config.OpLevelRequired, } @@ -205,9 +204,9 @@ func TestPostProcess(t *testing.T) { tk.MustExec("insert into db.tb values(1)") do, err := session.GetDomain(store) require.NoError(t, err) - dbInfo, ok := do.InfoSchema().SchemaByName(pmodel.NewCIStr("db")) + dbInfo, ok := do.InfoSchema().SchemaByName(ast.NewCIStr("db")) require.True(t, ok) - table, err := do.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("db"), pmodel.NewCIStr("tb")) + table, err := do.InfoSchema().TableByName(context.Background(), ast.NewCIStr("db"), ast.NewCIStr("tb")) require.NoError(t, err) plan := &importer.Plan{ DBID: dbInfo.ID, @@ -228,7 +227,7 @@ func TestPostProcess(t *testing.T) { require.NoError(t, importer.PostProcess(ctx, tk.Session(), nil, plan, localChecksum, logger)) // rebase success tk.MustExec("create table db.tb2(id int auto_increment primary key)") - table, err = do.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("db"), pmodel.NewCIStr("tb2")) + table, err = do.InfoSchema().TableByName(context.Background(), ast.NewCIStr("db"), ast.NewCIStr("tb2")) require.NoError(t, err) plan.TableInfo, plan.DesiredTableInfo = table.Meta(), table.Meta() integration.BeforeTestExternal(t) @@ -257,9 +256,9 @@ func getTableImporter(ctx context.Context, t *testing.T, store kv.Storage, table tk := testkit.NewTestKit(t, store) do, err := session.GetDomain(store) require.NoError(t, err) - dbInfo, ok := do.InfoSchema().SchemaByName(pmodel.NewCIStr("test")) + dbInfo, ok := do.InfoSchema().SchemaByName(ast.NewCIStr("test")) require.True(t, ok) - table, err := do.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr(tableName)) + table, err := do.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr(tableName)) require.NoError(t, err) var selectPlan base.PhysicalPlan if path == "" { diff --git a/pkg/executor/importer/precheck_test.go b/pkg/executor/importer/precheck_test.go index 6a086a7001766..7f6d11e4de5fc 100644 --- a/pkg/executor/importer/precheck_test.go +++ b/pkg/executor/importer/precheck_test.go @@ -29,7 +29,7 @@ import ( "github.com/pingcap/tidb/pkg/executor/importer" "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/kv" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/util/cdcutil" "github.com/pingcap/tidb/pkg/util/dbterror/exeerrors" @@ -75,7 +75,7 @@ func TestCheckRequirements(t *testing.T) { _, err := conn.Execute(ctx, "create table test.t(id int primary key)") require.NoError(t, err) is := tk.Session().GetDomainInfoSchema().(infoschema.InfoSchema) - tableObj, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tableObj, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) c := &importer.LoadDataController{ diff --git a/pkg/executor/importer/table_import_testkit_test.go b/pkg/executor/importer/table_import_testkit_test.go index d40e74437b200..020f264a34d17 100644 --- a/pkg/executor/importer/table_import_testkit_test.go +++ b/pkg/executor/importer/table_import_testkit_test.go @@ -26,7 +26,6 @@ import ( "github.com/pingcap/tidb/pkg/lightning/backend/local" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/ast" - pmodel "github.com/pingcap/tidb/pkg/parser/model" plannercore "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/planner/core/resolve" "github.com/pingcap/tidb/pkg/session" @@ -80,17 +79,17 @@ func TestImportFromSelectCleanup(t *testing.T) { tk.MustExec("create table t(a int)") do, err := session.GetDomain(store) require.NoError(t, err) - dbInfo, ok := do.InfoSchema().SchemaByName(pmodel.NewCIStr("test")) + dbInfo, ok := do.InfoSchema().SchemaByName(ast.NewCIStr("test")) require.True(t, ok) - table, err := do.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + table, err := do.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) plan, err := importer.NewImportPlan(ctx, tk.Session(), plannercore.ImportInto{ Table: &resolve.TableNameW{ TableName: &ast.TableName{ - Name: pmodel.NewCIStr("t"), + Name: ast.NewCIStr("t"), }, DBInfo: &model.DBInfo{ - Name: pmodel.NewCIStr("test"), + Name: ast.NewCIStr("test"), ID: dbInfo.ID, }, }, diff --git a/pkg/executor/infoschema_reader.go b/pkg/executor/infoschema_reader.go index 206ebb52fb431..30e47113a70ae 100644 --- a/pkg/executor/infoschema_reader.go +++ b/pkg/executor/infoschema_reader.go @@ -46,8 +46,8 @@ import ( "github.com/pingcap/tidb/pkg/meta/autoid" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/charset" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" plannercore "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/planner/core/base" @@ -447,7 +447,7 @@ func (e *memtableRetriever) setDataForStatistics(ctx context.Context, sctx sessi } func (e *memtableRetriever) setDataForStatisticsInTable( - schema pmodel.CIStr, + schema ast.CIStr, table *model.TableInfo, ex *plannercore.InfoSchemaStatisticsExtractor, ) { @@ -570,11 +570,11 @@ func (e *memtableRetriever) setDataFromReferConst(ctx context.Context, sctx sess continue } updateRule, deleteRule := "NO ACTION", "NO ACTION" - if pmodel.ReferOptionType(fk.OnUpdate) != 0 { - updateRule = pmodel.ReferOptionType(fk.OnUpdate).String() + if ast.ReferOptionType(fk.OnUpdate) != 0 { + updateRule = ast.ReferOptionType(fk.OnUpdate).String() } - if pmodel.ReferOptionType(fk.OnDelete) != 0 { - deleteRule = pmodel.ReferOptionType(fk.OnDelete).String() + if ast.ReferOptionType(fk.OnDelete) != 0 { + deleteRule = ast.ReferOptionType(fk.OnDelete).String() } record := types.MakeDatums( infoschema.CatalogVal, // CONSTRAINT_CATALOG @@ -610,7 +610,7 @@ func (e *memtableRetriever) setDataFromOneTable( sctx sessionctx.Context, loc *time.Location, checker privilege.Manager, - schema pmodel.CIStr, + schema ast.CIStr, table *model.TableInfo, rows [][]types.Datum, useStatsCache bool, @@ -960,7 +960,7 @@ type hugeMemTableRetriever struct { retrieved bool initialized bool rows [][]types.Datum - dbs []pmodel.CIStr + dbs []ast.CIStr curTables []*model.TableInfo dbsIdx int tblIdx int @@ -1026,7 +1026,7 @@ func (e *hugeMemTableRetriever) setDataForColumns(ctx context.Context, sctx sess func (e *hugeMemTableRetriever) setDataForColumnsWithOneTable( ctx context.Context, sctx sessionctx.Context, - schema pmodel.CIStr, + schema ast.CIStr, table *model.TableInfo, checker privilege.Manager) bool { hasPrivs := false @@ -1050,7 +1050,7 @@ func (e *hugeMemTableRetriever) setDataForColumnsWithOneTable( func (e *hugeMemTableRetriever) dataForColumnsInTable( ctx context.Context, sctx sessionctx.Context, - schema pmodel.CIStr, + schema ast.CIStr, tbl *model.TableInfo, priv mysql.PrivilegeType) { if tbl.IsView() { @@ -1300,9 +1300,9 @@ func (e *memtableRetriever) setDataFromPartitions(ctx context.Context, sctx sess } var partitionDesc string - if table.Partition.Type == pmodel.PartitionTypeRange { + if table.Partition.Type == ast.PartitionTypeRange { partitionDesc = strings.Join(pi.LessThan, ",") - } else if table.Partition.Type == pmodel.PartitionTypeList { + } else if table.Partition.Type == ast.PartitionTypeList { if len(pi.InValues) > 0 { buf := bytes.NewBuffer(nil) for i, vs := range pi.InValues { @@ -1325,11 +1325,11 @@ func (e *memtableRetriever) setDataFromPartitions(ctx context.Context, sctx sess partitionExpr := table.Partition.Expr if len(table.Partition.Columns) > 0 { switch table.Partition.Type { - case pmodel.PartitionTypeRange: + case ast.PartitionTypeRange: partitionMethod = "RANGE COLUMNS" - case pmodel.PartitionTypeList: + case ast.PartitionTypeList: partitionMethod = "LIST COLUMNS" - case pmodel.PartitionTypeKey: + case ast.PartitionTypeKey: partitionMethod = "KEY" default: return errors.Errorf("Inconsistent partition type, have type %v, but with COLUMNS > 0 (%d)", table.Partition.Type, len(table.Partition.Columns)) @@ -1414,7 +1414,7 @@ func (e *memtableRetriever) setDataFromIndexes(ctx context.Context, sctx session func (*memtableRetriever) setDataFromIndex( sctx sessionctx.Context, - schema pmodel.CIStr, + schema ast.CIStr, tb *model.TableInfo, rows [][]types.Datum) ([][]types.Datum, error) { checker := privilege.GetPrivilegeManager(sctx) @@ -1873,7 +1873,7 @@ func (e *memtableRetriever) setDataForMetricTables() { e.rows = rows } -func keyColumnUsageInTable(schema pmodel.CIStr, table *model.TableInfo, ex *plannercore.InfoSchemaKeyColumnUsageExtractor) [][]types.Datum { +func keyColumnUsageInTable(schema ast.CIStr, table *model.TableInfo, ex *plannercore.InfoSchemaKeyColumnUsageExtractor) [][]types.Datum { var rows [][]types.Datum if table.PKIsHandle && ex.HasPrimaryKey() { for _, col := range table.Columns { @@ -2356,7 +2356,7 @@ func (e *tableStorageStatsRetriever) initialize(ctx context.Context, sctx sessio for _, DB := range databases { // The user didn't specified the table, extract all tables of this db to initialTable. if len(tables) == 0 { - tbs, err := is.SchemaTableInfos(ctx, pmodel.NewCIStr(DB)) + tbs, err := is.SchemaTableInfos(ctx, ast.NewCIStr(DB)) if err != nil { return errors.Trace(err) } @@ -2369,7 +2369,7 @@ func (e *tableStorageStatsRetriever) initialize(ctx context.Context, sctx sessio } else { // The user specified the table, extract the specified tables of this db to initialTable. for tb := range tables { - if tb, err := is.TableByName(context.Background(), pmodel.NewCIStr(DB), pmodel.NewCIStr(tb)); err == nil { + if tb, err := is.TableByName(context.Background(), ast.NewCIStr(DB), ast.NewCIStr(tb)); err == nil { // For every db.table, check it's privileges. if checker(DB, tb.Meta().Name.L) { e.initialTables = append(e.initialTables, &initialTable{DB, tb.Meta()}) @@ -2542,7 +2542,7 @@ func getRemainDurationForAnalyzeStatusHelper( } var tid int64 is := sessiontxn.GetTxnManager(sctx).GetTxnInfoSchema() - tb, err := is.TableByName(ctx, pmodel.NewCIStr(dbName), pmodel.NewCIStr(tableName)) + tb, err := is.TableByName(ctx, ast.NewCIStr(dbName), ast.NewCIStr(tableName)) if err != nil { return nil, percentage, totalCnt, err } @@ -3764,7 +3764,7 @@ func (e *memtableRetriever) setDataFromResourceGroups() error { for _, group := range resourceGroups { //mode := "" burstable := burstdisableStr - priority := pmodel.PriorityValueToName(uint64(group.Priority)) + priority := ast.PriorityValueToName(uint64(group.Priority)) fillrate := unlimitedFillRate // RU_PER_SEC = unlimited like the default group settings. isDefaultInReservedSetting := group.RUSettings.RU.Settings.FillRate == math.MaxInt32 @@ -3798,19 +3798,19 @@ func (e *memtableRetriever) setDataFromResourceGroups() error { fmt.Fprintf(limitBuilder, "RU=%d", setting.Rule.RequestUnit) } // action settings - actionType := pmodel.RunawayActionType(setting.Action) + actionType := ast.RunawayActionType(setting.Action) switch actionType { - case pmodel.RunawayActionDryRun, pmodel.RunawayActionCooldown, pmodel.RunawayActionKill: + case ast.RunawayActionDryRun, ast.RunawayActionCooldown, ast.RunawayActionKill: fmt.Fprintf(limitBuilder, ", ACTION=%s", actionType.String()) - case pmodel.RunawayActionSwitchGroup: + case ast.RunawayActionSwitchGroup: fmt.Fprintf(limitBuilder, ", ACTION=%s(%s)", actionType.String(), setting.SwitchGroupName) } if setting.Watch != nil { if setting.Watch.LastingDurationMs > 0 { dur := time.Duration(setting.Watch.LastingDurationMs) * time.Millisecond - fmt.Fprintf(limitBuilder, ", WATCH=%s DURATION='%s'", pmodel.RunawayWatchType(setting.Watch.Type).String(), dur.String()) + fmt.Fprintf(limitBuilder, ", WATCH=%s DURATION='%s'", ast.RunawayWatchType(setting.Watch.Type).String(), dur.String()) } else { - fmt.Fprintf(limitBuilder, ", WATCH=%s DURATION=UNLIMITED", pmodel.RunawayWatchType(setting.Watch.Type).String()) + fmt.Fprintf(limitBuilder, ", WATCH=%s DURATION=UNLIMITED", ast.RunawayWatchType(setting.Watch.Type).String()) } } } @@ -4042,7 +4042,7 @@ func decodeTableIDFromRule(rule *label.Rule) (tableID int64, err error) { func tableOrPartitionNotExist(ctx context.Context, dbName string, tableName string, partitionName string, is infoschema.InfoSchema, tableID int64) (tableNotExist bool) { if len(partitionName) == 0 { - curTable, _ := is.TableByName(ctx, pmodel.NewCIStr(dbName), pmodel.NewCIStr(tableName)) + curTable, _ := is.TableByName(ctx, ast.NewCIStr(dbName), ast.NewCIStr(tableName)) if curTable == nil { return true } diff --git a/pkg/executor/infoschema_reader_internal_test.go b/pkg/executor/infoschema_reader_internal_test.go index 30a65a4e856ed..425953354a878 100644 --- a/pkg/executor/infoschema_reader_internal_test.go +++ b/pkg/executor/infoschema_reader_internal_test.go @@ -20,7 +20,7 @@ import ( "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" plannercore "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/types" @@ -31,23 +31,23 @@ func TestSetDataFromCheckConstraints(t *testing.T) { tblInfos := []*model.TableInfo{ { ID: 1, - Name: pmodel.NewCIStr("t1"), + Name: ast.NewCIStr("t1"), State: model.StatePublic, }, { ID: 2, - Name: pmodel.NewCIStr("t2"), + Name: ast.NewCIStr("t2"), Columns: []*model.ColumnInfo{ { - Name: pmodel.NewCIStr("id"), + Name: ast.NewCIStr("id"), FieldType: *types.NewFieldType(mysql.TypeLonglong), State: model.StatePublic, }, }, Constraints: []*model.ConstraintInfo{ { - Name: pmodel.NewCIStr("t2_c1"), - Table: pmodel.NewCIStr("t2"), + Name: ast.NewCIStr("t2_c1"), + Table: ast.NewCIStr("t2"), ExprString: "id<10", State: model.StatePublic, }, @@ -56,18 +56,18 @@ func TestSetDataFromCheckConstraints(t *testing.T) { }, { ID: 3, - Name: pmodel.NewCIStr("t3"), + Name: ast.NewCIStr("t3"), Columns: []*model.ColumnInfo{ { - Name: pmodel.NewCIStr("id"), + Name: ast.NewCIStr("id"), FieldType: *types.NewFieldType(mysql.TypeLonglong), State: model.StatePublic, }, }, Constraints: []*model.ConstraintInfo{ { - Name: pmodel.NewCIStr("t3_c1"), - Table: pmodel.NewCIStr("t3"), + Name: ast.NewCIStr("t3_c1"), + Table: ast.NewCIStr("t3"), ExprString: "id<10", State: model.StateDeleteOnly, }, @@ -95,23 +95,23 @@ func TestSetDataFromTiDBCheckConstraints(t *testing.T) { tblInfos := []*model.TableInfo{ { ID: 1, - Name: pmodel.NewCIStr("t1"), + Name: ast.NewCIStr("t1"), State: model.StatePublic, }, { ID: 2, - Name: pmodel.NewCIStr("t2"), + Name: ast.NewCIStr("t2"), Columns: []*model.ColumnInfo{ { - Name: pmodel.NewCIStr("id"), + Name: ast.NewCIStr("id"), FieldType: *types.NewFieldType(mysql.TypeLonglong), State: model.StatePublic, }, }, Constraints: []*model.ConstraintInfo{ { - Name: pmodel.NewCIStr("t2_c1"), - Table: pmodel.NewCIStr("t2"), + Name: ast.NewCIStr("t2_c1"), + Table: ast.NewCIStr("t2"), ExprString: "id<10", State: model.StatePublic, }, @@ -120,18 +120,18 @@ func TestSetDataFromTiDBCheckConstraints(t *testing.T) { }, { ID: 3, - Name: pmodel.NewCIStr("t3"), + Name: ast.NewCIStr("t3"), Columns: []*model.ColumnInfo{ { - Name: pmodel.NewCIStr("id"), + Name: ast.NewCIStr("id"), FieldType: *types.NewFieldType(mysql.TypeLonglong), State: model.StatePublic, }, }, Constraints: []*model.ConstraintInfo{ { - Name: pmodel.NewCIStr("t3_c1"), - Table: pmodel.NewCIStr("t3"), + Name: ast.NewCIStr("t3_c1"), + Table: ast.NewCIStr("t3"), ExprString: "id<10", State: model.StateDeleteOnly, }, diff --git a/pkg/executor/internal/calibrateresource/BUILD.bazel b/pkg/executor/internal/calibrateresource/BUILD.bazel index 9eab84e6d56b3..0f6b557ffd183 100644 --- a/pkg/executor/internal/calibrateresource/BUILD.bazel +++ b/pkg/executor/internal/calibrateresource/BUILD.bazel @@ -12,7 +12,6 @@ go_library( "//pkg/kv", "//pkg/parser/ast", "//pkg/parser/duration", - "//pkg/parser/model", "//pkg/sessionctx", "//pkg/sessionctx/variable", "//pkg/sessiontxn/staleread", diff --git a/pkg/executor/internal/calibrateresource/calibrate_resource.go b/pkg/executor/internal/calibrateresource/calibrate_resource.go index 74cb61fec5846..e47f9a519f3c8 100644 --- a/pkg/executor/internal/calibrateresource/calibrate_resource.go +++ b/pkg/executor/internal/calibrateresource/calibrate_resource.go @@ -37,7 +37,6 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/duration" - "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/sessiontxn/staleread" @@ -196,10 +195,10 @@ func (e *Executor) parseCalibrateDuration(ctx context.Context) (startTime time.T if startTimeExpr == nil { toTimeExpr := endTimeExpr if endTime.IsZero() { - toTimeExpr = &ast.FuncCallExpr{FnName: model.NewCIStr("CURRENT_TIMESTAMP")} + toTimeExpr = &ast.FuncCallExpr{FnName: ast.NewCIStr("CURRENT_TIMESTAMP")} } startTimeExpr = &ast.FuncCallExpr{ - FnName: model.NewCIStr("DATE_SUB"), + FnName: ast.NewCIStr("DATE_SUB"), Args: []ast.ExprNode{ toTimeExpr, op.Ts, @@ -213,7 +212,7 @@ func (e *Executor) parseCalibrateDuration(ctx context.Context) (startTime time.T // If endTime is set, duration will be ignored. if endTime.IsZero() { endTime, err = e.parseTsExpr(ctx, &ast.FuncCallExpr{ - FnName: model.NewCIStr("DATE_ADD"), + FnName: ast.NewCIStr("DATE_ADD"), Args: []ast.ExprNode{startTimeExpr, op.Ts, &ast.TimeUnitExpr{Unit: op.Unit}}, diff --git a/pkg/executor/internal/exec/BUILD.bazel b/pkg/executor/internal/exec/BUILD.bazel index e80cea2a91bbc..f7ae14d38010b 100644 --- a/pkg/executor/internal/exec/BUILD.bazel +++ b/pkg/executor/internal/exec/BUILD.bazel @@ -43,7 +43,7 @@ go_test( ":exec", "//pkg/domain", "//pkg/kv", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/sessionctx/stmtctx", "//pkg/statistics", "//pkg/statistics/handle/usage/indexusage", diff --git a/pkg/executor/internal/exec/indexusage_test.go b/pkg/executor/internal/exec/indexusage_test.go index 3621e79561b12..a380519425907 100644 --- a/pkg/executor/internal/exec/indexusage_test.go +++ b/pkg/executor/internal/exec/indexusage_test.go @@ -24,7 +24,7 @@ import ( "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/executor/internal/exec" "github.com/pingcap/tidb/pkg/kv" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" "github.com/pingcap/tidb/pkg/statistics" "github.com/pingcap/tidb/pkg/statistics/handle/usage/indexusage" @@ -220,7 +220,7 @@ func TestIndexUsageReporterWithRealData(t *testing.T) { tk.MustExec("use test") tk.MustExec("create table t (id_1 int, id_2 int, unique key idx_1(id_1), unique key idx_2(id_2))") - table, err := dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + table, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableID := table.Meta().ID idx1ID := int64(0) @@ -295,7 +295,7 @@ partition p1 values less than (20), partition p2 values less than (50), partition p3 values less than MAXVALUE)`) - table, err := dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + table, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) idx1ID := int64(0) for _, idx := range table.Indices() { @@ -366,7 +366,7 @@ partition p1 values less than (20), partition p2 values less than (50), partition p3 values less than MAXVALUE)`) - table, err := dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + table, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) idx1ID := int64(0) for _, idx := range table.Indices() { @@ -411,7 +411,7 @@ func TestDisableIndexUsageReporter(t *testing.T) { tk.MustExec("use test") tk.MustExec("create table t (id_1 int, id_2 int, unique key idx_1(id_1), unique key idx_2(id_2))") - table, err := dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + table, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableID := table.Meta().ID idx1ID := int64(0) @@ -459,7 +459,7 @@ func TestIndexUsageReporterWithClusterIndex(t *testing.T) { } testTableInfos := []testTableInfo{} for i := 0; i < 4; i++ { - table, err := dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr(fmt.Sprintf("t%d", i))) + table, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr(fmt.Sprintf("t%d", i))) require.NoError(t, err) tableID := table.Meta().ID pkID := int64(0) diff --git a/pkg/executor/internal/querywatch/BUILD.bazel b/pkg/executor/internal/querywatch/BUILD.bazel index bbd1c16906d26..88b8ddce61cc5 100644 --- a/pkg/executor/internal/querywatch/BUILD.bazel +++ b/pkg/executor/internal/querywatch/BUILD.bazel @@ -11,7 +11,6 @@ go_library( "//pkg/infoschema", "//pkg/parser", "//pkg/parser/ast", - "//pkg/parser/model", "//pkg/planner/util", "//pkg/resourcegroup", "//pkg/resourcegroup/runaway", diff --git a/pkg/executor/internal/querywatch/query_watch.go b/pkg/executor/internal/querywatch/query_watch.go index 1476cfec1c4e5..505f9f24d44f4 100644 --- a/pkg/executor/internal/querywatch/query_watch.go +++ b/pkg/executor/internal/querywatch/query_watch.go @@ -26,7 +26,6 @@ import ( "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/parser" "github.com/pingcap/tidb/pkg/parser/ast" - "github.com/pingcap/tidb/pkg/parser/model" plannerutil "github.com/pingcap/tidb/pkg/planner/util" "github.com/pingcap/tidb/pkg/resourcegroup" "github.com/pingcap/tidb/pkg/resourcegroup/runaway" @@ -89,14 +88,14 @@ func setWatchOption(ctx context.Context, } sql := stmts[0].Text() switch watchType { - case model.WatchNone: + case ast.WatchNone: return errors.Errorf("watch type must be specified") - case model.WatchExact: + case ast.WatchExact: record.WatchText = sql - case model.WatchSimilar: + case ast.WatchSimilar: _, digest := parser.NormalizeDigest(sql) record.WatchText = digest.String() - case model.WatchPlan: + case ast.WatchPlan: sqlExecutor := newSctx.GetSQLExecutor() if _, err := sqlExecutor.ExecuteInternal(ctx, fmt.Sprintf("explain %s", stmts[0].Text())); err != nil { return err diff --git a/pkg/executor/lockstats/BUILD.bazel b/pkg/executor/lockstats/BUILD.bazel index c0413217d7e18..64213ddde6bcb 100644 --- a/pkg/executor/lockstats/BUILD.bazel +++ b/pkg/executor/lockstats/BUILD.bazel @@ -13,7 +13,6 @@ go_library( "//pkg/executor/internal/exec", "//pkg/infoschema", "//pkg/parser/ast", - "//pkg/parser/model", "//pkg/statistics/handle/types", "//pkg/table/tables", "//pkg/util/chunk", @@ -31,7 +30,6 @@ go_test( "//pkg/infoschema", "//pkg/meta/model", "//pkg/parser/ast", - "//pkg/parser/model", "@com_github_stretchr_testify//require", ], ) diff --git a/pkg/executor/lockstats/lock_stats_executor.go b/pkg/executor/lockstats/lock_stats_executor.go index b6219fba84131..e16955d2b0250 100644 --- a/pkg/executor/lockstats/lock_stats_executor.go +++ b/pkg/executor/lockstats/lock_stats_executor.go @@ -23,7 +23,6 @@ import ( "github.com/pingcap/tidb/pkg/executor/internal/exec" "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/parser/ast" - "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/statistics/handle/types" "github.com/pingcap/tidb/pkg/table/tables" "github.com/pingcap/tidb/pkg/util/chunk" @@ -102,7 +101,7 @@ func (*LockExec) Open(context.Context) error { // populatePartitionIDAndNames returns the table ID and partition IDs for the given table name and partition names. func populatePartitionIDAndNames( table *ast.TableName, - partitionNames []model.CIStr, + partitionNames []ast.CIStr, is infoschema.InfoSchema, ) (int64, map[int64]string, error) { if len(partitionNames) == 0 { diff --git a/pkg/executor/lockstats/lock_stats_executor_test.go b/pkg/executor/lockstats/lock_stats_executor_test.go index 61ca33cb3dbc7..9b8a78fdfb2be 100644 --- a/pkg/executor/lockstats/lock_stats_executor_test.go +++ b/pkg/executor/lockstats/lock_stats_executor_test.go @@ -20,7 +20,6 @@ import ( "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/ast" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/stretchr/testify/require" ) @@ -30,11 +29,11 @@ func TestPopulatePartitionIDAndNames(t *testing.T) { }) table := &ast.TableName{ - Schema: pmodel.NewCIStr("test"), - Name: pmodel.NewCIStr("t1"), - PartitionNames: []pmodel.CIStr{ - pmodel.NewCIStr("p1"), - pmodel.NewCIStr("p2"), + Schema: ast.NewCIStr("test"), + Name: ast.NewCIStr("t1"), + PartitionNames: []ast.CIStr{ + ast.NewCIStr("p1"), + ast.NewCIStr("p2"), }, } @@ -59,16 +58,16 @@ func TestPopulateTableAndPartitionIDs(t *testing.T) { tables := []*ast.TableName{ { - Schema: pmodel.NewCIStr("test"), - Name: pmodel.NewCIStr("t1"), - PartitionNames: []pmodel.CIStr{ - pmodel.NewCIStr("p1"), - pmodel.NewCIStr("p2"), + Schema: ast.NewCIStr("test"), + Name: ast.NewCIStr("t1"), + PartitionNames: []ast.CIStr{ + ast.NewCIStr("p1"), + ast.NewCIStr("p2"), }, }, { - Schema: pmodel.NewCIStr("test"), - Name: pmodel.NewCIStr("t2"), + Schema: ast.NewCIStr("test"), + Name: ast.NewCIStr("t2"), }, } @@ -86,7 +85,7 @@ func TestPopulateTableAndPartitionIDs(t *testing.T) { func tInfo(id int, tableName string, partitionNames ...string) *model.TableInfo { tbl := &model.TableInfo{ ID: int64(id), - Name: pmodel.NewCIStr(tableName), + Name: ast.NewCIStr(tableName), } if len(partitionNames) > 0 { tbl.Partition = &model.PartitionInfo{ @@ -95,7 +94,7 @@ func tInfo(id int, tableName string, partitionNames ...string) *model.TableInfo for i, partitionName := range partitionNames { tbl.Partition.Definitions = append(tbl.Partition.Definitions, model.PartitionDefinition{ ID: int64(id + 1 + i), - Name: pmodel.NewCIStr(partitionName), + Name: ast.NewCIStr(partitionName), }) } } diff --git a/pkg/executor/partition_table_test.go b/pkg/executor/partition_table_test.go index 002690ac0c892..19585a7bb52e2 100644 --- a/pkg/executor/partition_table_test.go +++ b/pkg/executor/partition_table_test.go @@ -26,7 +26,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/infoschema" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/testkit/external" "github.com/pingcap/tidb/pkg/util/dbterror/exeerrors" @@ -148,7 +148,7 @@ func TestPartitionInfoDisable(t *testing.T) { PARTITION p202011 VALUES LESS THAN ("2020-12-01") )`) is := tk.Session().GetInfoSchema().(infoschema.InfoSchema) - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t_info_null")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t_info_null")) require.NoError(t, err) tbInfo := tbl.Meta() diff --git a/pkg/executor/point_get.go b/pkg/executor/point_get.go index fba16bc25d255..462d9fb37c247 100644 --- a/pkg/executor/point_get.go +++ b/pkg/executor/point_get.go @@ -29,7 +29,7 @@ import ( "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" plannercore "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/sessionctx" @@ -139,7 +139,7 @@ type PointGetExecutor struct { handle kv.Handle idxInfo *model.IndexInfo partitionDefIdx *int - partitionNames []pmodel.CIStr + partitionNames []ast.CIStr idxKey kv.Key handleVal []byte idxVals []types.Datum @@ -178,7 +178,7 @@ func GetPhysID(tblInfo *model.TableInfo, idx *int) int64 { return tblInfo.ID } -func matchPartitionNames(pid int64, partitionNames []pmodel.CIStr, pi *model.PartitionInfo) bool { +func matchPartitionNames(pid int64, partitionNames []ast.CIStr, pi *model.PartitionInfo) bool { if len(partitionNames) == 0 { return true } @@ -667,7 +667,7 @@ func (e *PointGetExecutor) get(ctx context.Context, key kv.Key) ([]byte, error) } lock := e.tblInfo.Lock - if lock != nil && (lock.Tp == pmodel.TableLockRead || lock.Tp == pmodel.TableLockReadOnly) { + if lock != nil && (lock.Tp == ast.TableLockRead || lock.Tp == ast.TableLockReadOnly) { if e.Ctx().GetSessionVars().EnablePointGetCache { cacheDB := e.Ctx().GetStore().GetMemCache() val, err = cacheDB.UnionGet(ctx, e.tblInfo.ID, e.snapshot, key) diff --git a/pkg/executor/prepared_test.go b/pkg/executor/prepared_test.go index 0349aebd6a019..da15072fe4102 100644 --- a/pkg/executor/prepared_test.go +++ b/pkg/executor/prepared_test.go @@ -23,8 +23,8 @@ import ( "testing" "github.com/pingcap/tidb/pkg/meta/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/auth" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/terror" plannercore "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/sessionctx/variable" @@ -1040,7 +1040,7 @@ func TestPrepareStmtAfterIsolationReadChange(t *testing.T) { // create virtual tiflash replica. is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ Count: 1, diff --git a/pkg/executor/show.go b/pkg/executor/show.go index d8578b8af70a1..d2931285aa1a8 100644 --- a/pkg/executor/show.go +++ b/pkg/executor/show.go @@ -47,7 +47,6 @@ import ( "github.com/pingcap/tidb/pkg/parser/auth" "github.com/pingcap/tidb/pkg/parser/charset" parserformat "github.com/pingcap/tidb/pkg/parser/format" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/parser/tidb" @@ -90,12 +89,12 @@ type ShowExec struct { exec.BaseExecutor Tp ast.ShowStmtType // Databases/Tables/Columns/.... - DBName pmodel.CIStr + DBName ast.CIStr Table *resolve.TableNameW // Used for showing columns. - Partition pmodel.CIStr // Used for showing partition + Partition ast.CIStr // Used for showing partition Column *ast.ColumnName // Used for `desc table column`. - IndexName pmodel.CIStr // Used for show table regions. - ResourceGroupName pmodel.CIStr // Used for showing resource group + IndexName ast.CIStr // Used for show table regions. + ResourceGroupName ast.CIStr // Used for showing resource group Flag int // Some flag parsed from sql, such as FULL. Roles []*auth.RoleIdentity // Used for show grants. User *auth.UserIdentity // Used by show grants, show create user. @@ -300,7 +299,7 @@ func (v *visibleChecker) Enter(in ast.Node) (out ast.Node, skipChildren bool) { if schema == "" { schema = v.defaultDB } - if !v.is.TableExists(pmodel.NewCIStr(schema), x.Name) { + if !v.is.TableExists(ast.NewCIStr(schema), x.Name) { return in, true } activeRoles := v.ctx.GetSessionVars().ActiveRoles @@ -489,7 +488,7 @@ func (*ShowExec) fetchShowOpenTables() error { // showInfo represents the result of `SHOW TABLES`. type showInfo struct { - Name pmodel.CIStr + Name ast.CIStr // only used for show full tables TableType string } @@ -510,7 +509,7 @@ func (e *ShowExec) getTableType(tb *model.TableInfo) string { // fetchShowInfoByName fetches the show info for `SHOW TABLES like 'xxx'` func (e *ShowExec) fetchShowInfoByName(ctx context.Context, name string) ([]*showInfo, error) { - tb, err := e.is.TableByName(ctx, e.DBName, pmodel.NewCIStr(name)) + tb, err := e.is.TableByName(ctx, e.DBName, ast.NewCIStr(name)) if err != nil { // do nothing if table not exists if infoschema.ErrTableNotExists.Equal(err) { @@ -1030,7 +1029,7 @@ func ConstructResultOfShowCreateTable(ctx sessionctx.Context, tableInfo *model.T return constructResultOfShowCreateTable(ctx, nil, tableInfo, allocators, buf) } -func constructResultOfShowCreateTable(ctx sessionctx.Context, dbName *pmodel.CIStr, tableInfo *model.TableInfo, allocators autoid.Allocators, buf *bytes.Buffer) (err error) { +func constructResultOfShowCreateTable(ctx sessionctx.Context, dbName *ast.CIStr, tableInfo *model.TableInfo, allocators autoid.Allocators, buf *bytes.Buffer) (err error) { if tableInfo.IsView() { fetchShowCreateTable4View(ctx, tableInfo, buf) return nil @@ -1241,7 +1240,7 @@ func constructResultOfShowCreateTable(ctx sessionctx.Context, dbName *pmodel.CIS if idxInfo.Comment != "" { fmt.Fprintf(buf, ` COMMENT '%s'`, format.OutputFormat(idxInfo.Comment)) } - if idxInfo.Tp == pmodel.IndexTypeHypo { + if idxInfo.Tp == ast.IndexTypeHypo { fmt.Fprintf(buf, ` /* HYPO INDEX */`) } if idxInfo.Primary { @@ -1278,11 +1277,11 @@ func constructResultOfShowCreateTable(ctx sessionctx.Context, dbName *pmodel.CIS refColNames = append(refColNames, stringutil.Escape(refCol.O, sqlMode)) } fmt.Fprintf(buf, "(%s)", strings.Join(refColNames, ",")) - if pmodel.ReferOptionType(fk.OnDelete) != 0 { - fmt.Fprintf(buf, " ON DELETE %s", pmodel.ReferOptionType(fk.OnDelete).String()) + if ast.ReferOptionType(fk.OnDelete) != 0 { + fmt.Fprintf(buf, " ON DELETE %s", ast.ReferOptionType(fk.OnDelete).String()) } - if pmodel.ReferOptionType(fk.OnUpdate) != 0 { - fmt.Fprintf(buf, " ON UPDATE %s", pmodel.ReferOptionType(fk.OnUpdate).String()) + if ast.ReferOptionType(fk.OnUpdate) != 0 { + fmt.Fprintf(buf, " ON UPDATE %s", ast.ReferOptionType(fk.OnUpdate).String()) } if fk.Version < model.FKVersion1 { buf.WriteString(" /* FOREIGN KEY INVALID */") @@ -2392,7 +2391,7 @@ func (e *ShowExec) fetchShowImportJobs(ctx context.Context) error { // tryFillViewColumnType fill the columns type info of a view. // Because view's underlying table's column could change or recreate, so view's column type may change over time. // To avoid this situation we need to generate a logical plan and extract current column types from Schema. -func tryFillViewColumnType(ctx context.Context, sctx sessionctx.Context, is infoschema.InfoSchema, dbName pmodel.CIStr, tbl *model.TableInfo) error { +func tryFillViewColumnType(ctx context.Context, sctx sessionctx.Context, is infoschema.InfoSchema, dbName ast.CIStr, tbl *model.TableInfo) error { if !tbl.IsView() { return nil } diff --git a/pkg/executor/show_placement.go b/pkg/executor/show_placement.go index 6cb2ff7d7f12d..21083d29be2c7 100644 --- a/pkg/executor/show_placement.go +++ b/pkg/executor/show_placement.go @@ -30,7 +30,6 @@ import ( infoschemacontext "github.com/pingcap/tidb/pkg/infoschema/context" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/ast" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/privilege" "github.com/pingcap/tidb/pkg/table" @@ -283,7 +282,7 @@ func (e *ShowExec) fetchRangesPlacementPlocy(ctx context.Context) error { if err != nil { return err } - policy, ok := e.is.PolicyByName(pmodel.NewCIStr(policyName)) + policy, ok := e.is.PolicyByName(ast.NewCIStr(policyName)) if !ok { return errors.Errorf("Policy with name '%s' not found", policyName) } @@ -304,7 +303,7 @@ func (e *ShowExec) fetchAllDBPlacements(ctx context.Context, scheduleState map[i activeRoles := e.Ctx().GetSessionVars().ActiveRoles dbs := e.is.AllSchemaNames() - slices.SortFunc(dbs, func(i, j pmodel.CIStr) int { return cmp.Compare(i.O, j.O) }) + slices.SortFunc(dbs, func(i, j ast.CIStr) int { return cmp.Compare(i.O, j.O) }) for _, dbName := range dbs { if checker != nil && e.Ctx().GetSessionVars().User != nil && !checker.DBIsVisible(activeRoles, dbName.O) { @@ -360,7 +359,7 @@ func (e *ShowExec) fetchAllTablePlacements(ctx context.Context, scheduleState ma activeRoles := e.Ctx().GetSessionVars().ActiveRoles dbs := e.is.AllSchemaNames() - slices.SortFunc(dbs, func(i, j pmodel.CIStr) int { return cmp.Compare(i.O, j.O) }) + slices.SortFunc(dbs, func(i, j ast.CIStr) int { return cmp.Compare(i.O, j.O) }) tbls := e.is.ListTablesWithSpecialAttribute(infoschemacontext.AllSpecialAttribute) for _, db := range tbls { diff --git a/pkg/executor/show_placement_test.go b/pkg/executor/show_placement_test.go index f0b19cab0a395..c9a2416512d47 100644 --- a/pkg/executor/show_placement_test.go +++ b/pkg/executor/show_placement_test.go @@ -20,8 +20,8 @@ import ( "testing" "github.com/pingcap/tidb/pkg/domain/infosync" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/auth" - "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/tablecodec" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/util/codec" @@ -506,7 +506,7 @@ func TestShowPlacementHandleRegionStatus(t *testing.T) { mockGetState := func(tblName, partition string, state string) *mock.Call { is := tk.Session().GetDomainInfoSchema() - tbl, err := is.TableInfoByName(model.NewCIStr("test"), model.NewCIStr(tblName)) + tbl, err := is.TableInfoByName(ast.NewCIStr("test"), ast.NewCIStr(tblName)) require.NoError(t, err) tblID := tbl.ID if partition != "" { diff --git a/pkg/executor/show_stats_test.go b/pkg/executor/show_stats_test.go index 94bb106064e75..4f19a5550aa54 100644 --- a/pkg/executor/show_stats_test.go +++ b/pkg/executor/show_stats_test.go @@ -22,7 +22,7 @@ import ( "github.com/docker/go-units" "github.com/pingcap/tidb/pkg/domain/infosync" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/testkit" "github.com/stretchr/testify/require" @@ -368,9 +368,9 @@ func TestShowColumnStatsUsage(t *testing.T) { tk.MustExec("create table t2 (a int, b int) partition by range(a) (partition p0 values less than (10), partition p1 values less than (20), partition p2 values less than maxvalue)") is := dom.InfoSchema() - t1, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t1")) + t1, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t1")) require.NoError(t, err) - t2, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t2")) + t2, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t2")) require.NoError(t, err) tk.MustExec(fmt.Sprintf("insert into mysql.column_stats_usage values (%d, %d, null, '2021-10-20 08:00:00')", t1.Meta().ID, t1.Meta().Columns[0].ID)) tk.MustExec(fmt.Sprintf("insert into mysql.column_stats_usage values (%d, %d, '2021-10-20 09:00:00', null)", t2.Meta().ID, t2.Meta().Columns[0].ID)) diff --git a/pkg/executor/simple.go b/pkg/executor/simple.go index ae4e6517e8a45..044aeb8114829 100644 --- a/pkg/executor/simple.go +++ b/pkg/executor/simple.go @@ -43,7 +43,6 @@ import ( "github.com/pingcap/tidb/pkg/meta" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/auth" - "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/planner/core/resolve" @@ -585,7 +584,7 @@ func (e *SimpleExec) dbAccessDenied(dbname string) error { } func (e *SimpleExec) executeUse(s *ast.UseStmt) error { - dbname := model.NewCIStr(s.DBName) + dbname := ast.NewCIStr(s.DBName) checker := privilege.GetPrivilegeManager(e.Ctx()) if checker != nil && e.Ctx().GetSessionVars().User != nil { @@ -1088,7 +1087,7 @@ func (e *SimpleExec) executeCreateUser(ctx context.Context, s *ast.CreateUserStm // check if specified resource group exists if resourceGroupName != resourcegroup.DefaultResourceGroupName && resourceGroupName != "" { - _, exists := e.is.ResourceGroupByName(model.NewCIStr(resourceGroupName)) + _, exists := e.is.ResourceGroupByName(ast.NewCIStr(resourceGroupName)) if !exists { return infoschema.ErrResourceGroupNotExists.GenWithStackByArgs(resourceGroupName) } @@ -1947,7 +1946,7 @@ func (e *SimpleExec) executeAlterUser(ctx context.Context, s *ast.AlterUserStmt) // check if specified resource group exists resourceGroupName := strings.ToLower(s.ResourceGroupNameOption.Value) if resourceGroupName != resourcegroup.DefaultResourceGroupName && s.ResourceGroupNameOption.Value != "" { - _, exists := e.is.ResourceGroupByName(model.NewCIStr(resourceGroupName)) + _, exists := e.is.ResourceGroupByName(ast.NewCIStr(resourceGroupName)) if !exists { return infoschema.ErrResourceGroupNotExists.GenWithStackByArgs(resourceGroupName) } @@ -2960,7 +2959,7 @@ func (e *SimpleExec) executeAlterRange(s *ast.AlterRangeStmt) error { return errors.New("only support alter range policy") } bundle := &placement.Bundle{} - policyName := model.NewCIStr(s.PlacementOption.StrValue) + policyName := ast.NewCIStr(s.PlacementOption.StrValue) if policyName.L != placement.DefaultKwd { policy, ok := e.is.PolicyByName(policyName) if !ok { diff --git a/pkg/executor/slow_query_sql_test.go b/pkg/executor/slow_query_sql_test.go index 704a8c4576d6c..fa33e0a28ab2e 100644 --- a/pkg/executor/slow_query_sql_test.go +++ b/pkg/executor/slow_query_sql_test.go @@ -25,8 +25,8 @@ import ( "github.com/pingcap/tidb/pkg/config" "github.com/pingcap/tidb/pkg/executor" "github.com/pingcap/tidb/pkg/meta/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/auth" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/testkit/testdata" "github.com/pingcap/tidb/pkg/util/logutil" @@ -413,7 +413,7 @@ func TestWarningsInSlowQuery(t *testing.T) { tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int, b int, c int, d int, e int, f int, g int, h set('11', '22', '33')," + "primary key (a), unique key c_d_e (c, d, e), unique key f (f), unique key f_g (f, g), key g (g))") - tbl, err := dom.InfoSchema().TableByName(context.Background(), pmodel.CIStr{O: "test", L: "test"}, pmodel.CIStr{O: "t", L: "t"}) + tbl, err := dom.InfoSchema().TableByName(context.Background(), ast.CIStr{O: "test", L: "test"}, ast.CIStr{O: "t", L: "t"}) require.NoError(t, err) tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{Count: 1, Available: true} diff --git a/pkg/executor/slow_query_test.go b/pkg/executor/slow_query_test.go index 069b0cb8c40f7..7234a479a198b 100644 --- a/pkg/executor/slow_query_test.go +++ b/pkg/executor/slow_query_test.go @@ -30,7 +30,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb/pkg/config" "github.com/pingcap/tidb/pkg/infoschema" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/terror" plannercore "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/sessionctx" @@ -65,7 +65,7 @@ func newSlowQueryRetriever() (*slowQueryRetriever, error) { return nil, err } is := newISBuilder.Build(math.MaxUint64) - tbl, err := is.TableByName(context.Background(), util.InformationSchemaName, model.NewCIStr(infoschema.TableSlowQuery)) + tbl, err := is.TableByName(context.Background(), util.InformationSchemaName, ast.NewCIStr(infoschema.TableSlowQuery)) if err != nil { return nil, err } diff --git a/pkg/executor/split.go b/pkg/executor/split.go index 594ffff6dc4e9..fd067641d56b6 100644 --- a/pkg/executor/split.go +++ b/pkg/executor/split.go @@ -25,7 +25,7 @@ import ( "github.com/pingcap/tidb/pkg/executor/internal/exec" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/util" "github.com/pingcap/tidb/pkg/sessionctx" @@ -47,7 +47,7 @@ type SplitIndexRegionExec struct { exec.BaseExecutor tableInfo *model.TableInfo - partitionNames []pmodel.CIStr + partitionNames []ast.CIStr indexInfo *model.IndexInfo lower []types.Datum upper []types.Datum @@ -271,7 +271,7 @@ type SplitTableRegionExec struct { exec.BaseExecutor tableInfo *model.TableInfo - partitionNames []pmodel.CIStr + partitionNames []ast.CIStr lower []types.Datum upper []types.Datum num int diff --git a/pkg/executor/split_test.go b/pkg/executor/split_test.go index 43ef97dead7bb..78ce061cc29b0 100644 --- a/pkg/executor/split_test.go +++ b/pkg/executor/split_test.go @@ -26,7 +26,7 @@ import ( "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/util" "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" @@ -39,11 +39,11 @@ import ( func TestSplitIndex(t *testing.T) { tbInfo := &model.TableInfo{ - Name: pmodel.NewCIStr("t1"), + Name: ast.NewCIStr("t1"), ID: rand.Int63(), Columns: []*model.ColumnInfo{ { - Name: pmodel.NewCIStr("c0"), + Name: ast.NewCIStr("c0"), ID: 1, Offset: 1, DefaultValue: 0, @@ -55,14 +55,14 @@ func TestSplitIndex(t *testing.T) { idxCols := []*model.IndexColumn{{Name: tbInfo.Columns[0].Name, Offset: 0, Length: types.UnspecifiedLength}} idxInfo := &model.IndexInfo{ ID: 2, - Name: pmodel.NewCIStr("idx1"), - Table: pmodel.NewCIStr("t1"), + Name: ast.NewCIStr("idx1"), + Table: ast.NewCIStr("t1"), Columns: idxCols, State: model.StatePublic, } firstIdxInfo0 := idxInfo.Clone() firstIdxInfo0.ID = 1 - firstIdxInfo0.Name = pmodel.NewCIStr("idx") + firstIdxInfo0.Name = ast.NewCIStr("idx") tbInfo.Indices = []*model.IndexInfo{firstIdxInfo0, idxInfo} // Test for int index. @@ -241,11 +241,11 @@ func TestSplitIndex(t *testing.T) { func TestSplitTable(t *testing.T) { tbInfo := &model.TableInfo{ - Name: pmodel.NewCIStr("t1"), + Name: ast.NewCIStr("t1"), ID: rand.Int63(), Columns: []*model.ColumnInfo{ { - Name: pmodel.NewCIStr("c0"), + Name: ast.NewCIStr("c0"), ID: 1, Offset: 1, DefaultValue: 0, @@ -318,11 +318,11 @@ func TestSplitTable(t *testing.T) { func TestStepShouldLargeThanMinStep(t *testing.T) { ctx := mock.NewContext() tbInfo := &model.TableInfo{ - Name: pmodel.NewCIStr("t1"), + Name: ast.NewCIStr("t1"), ID: rand.Int63(), Columns: []*model.ColumnInfo{ { - Name: pmodel.NewCIStr("c0"), + Name: ast.NewCIStr("c0"), ID: 1, Offset: 1, DefaultValue: 0, @@ -345,7 +345,7 @@ func TestStepShouldLargeThanMinStep(t *testing.T) { func TestClusterIndexSplitTable(t *testing.T) { tbInfo := &model.TableInfo{ - Name: pmodel.NewCIStr("t"), + Name: ast.NewCIStr("t"), ID: 1, IsCommonHandle: true, CommonHandleVersion: 1, @@ -362,21 +362,21 @@ func TestClusterIndexSplitTable(t *testing.T) { }, Columns: []*model.ColumnInfo{ { - Name: pmodel.NewCIStr("c0"), + Name: ast.NewCIStr("c0"), ID: 1, Offset: 0, State: model.StatePublic, FieldType: *types.NewFieldType(mysql.TypeDouble), }, { - Name: pmodel.NewCIStr("c1"), + Name: ast.NewCIStr("c1"), ID: 2, Offset: 1, State: model.StatePublic, FieldType: *types.NewFieldType(mysql.TypeLonglong), }, { - Name: pmodel.NewCIStr("c2"), + Name: ast.NewCIStr("c2"), ID: 3, Offset: 2, State: model.StatePublic, diff --git a/pkg/executor/stmtsummary_test.go b/pkg/executor/stmtsummary_test.go index 9ef4960b4798d..c8f504f41c179 100644 --- a/pkg/executor/stmtsummary_test.go +++ b/pkg/executor/stmtsummary_test.go @@ -22,7 +22,7 @@ import ( "time" "github.com/pingcap/tidb/pkg/infoschema" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util" @@ -37,7 +37,7 @@ func TestStmtSummaryRetriverV2_TableStatementsSummary(t *testing.T) { err := infoSchemaBuilder.InitWithDBInfos(nil, nil, nil, 0) require.NoError(t, err) infoSchema := infoSchemaBuilder.Build(math.MaxUint64) - table, err := infoSchema.TableByName(context.Background(), util.InformationSchemaName, model.NewCIStr(infoschema.TableStatementsSummary)) + table, err := infoSchema.TableByName(context.Background(), util.InformationSchemaName, ast.NewCIStr(infoschema.TableStatementsSummary)) require.NoError(t, err) columns := table.Meta().Columns @@ -82,7 +82,7 @@ func TestStmtSummaryRetriverV2_TableStatementsSummaryEvicted(t *testing.T) { err := infoSchemaBuilder.InitWithDBInfos(nil, nil, nil, 0) require.NoError(t, err) infoSchema := infoSchemaBuilder.Build(math.MaxUint64) - table, err := infoSchema.TableByName(context.Background(), util.InformationSchemaName, model.NewCIStr(infoschema.TableStatementsSummaryEvicted)) + table, err := infoSchema.TableByName(context.Background(), util.InformationSchemaName, ast.NewCIStr(infoschema.TableStatementsSummaryEvicted)) require.NoError(t, err) columns := table.Meta().Columns @@ -162,7 +162,7 @@ func TestStmtSummaryRetriverV2_TableStatementsSummaryHistory(t *testing.T) { err = infoSchemaBuilder.InitWithDBInfos(nil, nil, nil, 0) require.NoError(t, err) infoSchema := infoSchemaBuilder.Build(math.MaxUint64) - table, err := infoSchema.TableByName(context.Background(), util.InformationSchemaName, model.NewCIStr(infoschema.TableStatementsSummaryHistory)) + table, err := infoSchema.TableByName(context.Background(), util.InformationSchemaName, ast.NewCIStr(infoschema.TableStatementsSummaryHistory)) require.NoError(t, err) columns := table.Meta().Columns diff --git a/pkg/executor/test/admintest/BUILD.bazel b/pkg/executor/test/admintest/BUILD.bazel index f0794f6281482..214217f6113bd 100644 --- a/pkg/executor/test/admintest/BUILD.bazel +++ b/pkg/executor/test/admintest/BUILD.bazel @@ -18,7 +18,7 @@ go_test( "//pkg/kv", "//pkg/meta/autoid", "//pkg/meta/model", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/session", "//pkg/sessionctx", "//pkg/sessionctx/variable", diff --git a/pkg/executor/test/admintest/admin_test.go b/pkg/executor/test/admintest/admin_test.go index bf90c761f1bc5..dda1139c486f7 100644 --- a/pkg/executor/test/admintest/admin_test.go +++ b/pkg/executor/test/admintest/admin_test.go @@ -31,7 +31,7 @@ import ( "github.com/pingcap/tidb/pkg/executor" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + 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" @@ -87,8 +87,8 @@ func TestAdminRecoverIndex(t *testing.T) { sctx.Store = store ctx := sctx.GetTableCtx() is := domain.InfoSchema() - dbName := pmodel.NewCIStr("test") - tblName := pmodel.NewCIStr("admin_test") + dbName := ast.NewCIStr("test") + tblName := ast.NewCIStr("admin_test") tbl, err := is.TableByName(context.Background(), dbName, tblName) require.NoError(t, err) @@ -177,8 +177,8 @@ func TestAdminRecoverIndex(t *testing.T) { sctx.Store = store ctx = sctx.GetTableCtx() is = domain.InfoSchema() - dbName = pmodel.NewCIStr("test") - tblName = pmodel.NewCIStr("admin_test") + dbName = ast.NewCIStr("test") + tblName = ast.NewCIStr("admin_test") tbl, err = is.TableByName(context.Background(), dbName, tblName) require.NoError(t, err) @@ -223,8 +223,8 @@ func TestAdminRecoverMVIndex(t *testing.T) { sctx.Store = store ctx := sctx.GetTableCtx() is := domain.InfoSchema() - dbName := pmodel.NewCIStr("test") - tblName := pmodel.NewCIStr("t") + dbName := ast.NewCIStr("test") + tblName := ast.NewCIStr("t") tbl, err := is.TableByName(context.Background(), dbName, tblName) require.NoError(t, err) tblInfo := tbl.Meta() @@ -268,8 +268,8 @@ func TestAdminCleanupMVIndex(t *testing.T) { sctx.Store = store ctx := sctx.GetTableCtx() is := domain.InfoSchema() - dbName := pmodel.NewCIStr("test") - tblName := pmodel.NewCIStr("t") + dbName := ast.NewCIStr("test") + tblName := ast.NewCIStr("t") tbl, err := is.TableByName(context.Background(), dbName, tblName) require.NoError(t, err) tblInfo := tbl.Meta() @@ -303,8 +303,8 @@ func TestClusteredIndexAdminRecoverIndex(t *testing.T) { tk.MustExec("create database test_cluster_index_admin_recover;") tk.MustExec("use test_cluster_index_admin_recover;") tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn - dbName := pmodel.NewCIStr("test_cluster_index_admin_recover") - tblName := pmodel.NewCIStr("t") + dbName := ast.NewCIStr("test_cluster_index_admin_recover") + tblName := ast.NewCIStr("t") // Test no corruption case. tk.MustExec("create table t (a varchar(255), b int, c char(10), primary key(a, c), index idx(b), index idx1(c));") @@ -367,8 +367,8 @@ func TestAdminRecoverPartitionTableIndex(t *testing.T) { ctx := mock.NewContext() ctx.Store = store is := domain.InfoSchema() - dbName := pmodel.NewCIStr("test") - tblName := pmodel.NewCIStr("admin_test") + dbName := ast.NewCIStr("test") + tblName := ast.NewCIStr("admin_test") tbl, err := is.TableByName(context.Background(), dbName, tblName) require.NoError(t, err) return tbl @@ -436,8 +436,8 @@ func TestAdminRecoverIndex1(t *testing.T) { sctx := mock.NewContext() sctx.Store = store ctx := sctx.GetTableCtx() - dbName := pmodel.NewCIStr("test") - tblName := pmodel.NewCIStr("admin_test") + dbName := ast.NewCIStr("test") + tblName := ast.NewCIStr("admin_test") tk.MustExec("use test") tk.MustExec("drop table if exists admin_test") tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly @@ -506,8 +506,8 @@ func TestAdminCleanupIndex(t *testing.T) { sctx.Store = store ctx := sctx.GetTableCtx() is := domain.InfoSchema() - dbName := pmodel.NewCIStr("test") - tblName := pmodel.NewCIStr("admin_test") + dbName := ast.NewCIStr("test") + tblName := ast.NewCIStr("admin_test") tbl, err := is.TableByName(context.Background(), dbName, tblName) require.NoError(t, err) @@ -575,8 +575,8 @@ func TestAdminCleanupIndexForPartitionTable(t *testing.T) { ctx := mock.NewContext() ctx.Store = store is := domain.InfoSchema() - dbName := pmodel.NewCIStr("test") - tblName := pmodel.NewCIStr("admin_test") + dbName := ast.NewCIStr("test") + tblName := ast.NewCIStr("admin_test") tbl, err := is.TableByName(context.Background(), dbName, tblName) require.NoError(t, err) return tbl @@ -666,8 +666,8 @@ func TestAdminCleanupIndexPKNotHandle(t *testing.T) { sctx.Store = store ctx := sctx.GetTableCtx() is := domain.InfoSchema() - dbName := pmodel.NewCIStr("test") - tblName := pmodel.NewCIStr("admin_test") + dbName := ast.NewCIStr("test") + tblName := ast.NewCIStr("admin_test") tbl, err := is.TableByName(context.Background(), dbName, tblName) require.NoError(t, err) @@ -717,8 +717,8 @@ func TestAdminCleanupIndexMore(t *testing.T) { sctx.Store = store ctx := sctx.GetTableCtx() is := domain.InfoSchema() - dbName := pmodel.NewCIStr("test") - tblName := pmodel.NewCIStr("admin_test") + dbName := ast.NewCIStr("test") + tblName := ast.NewCIStr("admin_test") tbl, err := is.TableByName(context.Background(), dbName, tblName) require.NoError(t, err) @@ -786,7 +786,7 @@ func TestClusteredAdminCleanupIndex(t *testing.T) { sctx := mock.NewContext() sctx.Store = store ctx := sctx.GetTableCtx() - tbl, err := domain.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("admin_test")) + tbl, err := domain.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("admin_test")) require.NoError(t, err) // cleanup clustered primary key takes no effect. @@ -864,8 +864,8 @@ func TestAdminCheckTableWithMultiValuedIndex(t *testing.T) { sctx.Store = store ctx := sctx.GetTableCtx() is := domain.InfoSchema() - dbName := pmodel.NewCIStr("test") - tblName := pmodel.NewCIStr("t") + dbName := ast.NewCIStr("test") + tblName := ast.NewCIStr("t") tbl, err := is.TableByName(context.Background(), dbName, tblName) require.NoError(t, err) tblInfo := tbl.Meta() @@ -919,8 +919,8 @@ func TestAdminCheckPartitionTableFailed(t *testing.T) { sctx.Store = store ctx := sctx.GetTableCtx() is := domain.InfoSchema() - dbName := pmodel.NewCIStr("test") - tblName := pmodel.NewCIStr("admin_test_p") + dbName := ast.NewCIStr("test") + tblName := ast.NewCIStr("admin_test_p") tbl, err := is.TableByName(context.Background(), dbName, tblName) require.NoError(t, err) tblInfo := tbl.Meta() @@ -1052,7 +1052,7 @@ func newInconsistencyKit(t *testing.T, tk *testkit.AsyncTestKit, opt *kitOpt) *i func (tk *inconsistencyTestKit) rebuild() { tk.MustExec(tk.ctx, "truncate table "+tblName) is := domain.GetDomain(testkit.TryRetrieveSession(tk.ctx)).InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr(dbName), pmodel.NewCIStr(tblName)) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr(dbName), ast.NewCIStr(tblName)) require.NoError(tk.t, err) tk.uniqueIndex = tables.NewIndex(tbl.Meta().ID, tbl.Meta(), tbl.Meta().Indices[0]) tk.plainIndex = tables.NewIndex(tbl.Meta().ID, tbl.Meta(), tbl.Meta().Indices[1]) @@ -1304,8 +1304,8 @@ func TestAdminCheckWithSnapshot(t *testing.T) { sctx.Store = store ctx := sctx.GetTableCtx() is := domain.InfoSchema() - dbName := pmodel.NewCIStr("test") - tblName := pmodel.NewCIStr("admin_t_s") + dbName := ast.NewCIStr("test") + tblName := ast.NewCIStr("admin_t_s") tbl, err := is.TableByName(context.Background(), dbName, tblName) require.NoError(t, err) @@ -1363,8 +1363,8 @@ func TestAdminCheckTableFailed(t *testing.T) { sctx.Store = store ctx := sctx.GetTableCtx() is := domain.InfoSchema() - dbName := pmodel.NewCIStr("test") - tblName := pmodel.NewCIStr("admin_test") + dbName := ast.NewCIStr("test") + tblName := ast.NewCIStr("admin_test") tbl, err := is.TableByName(context.Background(), dbName, tblName) require.NoError(t, err) tblInfo := tbl.Meta() @@ -1528,8 +1528,8 @@ func TestAdminCheckTableErrorLocate(t *testing.T) { sctx.Store = store ctx := sctx.GetTableCtx() is := domain.InfoSchema() - dbName := pmodel.NewCIStr("test") - tblName := pmodel.NewCIStr("admin_test") + dbName := ast.NewCIStr("test") + tblName := ast.NewCIStr("admin_test") tbl, err := is.TableByName(context.Background(), dbName, tblName) require.NoError(t, err) tblInfo := tbl.Meta() @@ -1660,8 +1660,8 @@ func TestAdminCheckTableErrorLocateForClusterIndex(t *testing.T) { sctx.Store = store ctx := sctx.GetTableCtx() is := domain.InfoSchema() - dbName := pmodel.NewCIStr("test") - tblName := pmodel.NewCIStr("admin_test") + dbName := ast.NewCIStr("test") + tblName := ast.NewCIStr("admin_test") tbl, err := is.TableByName(context.Background(), dbName, tblName) require.NoError(t, err) tblInfo := tbl.Meta() @@ -1794,8 +1794,8 @@ func TestAdminCleanUpGlobalIndex(t *testing.T) { sctx := mock.NewContext() sctx.Store = store is := domain.InfoSchema() - dbName := pmodel.NewCIStr("test") - tblName := pmodel.NewCIStr("admin_test") + dbName := ast.NewCIStr("test") + tblName := ast.NewCIStr("admin_test") tbl, err := is.TableByName(context.Background(), dbName, tblName) require.NoError(t, err) tblInfo := tbl.Meta() @@ -1838,8 +1838,8 @@ func TestAdminRecoverGlobalIndex(t *testing.T) { sctx := mock.NewContext() sctx.Store = store is := domain.InfoSchema() - dbName := pmodel.NewCIStr("test") - tblName := pmodel.NewCIStr("admin_test") + dbName := ast.NewCIStr("test") + tblName := ast.NewCIStr("admin_test") tbl, err := is.TableByName(context.Background(), dbName, tblName) require.NoError(t, err) tblInfo := tbl.Meta() @@ -1887,8 +1887,8 @@ func TestAdminCheckGlobalIndex(t *testing.T) { sctx := mock.NewContext() sctx.Store = store is := domain.InfoSchema() - dbName := pmodel.NewCIStr("test") - tblName := pmodel.NewCIStr("admin_test") + dbName := ast.NewCIStr("test") + tblName := ast.NewCIStr("admin_test") tbl, err := is.TableByName(context.Background(), dbName, tblName) require.NoError(t, err) tblInfo := tbl.Meta() @@ -1983,8 +1983,8 @@ func TestAdminCheckGlobalIndexWithClusterIndex(t *testing.T) { sctx := mock.NewContext() sctx.Store = store is := domain.InfoSchema() - dbName := pmodel.NewCIStr("test") - tblName := pmodel.NewCIStr("admin_test") + dbName := ast.NewCIStr("test") + tblName := ast.NewCIStr("admin_test") tbl, err := is.TableByName(context.Background(), dbName, tblName) require.NoError(t, err) tblInfo := tbl.Meta() @@ -2114,8 +2114,8 @@ func TestAdminCheckGeneratedColumns(t *testing.T) { sctx.Store = store ctx := sctx.GetTableCtx() is := domain.InfoSchema() - dbName := pmodel.NewCIStr("test") - tblName := pmodel.NewCIStr("t") + dbName := ast.NewCIStr("test") + tblName := ast.NewCIStr("t") tbl, err := is.TableByName(context.Background(), dbName, tblName) require.NoError(t, err) tblInfo := tbl.Meta() diff --git a/pkg/executor/test/analyzetest/BUILD.bazel b/pkg/executor/test/analyzetest/BUILD.bazel index 7fb0b111fa4e3..38edb3ef02f25 100644 --- a/pkg/executor/test/analyzetest/BUILD.bazel +++ b/pkg/executor/test/analyzetest/BUILD.bazel @@ -19,7 +19,7 @@ go_test( "//pkg/executor", "//pkg/infoschema", "//pkg/kv", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/parser/mysql", "//pkg/parser/terror", "//pkg/planner/core", diff --git a/pkg/executor/test/analyzetest/analyze_test.go b/pkg/executor/test/analyzetest/analyze_test.go index 14cc0793af312..b7449f3602f95 100644 --- a/pkg/executor/test/analyzetest/analyze_test.go +++ b/pkg/executor/test/analyzetest/analyze_test.go @@ -31,7 +31,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/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/planner/core" @@ -68,7 +68,7 @@ PARTITION BY RANGE ( a ) ( tk.MustExec("analyze table t") is := tk.Session().(sessionctx.Context).GetInfoSchema().(infoschema.InfoSchema) - table, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + table, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) pi := table.Meta().GetPartitionInfo() require.NotNil(t, pi) @@ -97,7 +97,7 @@ PARTITION BY RANGE ( a ) ( } tk.MustExec("alter table t analyze partition p0") is = tk.Session().(sessionctx.Context).GetInfoSchema().(infoschema.InfoSchema) - table, err = is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + table, err = is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) pi = table.Meta().GetPartitionInfo() require.NotNil(t, pi) @@ -154,7 +154,7 @@ func TestAnalyzeParameters(t *testing.T) { tk.MustExec("set @@tidb_analyze_version = 1") tk.MustExec("analyze table t with 30 samples") is := tk.Session().(sessionctx.Context).GetInfoSchema().(infoschema.InfoSchema) - table, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + table, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo := table.Meta() tbl := dom.StatsHandle().GetTableStats(tableInfo) @@ -210,7 +210,7 @@ func TestAnalyzeTooLongColumns(t *testing.T) { analyzehelper.TriggerPredicateColumnsCollection(t, tk, store, "t", "a") tk.MustExec("analyze table t") is := tk.Session().(sessionctx.Context).GetInfoSchema().(infoschema.InfoSchema) - table, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + table, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo := table.Meta() tbl := dom.StatsHandle().GetTableStats(tableInfo) @@ -252,7 +252,7 @@ func TestExtractTopN(t *testing.T) { } tk.MustExec("analyze table test_extract_topn") is := dom.InfoSchema() - table, err := is.TableByName(context.Background(), model.NewCIStr("test_extract_topn"), model.NewCIStr("test_extract_topn")) + table, err := is.TableByName(context.Background(), ast.NewCIStr("test_extract_topn"), ast.NewCIStr("test_extract_topn")) require.NoError(t, err) tblInfo := table.Meta() tblStats := dom.StatsHandle().GetTableStats(tblInfo) @@ -435,7 +435,7 @@ func testSnapshotAnalyzeAndMaxTSAnalyzeHelper(analyzeSnapshot bool) func(t *test tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int, index index_a(a))") is := tk.Session().(sessionctx.Context).GetInfoSchema().(infoschema.InfoSchema) - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tblInfo := tbl.Meta() tid := tblInfo.ID @@ -500,7 +500,7 @@ func TestAdjustSampleRateNote(t *testing.T) { err := statstestutil.HandleNextDDLEventWithTxn(statsHandle) require.NoError(t, err) is := tk.Session().(sessionctx.Context).GetInfoSchema().(infoschema.InfoSchema) - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tblInfo := tbl.Meta() tid := tblInfo.ID @@ -727,7 +727,7 @@ func TestSavedAnalyzeOptions(t *testing.T) { is := dom.InfoSchema() tk.MustQuery("select * from t where b > 1 and c > 1") require.NoError(t, h.LoadNeededHistograms(dom.InfoSchema())) - table, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + table, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo := table.Meta() tbl := h.GetTableStats(tableInfo) @@ -816,7 +816,7 @@ PARTITION BY RANGE ( a ) ( // analyze partition only sets options of partition tk.MustExec("analyze table t partition p0 with 1 topn, 3 buckets") is := dom.InfoSchema() - table, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + table, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo := table.Meta() pi := tableInfo.GetPartitionInfo() @@ -924,7 +924,7 @@ PARTITION BY RANGE ( a ) ( tk.MustExec("insert into t values (21,21,21),(22,22,22),(23,23,23),(24,24,24)") tk.MustExec("analyze table t partition p2") is = dom.InfoSchema() - table, err = is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + table, err = is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo = table.Meta() pi = tableInfo.GetPartitionInfo() @@ -1012,9 +1012,9 @@ func TestSavedAnalyzeOptionsForMultipleTables(t *testing.T) { tk.MustExec("analyze table t2 with 0 topn, 2 buckets") tk.MustExec("analyze table t1,t2 with 2 topn") is := dom.InfoSchema() - table1, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t1")) + table1, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t1")) require.NoError(t, err) - table2, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t2")) + table2, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t2")) require.NoError(t, err) tableInfo1 := table1.Meta() tableInfo2 := table2.Meta() @@ -1069,7 +1069,7 @@ func TestSavedAnalyzeColumnOptions(t *testing.T) { h.SetLease(oriLease) }() is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tblInfo := tbl.Meta() tk.MustExec("select * from t where b > 1") @@ -1136,8 +1136,8 @@ func TestSavedAnalyzeColumnOptions(t *testing.T) { } func TestAnalyzeColumnsWithPrimaryKey(t *testing.T) { - for _, val := range []model.ColumnChoice{model.ColumnList, model.PredicateColumns} { - func(choice model.ColumnChoice) { + for _, val := range []ast.ColumnChoice{ast.ColumnList, ast.PredicateColumns} { + func(choice ast.ColumnChoice) { store, dom := testkit.CreateMockStoreAndDomain(t) tk := testkit.NewTestKit(t, store) @@ -1151,18 +1151,18 @@ func TestAnalyzeColumnsWithPrimaryKey(t *testing.T) { require.NoError(t, h.DumpStatsDeltaToKV(true)) is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tblID := tbl.Meta().ID switch choice { - case model.ColumnList: + case ast.ColumnList: tk.MustExec("analyze table t columns a with 2 topn, 2 buckets") tk.MustQuery("show warnings").Sort().Check(testkit.Rows( "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t, reason to use this rate is \"use min(1, 110000/10000) as the sample-rate=1\"", "Warning 1105 Columns c are missing in ANALYZE but their stats are needed for calculating stats for indexes/primary key/extended stats", )) - case model.PredicateColumns: + case ast.PredicateColumns: originalVal := tk.MustQuery("select @@tidb_enable_column_tracking").Rows()[0][0].(string) defer func() { tk.MustExec(fmt.Sprintf("set global tidb_enable_column_tracking = %v", originalVal)) @@ -1204,8 +1204,8 @@ func TestAnalyzeColumnsWithPrimaryKey(t *testing.T) { } func TestAnalyzeColumnsWithIndex(t *testing.T) { - for _, val := range []model.ColumnChoice{model.ColumnList, model.PredicateColumns} { - func(choice model.ColumnChoice) { + for _, val := range []ast.ColumnChoice{ast.ColumnList, ast.PredicateColumns} { + func(choice ast.ColumnChoice) { store, dom := testkit.CreateMockStoreAndDomain(t) tk := testkit.NewTestKit(t, store) @@ -1219,18 +1219,18 @@ func TestAnalyzeColumnsWithIndex(t *testing.T) { require.NoError(t, h.DumpStatsDeltaToKV(true)) is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tblID := tbl.Meta().ID switch choice { - case model.ColumnList: + case ast.ColumnList: tk.MustExec("analyze table t columns c with 2 topn, 2 buckets") tk.MustQuery("show warnings").Sort().Check(testkit.Rows( "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t, reason to use this rate is \"use min(1, 110000/10000) as the sample-rate=1\"", "Warning 1105 Columns b,d are missing in ANALYZE but their stats are needed for calculating stats for indexes/primary key/extended stats", )) - case model.PredicateColumns: + case ast.PredicateColumns: originalVal := tk.MustQuery("select @@tidb_enable_column_tracking").Rows()[0][0].(string) defer func() { tk.MustExec(fmt.Sprintf("set global tidb_enable_column_tracking = %v", originalVal)) @@ -1281,8 +1281,8 @@ func TestAnalyzeColumnsWithIndex(t *testing.T) { } func TestAnalyzeColumnsWithClusteredIndex(t *testing.T) { - for _, val := range []model.ColumnChoice{model.ColumnList, model.PredicateColumns} { - func(choice model.ColumnChoice) { + for _, val := range []ast.ColumnChoice{ast.ColumnList, ast.PredicateColumns} { + func(choice ast.ColumnChoice) { store, dom := testkit.CreateMockStoreAndDomain(t) tk := testkit.NewTestKit(t, store) @@ -1296,18 +1296,18 @@ func TestAnalyzeColumnsWithClusteredIndex(t *testing.T) { require.NoError(t, h.DumpStatsDeltaToKV(true)) is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tblID := tbl.Meta().ID switch choice { - case model.ColumnList: + case ast.ColumnList: tk.MustExec("analyze table t columns c with 2 topn, 2 buckets") tk.MustQuery("show warnings").Sort().Check(testkit.Rows( "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t, reason to use this rate is \"use min(1, 110000/10000) as the sample-rate=1\"", "Warning 1105 Columns b,d are missing in ANALYZE but their stats are needed for calculating stats for indexes/primary key/extended stats", )) - case model.PredicateColumns: + case ast.PredicateColumns: originalVal := tk.MustQuery("select @@tidb_enable_column_tracking").Rows()[0][0].(string) defer func() { tk.MustExec(fmt.Sprintf("set global tidb_enable_column_tracking = %v", originalVal)) @@ -1358,8 +1358,8 @@ func TestAnalyzeColumnsWithClusteredIndex(t *testing.T) { } func TestAnalyzeColumnsWithDynamicPartitionTable(t *testing.T) { - for _, val := range []model.ColumnChoice{model.ColumnList, model.PredicateColumns} { - func(choice model.ColumnChoice) { + for _, val := range []ast.ColumnChoice{ast.ColumnList, ast.PredicateColumns} { + func(choice ast.ColumnChoice) { store, dom := testkit.CreateMockStoreAndDomain(t) tk := testkit.NewTestKit(t, store) @@ -1374,7 +1374,7 @@ func TestAnalyzeColumnsWithDynamicPartitionTable(t *testing.T) { require.NoError(t, h.DumpStatsDeltaToKV(true)) is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tblID := tbl.Meta().ID defs := tbl.Meta().Partition.Definitions @@ -1382,14 +1382,14 @@ func TestAnalyzeColumnsWithDynamicPartitionTable(t *testing.T) { p1ID := defs[1].ID switch choice { - case model.ColumnList: + case ast.ColumnList: tk.MustExec("analyze table t columns a with 2 topn, 2 buckets") tk.MustQuery("show warnings").Sort().Check(testkit.Rows( "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t's partition p0, reason to use this rate is \"use min(1, 110000/10000) as the sample-rate=1\"", "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t's partition p1, reason to use this rate is \"use min(1, 110000/10000) as the sample-rate=1\"", "Warning 1105 Columns c are missing in ANALYZE but their stats are needed for calculating stats for indexes/primary key/extended stats", )) - case model.PredicateColumns: + case ast.PredicateColumns: originalVal := tk.MustQuery("select @@tidb_enable_column_tracking").Rows()[0][0].(string) defer func() { tk.MustExec(fmt.Sprintf("set global tidb_enable_column_tracking = %v", originalVal)) @@ -1484,8 +1484,8 @@ func TestAnalyzeColumnsWithDynamicPartitionTable(t *testing.T) { } func TestAnalyzeColumnsWithStaticPartitionTable(t *testing.T) { - for _, val := range []model.ColumnChoice{model.ColumnList, model.PredicateColumns} { - func(choice model.ColumnChoice) { + for _, val := range []ast.ColumnChoice{ast.ColumnList, ast.PredicateColumns} { + func(choice ast.ColumnChoice) { store, dom := testkit.CreateMockStoreAndDomain(t) tk := testkit.NewTestKit(t, store) @@ -1500,7 +1500,7 @@ func TestAnalyzeColumnsWithStaticPartitionTable(t *testing.T) { require.NoError(t, h.DumpStatsDeltaToKV(true)) is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tblID := tbl.Meta().ID defs := tbl.Meta().Partition.Definitions @@ -1508,14 +1508,14 @@ func TestAnalyzeColumnsWithStaticPartitionTable(t *testing.T) { p1ID := defs[1].ID switch choice { - case model.ColumnList: + case ast.ColumnList: tk.MustExec("analyze table t columns a with 2 topn, 2 buckets") tk.MustQuery("show warnings").Sort().Check(testkit.Rows( "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t's partition p0, reason to use this rate is \"use min(1, 110000/10000) as the sample-rate=1\"", "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t's partition p1, reason to use this rate is \"use min(1, 110000/10000) as the sample-rate=1\"", "Warning 1105 Columns c are missing in ANALYZE but their stats are needed for calculating stats for indexes/primary key/extended stats", )) - case model.PredicateColumns: + case ast.PredicateColumns: originalVal := tk.MustQuery("select @@tidb_enable_column_tracking").Rows()[0][0].(string) defer func() { tk.MustExec(fmt.Sprintf("set global tidb_enable_column_tracking = %v", originalVal)) @@ -1595,8 +1595,8 @@ func TestAnalyzeColumnsWithStaticPartitionTable(t *testing.T) { } func TestAnalyzeColumnsWithExtendedStats(t *testing.T) { - for _, val := range []model.ColumnChoice{model.ColumnList, model.PredicateColumns} { - func(choice model.ColumnChoice) { + for _, val := range []ast.ColumnChoice{ast.ColumnList, ast.PredicateColumns} { + func(choice ast.ColumnChoice) { store, dom := testkit.CreateMockStoreAndDomain(t) tk := testkit.NewTestKit(t, store) @@ -1612,18 +1612,18 @@ func TestAnalyzeColumnsWithExtendedStats(t *testing.T) { require.NoError(t, h.DumpStatsDeltaToKV(true)) is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tblID := tbl.Meta().ID switch choice { - case model.ColumnList: + case ast.ColumnList: tk.MustExec("analyze table t columns b with 2 topn, 2 buckets") tk.MustQuery("show warnings").Sort().Check(testkit.Rows( "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t, reason to use this rate is \"use min(1, 110000/10000) as the sample-rate=1\"", "Warning 1105 Columns c are missing in ANALYZE but their stats are needed for calculating stats for indexes/primary key/extended stats", )) - case model.PredicateColumns: + case ast.PredicateColumns: originalVal := tk.MustQuery("select @@tidb_enable_column_tracking").Rows()[0][0].(string) defer func() { tk.MustExec(fmt.Sprintf("set global tidb_enable_column_tracking = %v", originalVal)) @@ -1667,8 +1667,8 @@ func TestAnalyzeColumnsWithExtendedStats(t *testing.T) { } func TestAnalyzeColumnsWithVirtualColumnIndex(t *testing.T) { - for _, val := range []model.ColumnChoice{model.ColumnList, model.PredicateColumns} { - func(choice model.ColumnChoice) { + for _, val := range []ast.ColumnChoice{ast.ColumnList, ast.PredicateColumns} { + func(choice ast.ColumnChoice) { store, dom := testkit.CreateMockStoreAndDomain(t) tk := testkit.NewTestKit(t, store) @@ -1682,18 +1682,18 @@ func TestAnalyzeColumnsWithVirtualColumnIndex(t *testing.T) { require.NoError(t, h.DumpStatsDeltaToKV(true)) is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tblID := tbl.Meta().ID switch choice { - case model.ColumnList: + case ast.ColumnList: tk.MustExec("analyze table t columns b with 2 topn, 2 buckets") tk.MustQuery("show warnings").Sort().Check(testkit.Rows( "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t, reason to use this rate is \"use min(1, 110000/10000) as the sample-rate=1\"", "Warning 1105 Columns c are missing in ANALYZE but their stats are needed for calculating stats for indexes/primary key/extended stats", )) - case model.PredicateColumns: + case ast.PredicateColumns: originalVal := tk.MustQuery("select @@tidb_enable_column_tracking").Rows()[0][0].(string) defer func() { tk.MustExec(fmt.Sprintf("set global tidb_enable_column_tracking = %v", originalVal)) @@ -1734,8 +1734,8 @@ func TestAnalyzeColumnsWithVirtualColumnIndex(t *testing.T) { } func TestAnalyzeColumnsAfterAnalyzeAll(t *testing.T) { - for _, val := range []model.ColumnChoice{model.ColumnList, model.PredicateColumns} { - func(choice model.ColumnChoice) { + for _, val := range []ast.ColumnChoice{ast.ColumnList, ast.PredicateColumns} { + func(choice ast.ColumnChoice) { store, dom := testkit.CreateMockStoreAndDomain(t) tk := testkit.NewTestKit(t, store) @@ -1748,7 +1748,7 @@ func TestAnalyzeColumnsAfterAnalyzeAll(t *testing.T) { require.NoError(t, h.DumpStatsDeltaToKV(true)) is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tblID := tbl.Meta().ID @@ -1773,9 +1773,9 @@ func TestAnalyzeColumnsAfterAnalyzeAll(t *testing.T) { require.NoError(t, h.DumpStatsDeltaToKV(true)) switch choice { - case model.ColumnList: + case ast.ColumnList: tk.MustExec("analyze table t columns b with 2 topn, 2 buckets") - case model.PredicateColumns: + case ast.PredicateColumns: tk.MustExec("select * from t where b > 1") require.NoError(t, h.DumpColStatsUsageToKV()) rows := tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't' and last_used_at is not null").Rows() @@ -1855,15 +1855,15 @@ func TestAnalyzeColumnsErrorAndWarning(t *testing.T) { "Warning 1105 No predicate column has been collected yet for table test.t, so only indexes and the columns composing the indexes will be analyzed", )) - for _, val := range []model.ColumnChoice{model.ColumnList, model.PredicateColumns} { - func(choice model.ColumnChoice) { + for _, val := range []ast.ColumnChoice{ast.ColumnList, ast.PredicateColumns} { + func(choice ast.ColumnChoice) { tk.MustExec("set @@tidb_analyze_version = 1") tk.MustExec("analyze table t") tk.MustExec("set @@tidb_analyze_version = 2") switch choice { - case model.ColumnList: + case ast.ColumnList: tk.MustExec("analyze table t columns b") - case model.PredicateColumns: + case ast.PredicateColumns: originalVal := tk.MustQuery("select @@tidb_enable_column_tracking").Rows()[0][0].(string) defer func() { tk.MustExec(fmt.Sprintf("set global tidb_enable_column_tracking = %v", originalVal)) @@ -1921,7 +1921,7 @@ func testKillAutoAnalyze(t *testing.T, ver int) { tk.MustExec("insert into t values (5,6), (7,8), (9, 10)") require.NoError(t, h.DumpStatsDeltaToKV(true)) require.NoError(t, h.Update(context.Background(), is)) - table, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + table, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo := table.Meta() lastVersion := h.GetTableStats(tableInfo).Version @@ -2003,7 +2003,7 @@ func TestKillAutoAnalyzeIndex(t *testing.T) { require.NoError(t, h.DumpStatsDeltaToKV(true)) tk.MustExec("analyze table t") tk.MustExec("alter table t add index idx(b)") - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tblInfo := tbl.Meta() lastVersion := h.GetTableStats(tblInfo).Version @@ -2224,7 +2224,7 @@ PARTITION BY RANGE ( a ) ( h.SetLease(oriLease) }() is := dom.InfoSchema() - table, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + table, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo := table.Meta() pi := tableInfo.GetPartitionInfo() @@ -2318,7 +2318,7 @@ PARTITION BY RANGE ( a ) ( h.SetLease(oriLease) }() is := dom.InfoSchema() - table, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + table, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo := table.Meta() pi := tableInfo.GetPartitionInfo() @@ -2453,7 +2453,7 @@ PARTITION BY RANGE ( a ) ( h.SetLease(oriLease) }() is := dom.InfoSchema() - table, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + table, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo := table.Meta() pi := tableInfo.GetPartitionInfo() @@ -2510,7 +2510,7 @@ PARTITION BY RANGE ( a ) ( h.SetLease(oriLease) }() is := dom.InfoSchema() - table, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + table, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo := table.Meta() pi := tableInfo.GetPartitionInfo() @@ -2592,7 +2592,7 @@ PARTITION BY RANGE ( a ) ( h.SetLease(oriLease) }() is := dom.InfoSchema() - table, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + table, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo := table.Meta() pi := tableInfo.GetPartitionInfo() @@ -2642,7 +2642,7 @@ PARTITION BY RANGE ( id ) ( h.HandleAutoAnalyze() tk.MustExec("create index idxa on t (a)") tk.MustExec("create index idxb on t (b)") - table, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + table, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo := table.Meta() pi := tableInfo.GetPartitionInfo() @@ -2677,7 +2677,7 @@ PARTITION BY RANGE ( id ) ( h.HandleAutoAnalyze() tk.MustExec("alter table t add column a int") tk.MustExec("alter table t add column b int") - table, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + table, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo := table.Meta() pi := tableInfo.GetPartitionInfo() @@ -2709,7 +2709,7 @@ PARTITION BY RANGE ( a ) ( h.SetLease(oriLease) }() is := dom.InfoSchema() - table, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + table, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo := table.Meta() pi := tableInfo.GetPartitionInfo() @@ -2736,7 +2736,7 @@ func TestAutoAnalyzeAwareGlobalVariableChange(t *testing.T) { h := dom.StatsHandle() err := statstestutil.HandleNextDDLEventWithTxn(h) require.NoError(t, err) - tbl, err := dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tid := tbl.Meta().ID tk.MustExec("insert into t values(1),(2),(3)") @@ -2808,7 +2808,7 @@ func TestAnalyzeColumnsSkipMVIndexJsonCol(t *testing.T) { )) is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tblInfo := tbl.Meta() stats := h.GetTableStats(tblInfo) diff --git a/pkg/executor/test/ddl/BUILD.bazel b/pkg/executor/test/ddl/BUILD.bazel index 1af675ad75759..aa5dc958422ae 100644 --- a/pkg/executor/test/ddl/BUILD.bazel +++ b/pkg/executor/test/ddl/BUILD.bazel @@ -19,7 +19,7 @@ go_test( "//pkg/kv", "//pkg/meta", "//pkg/meta/autoid", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/parser/mysql", "//pkg/sessionctx/variable", "//pkg/sessiontxn", diff --git a/pkg/executor/test/ddl/ddl_test.go b/pkg/executor/test/ddl/ddl_test.go index b1c6eea312c8f..c8fc4c812e6b3 100644 --- a/pkg/executor/test/ddl/ddl_test.go +++ b/pkg/executor/test/ddl/ddl_test.go @@ -32,7 +32,7 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta" "github.com/pingcap/tidb/pkg/meta/autoid" - "github.com/pingcap/tidb/pkg/parser/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/sessiontxn" @@ -118,7 +118,7 @@ func TestCreateTable(t *testing.T) { // test multiple collate specified in column when create. tk.MustExec("drop table if exists test_multiple_column_collate;") tk.MustExec("create table test_multiple_column_collate (a char(1) collate utf8_bin collate utf8_general_ci) charset utf8mb4 collate utf8mb4_bin") - tt, err := domain.GetDomain(tk.Session()).InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("test_multiple_column_collate")) + tt, err := domain.GetDomain(tk.Session()).InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("test_multiple_column_collate")) require.NoError(t, err) require.Equal(t, "utf8", tt.Cols()[0].GetCharset()) require.Equal(t, "utf8_general_ci", tt.Cols()[0].GetCollate()) @@ -127,7 +127,7 @@ func TestCreateTable(t *testing.T) { tk.MustExec("drop table if exists test_multiple_column_collate;") tk.MustExec("create table test_multiple_column_collate (a char(1) charset utf8 collate utf8_bin collate utf8_general_ci) charset utf8mb4 collate utf8mb4_bin") - tt, err = domain.GetDomain(tk.Session()).InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("test_multiple_column_collate")) + tt, err = domain.GetDomain(tk.Session()).InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("test_multiple_column_collate")) require.NoError(t, err) require.Equal(t, "utf8", tt.Cols()[0].GetCharset()) require.Equal(t, "utf8_general_ci", tt.Cols()[0].GetCollate()) @@ -312,7 +312,7 @@ func TestAddNotNullColumnNoDefault(t *testing.T) { tk.MustExec("insert nn values (1), (2)") tk.MustExec("alter table nn add column c2 int not null") - tbl, err := domain.GetDomain(tk.Session()).InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("nn")) + tbl, err := domain.GetDomain(tk.Session()).InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("nn")) require.NoError(t, err) col2 := tbl.Meta().Columns[1] require.Nil(t, col2.DefaultValue) @@ -357,7 +357,7 @@ func TestAlterTableModifyColumn(t *testing.T) { tk.MustExec("drop table if exists modify_column_multiple_collate") tk.MustExec("create table modify_column_multiple_collate (a char(1) collate utf8_bin collate utf8_general_ci) charset utf8mb4 collate utf8mb4_bin") tk.MustExec("alter table modify_column_multiple_collate modify column a char(1) collate utf8mb4_bin;") - tt, err := domain.GetDomain(tk.Session()).InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("modify_column_multiple_collate")) + tt, err := domain.GetDomain(tk.Session()).InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("modify_column_multiple_collate")) require.NoError(t, err) require.Equal(t, "utf8mb4", tt.Cols()[0].GetCharset()) require.Equal(t, "utf8mb4_bin", tt.Cols()[0].GetCollate()) @@ -367,7 +367,7 @@ func TestAlterTableModifyColumn(t *testing.T) { tk.MustExec("drop table if exists modify_column_multiple_collate;") tk.MustExec("create table modify_column_multiple_collate (a char(1) collate utf8_bin collate utf8_general_ci) charset utf8mb4 collate utf8mb4_bin") tk.MustExec("alter table modify_column_multiple_collate modify column a char(1) charset utf8mb4 collate utf8mb4_bin;") - tt, err = domain.GetDomain(tk.Session()).InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("modify_column_multiple_collate")) + tt, err = domain.GetDomain(tk.Session()).InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("modify_column_multiple_collate")) require.NoError(t, err) require.Equal(t, "utf8mb4", tt.Cols()[0].GetCharset()) require.Equal(t, "utf8mb4_bin", tt.Cols()[0].GetCollate()) @@ -457,7 +457,7 @@ func TestColumnCharsetAndCollate(t *testing.T) { is := dm.InfoSchema() require.NotNil(t, is) - tb, err := is.TableByName(context.Background(), model.NewCIStr(dbName), model.NewCIStr(tblName)) + tb, err := is.TableByName(context.Background(), ast.NewCIStr(dbName), ast.NewCIStr(tblName)) require.NoError(t, err) require.Equalf(t, tt.exptCharset, tb.Meta().Columns[0].GetCharset(), sql) require.Equalf(t, tt.exptCollate, tb.Meta().Columns[0].GetCollate(), sql) @@ -480,7 +480,7 @@ func TestShardRowIDBits(t *testing.T) { } dom := domain.GetDomain(tk.Session()) - tbl, err := dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) assertCountAndShard := func(tt table.Table, expectCount int) { @@ -531,9 +531,9 @@ func TestShardRowIDBits(t *testing.T) { tk.MustExec("alter table auto shard_row_id_bits = 0") // Hack an existing table with shard_row_id_bits and primary key as handle - db, ok := dom.InfoSchema().SchemaByName(model.NewCIStr("test")) + db, ok := dom.InfoSchema().SchemaByName(ast.NewCIStr("test")) require.True(t, ok) - tbl, err = dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("auto")) + tbl, err = dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("auto")) tblInfo := tbl.Meta() tblInfo.ShardRowIDBits = 5 tblInfo.MaxShardRowIDBits = 5 @@ -561,7 +561,7 @@ func TestShardRowIDBits(t *testing.T) { for i := 0; i < 100; i++ { tk.MustExec("insert into auto(a) values (?)", i) } - tbl, err = dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("auto")) + tbl, err = dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("auto")) assertCountAndShard(tbl, 100) prevB, err := strconv.Atoi(tk.MustQuery("select b from auto where a=0").Rows()[0][0].(string)) require.NoError(t, err) @@ -577,7 +577,7 @@ func TestShardRowIDBits(t *testing.T) { tk.MustExec("create table t1 (a int) shard_row_id_bits = 15") defer tk.MustExec("drop table if exists t1") - tbl, err = dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t1")) + tbl, err = dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t1")) require.NoError(t, err) maxID := 1<<(64-15-1) - 1 alloc := tbl.Allocators(tk.Session().GetTableCtx()).Get(autoid.RowIDAllocType) @@ -723,7 +723,7 @@ func TestAutoRandomTableOption(t *testing.T) { // test table option is auto-random tk.MustExec("drop table if exists auto_random_table_option") tk.MustExec("create table auto_random_table_option (a bigint auto_random(5) key) auto_random_base = 1000") - tt, err := domain.GetDomain(tk.Session()).InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("auto_random_table_option")) + tt, err := domain.GetDomain(tk.Session()).InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("auto_random_table_option")) require.NoError(t, err) require.Equal(t, int64(1000), tt.Meta().AutoRandID) tk.MustExec("insert into auto_random_table_option values (),(),(),(),()") @@ -739,7 +739,7 @@ func TestAutoRandomTableOption(t *testing.T) { tk.MustExec("drop table if exists alter_table_auto_random_option") tk.MustExec("create table alter_table_auto_random_option (a bigint primary key auto_random(4), b int)") - tt, err = domain.GetDomain(tk.Session()).InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("alter_table_auto_random_option")) + tt, err = domain.GetDomain(tk.Session()).InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("alter_table_auto_random_option")) require.NoError(t, err) require.Equal(t, int64(0), tt.Meta().AutoRandID) tk.MustExec("insert into alter_table_auto_random_option values(),(),(),(),()") @@ -757,7 +757,7 @@ func TestAutoRandomTableOption(t *testing.T) { // value is not what we rebased, because the local cache is dropped, here we choose // a quite big value to do this. tk.MustExec("alter table alter_table_auto_random_option auto_random_base = 3000000") - tt, err = domain.GetDomain(tk.Session()).InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("alter_table_auto_random_option")) + tt, err = domain.GetDomain(tk.Session()).InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("alter_table_auto_random_option")) require.NoError(t, err) require.Equal(t, int64(3000000), tt.Meta().AutoRandID) tk.MustExec("insert into alter_table_auto_random_option values(),(),(),(),()") diff --git a/pkg/executor/test/executor/BUILD.bazel b/pkg/executor/test/executor/BUILD.bazel index 09967dbe61a4b..b8ab704ebe3e7 100644 --- a/pkg/executor/test/executor/BUILD.bazel +++ b/pkg/executor/test/executor/BUILD.bazel @@ -23,7 +23,7 @@ go_test( "//pkg/meta/autoid", "//pkg/meta/model", "//pkg/parser", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/parser/mysql", "//pkg/parser/terror", "//pkg/planner", diff --git a/pkg/executor/test/executor/executor_test.go b/pkg/executor/test/executor/executor_test.go index d0afcc68c03e5..8d53c1a212da2 100644 --- a/pkg/executor/test/executor/executor_test.go +++ b/pkg/executor/test/executor/executor_test.go @@ -45,7 +45,7 @@ import ( "github.com/pingcap/tidb/pkg/meta/autoid" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/planner" @@ -301,11 +301,11 @@ func TestCheckIndex(t *testing.T) { require.NoError(t, err) is := dom.InfoSchema() - db := pmodel.NewCIStr("test_admin") + db := ast.NewCIStr("test_admin") dbInfo, ok := is.SchemaByName(db) require.True(t, ok) - tblName := pmodel.NewCIStr("t") + tblName := ast.NewCIStr("t") tbl, err := is.TableByName(context.Background(), db, tblName) require.NoError(t, err) tbInfo := tbl.Meta() @@ -421,7 +421,7 @@ func TestTimestampDefaultValueTimeZone(t *testing.T) { // Test the column's version is greater than ColumnInfoVersion1. is := domain.GetDomain(tk.Session()).InfoSchema() require.NotNil(t, is) - tb, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tb, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tb.Cols()[1].Version = model.ColumnInfoVersion1 + 1 tk.MustExec("insert into t set a=3") @@ -682,7 +682,7 @@ func TestIssue19148(t *testing.T) { tk.MustExec("create table t(a decimal(16, 2));") tk.MustExec("select * from t where a > any_value(a);") is := domain.GetDomain(tk.Session()).InfoSchema() - tblInfo, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tblInfo, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) require.Zero(t, tblInfo.Meta().Columns[0].GetFlag()) } @@ -2580,7 +2580,7 @@ func TestAdmin(t *testing.T) { dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() require.NotNil(t, is) - tb, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("admin_test")) + tb, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("admin_test")) require.NoError(t, err) require.Len(t, tb.Indices(), 1) _, err = tb.Indices()[0].Create(mock.NewContext().GetTableCtx(), txn, types.MakeDatums(int64(10)), kv.IntHandle(1), nil) diff --git a/pkg/executor/test/fktest/BUILD.bazel b/pkg/executor/test/fktest/BUILD.bazel index c538eb7ae46de..3a37a20c5777e 100644 --- a/pkg/executor/test/fktest/BUILD.bazel +++ b/pkg/executor/test/fktest/BUILD.bazel @@ -19,7 +19,6 @@ go_test( "//pkg/parser/ast", "//pkg/parser/auth", "//pkg/parser/format", - "//pkg/parser/model", "//pkg/parser/mysql", "//pkg/testkit", "//pkg/testkit/testfailpoint", diff --git a/pkg/executor/test/fktest/foreign_key_test.go b/pkg/executor/test/fktest/foreign_key_test.go index 130cb8392797e..76f07d13c104e 100644 --- a/pkg/executor/test/fktest/foreign_key_test.go +++ b/pkg/executor/test/fktest/foreign_key_test.go @@ -31,7 +31,6 @@ import ( "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/auth" "github.com/pingcap/tidb/pkg/parser/format" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/testkit/testfailpoint" @@ -1221,8 +1220,8 @@ func TestForeignKeyGenerateCascadeAST(t *testing.T) { {types.NewDatum(2), types.NewDatum("b")}, } cols := []*model.ColumnInfo{ - {ID: 1, Name: pmodel.NewCIStr("a"), FieldType: *types.NewFieldType(mysql.TypeLonglong)}, - {ID: 2, Name: pmodel.NewCIStr("name"), FieldType: *types.NewFieldType(mysql.TypeVarchar)}, + {ID: 1, Name: ast.NewCIStr("a"), FieldType: *types.NewFieldType(mysql.TypeLonglong)}, + {ID: 2, Name: ast.NewCIStr("name"), FieldType: *types.NewFieldType(mysql.TypeVarchar)}, } restoreFn := func(stmt ast.StmtNode) string { var sb strings.Builder @@ -1238,29 +1237,29 @@ func TestForeignKeyGenerateCascadeAST(t *testing.T) { require.Equal(t, restoreFn(expectedStmt), restoreFn(stmt)) } var stmt ast.StmtNode - stmt = executor.GenCascadeDeleteAST(pmodel.NewCIStr("test"), pmodel.NewCIStr("t2"), pmodel.NewCIStr(""), cols, fkValues) + stmt = executor.GenCascadeDeleteAST(ast.NewCIStr("test"), ast.NewCIStr("t2"), ast.NewCIStr(""), cols, fkValues) checkStmtFn(stmt, "delete from test.t2 where (a,name) in ((1,'a'), (2,'b'))") - stmt = executor.GenCascadeDeleteAST(pmodel.NewCIStr("test"), pmodel.NewCIStr("t2"), pmodel.NewCIStr("idx"), cols, fkValues) + stmt = executor.GenCascadeDeleteAST(ast.NewCIStr("test"), ast.NewCIStr("t2"), ast.NewCIStr("idx"), cols, fkValues) checkStmtFn(stmt, "delete from test.t2 use index(idx) where (a,name) in ((1,'a'), (2,'b'))") - stmt = executor.GenCascadeSetNullAST(pmodel.NewCIStr("test"), pmodel.NewCIStr("t2"), pmodel.NewCIStr(""), cols, fkValues) + stmt = executor.GenCascadeSetNullAST(ast.NewCIStr("test"), ast.NewCIStr("t2"), ast.NewCIStr(""), cols, fkValues) checkStmtFn(stmt, "update test.t2 set a = null, name = null where (a,name) in ((1,'a'), (2,'b'))") - stmt = executor.GenCascadeSetNullAST(pmodel.NewCIStr("test"), pmodel.NewCIStr("t2"), pmodel.NewCIStr("idx"), cols, fkValues) + stmt = executor.GenCascadeSetNullAST(ast.NewCIStr("test"), ast.NewCIStr("t2"), ast.NewCIStr("idx"), cols, fkValues) checkStmtFn(stmt, "update test.t2 use index(idx) set a = null, name = null where (a,name) in ((1,'a'), (2,'b'))") newValue1 := []types.Datum{types.NewDatum(10), types.NewDatum("aa")} couple := &executor.UpdatedValuesCouple{ NewValues: newValue1, OldValuesList: fkValues, } - stmt = executor.GenCascadeUpdateAST(pmodel.NewCIStr("test"), pmodel.NewCIStr("t2"), pmodel.NewCIStr(""), cols, couple) + stmt = executor.GenCascadeUpdateAST(ast.NewCIStr("test"), ast.NewCIStr("t2"), ast.NewCIStr(""), cols, couple) checkStmtFn(stmt, "update test.t2 set a = 10, name = 'aa' where (a,name) in ((1,'a'), (2,'b'))") - stmt = executor.GenCascadeUpdateAST(pmodel.NewCIStr("test"), pmodel.NewCIStr("t2"), pmodel.NewCIStr("idx"), cols, couple) + stmt = executor.GenCascadeUpdateAST(ast.NewCIStr("test"), ast.NewCIStr("t2"), ast.NewCIStr("idx"), cols, couple) checkStmtFn(stmt, "update test.t2 use index(idx) set a = 10, name = 'aa' where (a,name) in ((1,'a'), (2,'b'))") // Test for 1 fk column. fkValues = [][]types.Datum{{types.NewDatum(1)}, {types.NewDatum(2)}} - cols = []*model.ColumnInfo{{ID: 1, Name: pmodel.NewCIStr("a"), FieldType: *types.NewFieldType(mysql.TypeLonglong)}} - stmt = executor.GenCascadeDeleteAST(pmodel.NewCIStr("test"), pmodel.NewCIStr("t2"), pmodel.NewCIStr(""), cols, fkValues) + cols = []*model.ColumnInfo{{ID: 1, Name: ast.NewCIStr("a"), FieldType: *types.NewFieldType(mysql.TypeLonglong)}} + stmt = executor.GenCascadeDeleteAST(ast.NewCIStr("test"), ast.NewCIStr("t2"), ast.NewCIStr(""), cols, fkValues) checkStmtFn(stmt, "delete from test.t2 where a in (1,2)") - stmt = executor.GenCascadeDeleteAST(pmodel.NewCIStr("test"), pmodel.NewCIStr("t2"), pmodel.NewCIStr("idx"), cols, fkValues) + stmt = executor.GenCascadeDeleteAST(ast.NewCIStr("test"), ast.NewCIStr("t2"), ast.NewCIStr("idx"), cols, fkValues) checkStmtFn(stmt, "delete from test.t2 use index(idx) where a in (1,2)") } diff --git a/pkg/executor/test/seqtest/BUILD.bazel b/pkg/executor/test/seqtest/BUILD.bazel index 3f6e3bf271925..1473aaa6cda03 100644 --- a/pkg/executor/test/seqtest/BUILD.bazel +++ b/pkg/executor/test/seqtest/BUILD.bazel @@ -23,7 +23,6 @@ go_test( "//pkg/meta/autoid", "//pkg/metrics", "//pkg/parser/ast", - "//pkg/parser/model", "//pkg/parser/mysql", "//pkg/planner/core", "//pkg/server", diff --git a/pkg/executor/test/seqtest/seq_executor_test.go b/pkg/executor/test/seqtest/seq_executor_test.go index 1fc1e69803338..edec2ef512c27 100644 --- a/pkg/executor/test/seqtest/seq_executor_test.go +++ b/pkg/executor/test/seqtest/seq_executor_test.go @@ -40,7 +40,7 @@ import ( "github.com/pingcap/tidb/pkg/executor/join" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta/autoid" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" plannercore "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/session" @@ -79,7 +79,7 @@ func TestEarlyClose(t *testing.T) { // Get table ID for split. is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("earlyclose")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("earlyclose")) require.NoError(t, err) tblID := tbl.Meta().ID diff --git a/pkg/executor/test/showtest/BUILD.bazel b/pkg/executor/test/showtest/BUILD.bazel index 70c465b22f48a..1a9311a054a79 100644 --- a/pkg/executor/test/showtest/BUILD.bazel +++ b/pkg/executor/test/showtest/BUILD.bazel @@ -16,8 +16,8 @@ go_test( "//pkg/infoschema", "//pkg/meta/autoid", "//pkg/meta/model", + "//pkg/parser/ast", "//pkg/parser/auth", - "//pkg/parser/model", "//pkg/parser/mysql", "//pkg/parser/types", "//pkg/privilege/privileges", diff --git a/pkg/executor/test/showtest/show_test.go b/pkg/executor/test/showtest/show_test.go index 55f2384522130..0e91999db9c07 100644 --- a/pkg/executor/test/showtest/show_test.go +++ b/pkg/executor/test/showtest/show_test.go @@ -26,8 +26,8 @@ import ( "github.com/pingcap/tidb/pkg/executor" "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/meta/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/auth" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" parsertypes "github.com/pingcap/tidb/pkg/parser/types" "github.com/pingcap/tidb/pkg/privilege/privileges" @@ -286,7 +286,7 @@ func TestShowWarningsForExprPushdown(t *testing.T) { // create tiflash replica { is := dom.InfoSchema() - tblInfo, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("show_warnings_expr_pushdown")) + tblInfo, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("show_warnings_expr_pushdown")) require.NoError(t, err) tblInfo.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ Count: 1, @@ -529,7 +529,7 @@ func TestShow2(t *testing.T) { tk.MustQuery("SHOW FULL TABLES in metrics_schema like 'uptime'").Check(testkit.Rows("uptime SYSTEM VIEW")) is := dom.InfoSchema() - tblInfo, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tblInfo, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) createTime := model.TSConvert2Time(tblInfo.Meta().UpdateTS).Format(time.DateTime) @@ -672,7 +672,7 @@ func TestUnprivilegedShow(t *testing.T) { tk.Session().Auth(&auth.UserIdentity{Username: "lowprivuser", Hostname: "192.168.0.1", AuthUsername: "lowprivuser", AuthHostname: "%"}, nil, []byte("012345678901234567890"), nil) is := dom.InfoSchema() - tblInfo, err := is.TableByName(context.Background(), pmodel.NewCIStr("testshow"), pmodel.NewCIStr("t1")) + tblInfo, err := is.TableByName(context.Background(), ast.NewCIStr("testshow"), ast.NewCIStr("t1")) require.NoError(t, err) createTime := model.TSConvert2Time(tblInfo.Meta().UpdateTS).Format(time.DateTime) diff --git a/pkg/executor/test/simpletest/BUILD.bazel b/pkg/executor/test/simpletest/BUILD.bazel index 63aecc3b4a8ee..c0b4868935ebf 100644 --- a/pkg/executor/test/simpletest/BUILD.bazel +++ b/pkg/executor/test/simpletest/BUILD.bazel @@ -12,8 +12,8 @@ go_test( shard_count = 11, deps = [ "//pkg/config", + "//pkg/parser/ast", "//pkg/parser/auth", - "//pkg/parser/model", "//pkg/parser/mysql", "//pkg/parser/terror", "//pkg/server", diff --git a/pkg/executor/test/simpletest/simple_test.go b/pkg/executor/test/simpletest/simple_test.go index 0766a40381af2..c9e45f5013cd7 100644 --- a/pkg/executor/test/simpletest/simple_test.go +++ b/pkg/executor/test/simpletest/simple_test.go @@ -22,8 +22,8 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/config" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/auth" - "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/server" @@ -571,7 +571,7 @@ func TestDropStats(t *testing.T) { testKit.MustExec("use test") testKit.MustExec("create table t (c1 int, c2 int, index idx(c1, c2))") is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo := tbl.Meta() h := dom.StatsHandle() @@ -605,11 +605,11 @@ func TestDropStatsForMultipleTable(t *testing.T) { testKit.MustExec("create table t2 (c1 int, c2 int, index idx(c1, c2))") is := dom.InfoSchema() - tbl1, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t1")) + tbl1, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t1")) require.NoError(t, err) tableInfo1 := tbl1.Meta() - tbl2, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t2")) + tbl2, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t2")) require.NoError(t, err) tableInfo2 := tbl2.Meta() diff --git a/pkg/executor/test/writetest/BUILD.bazel b/pkg/executor/test/writetest/BUILD.bazel index 9274fe7730b26..84ca014e5f6bb 100644 --- a/pkg/executor/test/writetest/BUILD.bazel +++ b/pkg/executor/test/writetest/BUILD.bazel @@ -16,7 +16,7 @@ go_test( "//pkg/kv", "//pkg/lightning/mydump", "//pkg/meta/autoid", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/parser/mysql", "//pkg/session", "//pkg/sessionctx", diff --git a/pkg/executor/test/writetest/write_test.go b/pkg/executor/test/writetest/write_test.go index b1c420e3352bc..679833fcdcd1c 100644 --- a/pkg/executor/test/writetest/write_test.go +++ b/pkg/executor/test/writetest/write_test.go @@ -25,7 +25,7 @@ import ( "github.com/pingcap/tidb/pkg/executor" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/lightning/mydump" - "github.com/pingcap/tidb/pkg/parser/model" + "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" @@ -336,8 +336,8 @@ func TestReplaceLog(t *testing.T) { // Make some dangling index. ctx := testkit.NewSession(t, store) is := domain.InfoSchema() - dbName := model.NewCIStr("test") - tblName := model.NewCIStr("testLog") + dbName := ast.NewCIStr("test") + tblName := ast.NewCIStr("testLog") tbl, err := is.TableByName(context.Background(), dbName, tblName) require.NoError(t, err) tblInfo := tbl.Meta() @@ -368,7 +368,7 @@ func TestRebaseIfNeeded(t *testing.T) { tk.MustExec(`insert into t (b) values (1);`) ctx := testkit.NewSession(t, store) - tbl, err := domain.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := domain.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) require.Nil(t, sessiontxn.NewTxn(context.Background(), ctx)) txn, err := ctx.Txn(true) diff --git a/pkg/executor/update.go b/pkg/executor/update.go index 77487581f9b8c..8e237e8a21286 100644 --- a/pkg/executor/update.go +++ b/pkg/executor/update.go @@ -26,7 +26,7 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta/autoid" mmodel "github.com/pingcap/tidb/pkg/meta/model" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" plannercore "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/sessionctx" @@ -458,7 +458,7 @@ func (e *UpdateExec) updateRows(ctx context.Context) (int, error) { return totalNumRows, nil } -func handleUpdateError(sctx sessionctx.Context, colName model.CIStr, colInfo *mmodel.ColumnInfo, rowIdx int, err error) error { +func handleUpdateError(sctx sessionctx.Context, colName ast.CIStr, colInfo *mmodel.ColumnInfo, rowIdx int, err error) error { if err == nil { return nil } diff --git a/pkg/expression/BUILD.bazel b/pkg/expression/BUILD.bazel index d4e770d3c3805..cffed01908018 100644 --- a/pkg/expression/BUILD.bazel +++ b/pkg/expression/BUILD.bazel @@ -90,7 +90,6 @@ go_library( "//pkg/parser/ast", "//pkg/parser/auth", "//pkg/parser/charset", - "//pkg/parser/model", "//pkg/parser/mysql", "//pkg/parser/opcode", "//pkg/parser/terror", @@ -222,7 +221,6 @@ go_test( "//pkg/parser/ast", "//pkg/parser/auth", "//pkg/parser/charset", - "//pkg/parser/model", "//pkg/parser/mysql", "//pkg/parser/terror", "//pkg/planner/cascades/base", diff --git a/pkg/expression/builtin_cast.go b/pkg/expression/builtin_cast.go index c1a9aec8eae50..fe7fbadcc1be1 100644 --- a/pkg/expression/builtin_cast.go +++ b/pkg/expression/builtin_cast.go @@ -32,7 +32,6 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/charset" - "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" @@ -2518,7 +2517,7 @@ func BuildCastFunctionWithCheck(ctx BuildContext, expr Expression, tp *types.Fie } f, err := fc.getFunction(ctx, []Expression{expr}) res = &ScalarFunction{ - FuncName: model.NewCIStr(ast.Cast), + FuncName: ast.NewCIStr(ast.Cast), RetType: tp, Function: f, } diff --git a/pkg/expression/builtin_convert_charset.go b/pkg/expression/builtin_convert_charset.go index 33524d80dcdb7..2b92f4826a45f 100644 --- a/pkg/expression/builtin_convert_charset.go +++ b/pkg/expression/builtin_convert_charset.go @@ -23,7 +23,6 @@ import ( "github.com/pingcap/tidb/pkg/errno" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/charset" - "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/chunk" @@ -259,7 +258,7 @@ func BuildToBinaryFunction(ctx BuildContext, expr Expression) (res Expression) { return expr } res = &ScalarFunction{ - FuncName: model.NewCIStr(InternalFuncToBinary), + FuncName: ast.NewCIStr(InternalFuncToBinary), RetType: f.getRetTp(), Function: f, } @@ -274,7 +273,7 @@ func BuildFromBinaryFunction(ctx BuildContext, expr Expression, tp *types.FieldT return expr } res = &ScalarFunction{ - FuncName: model.NewCIStr(InternalFuncFromBinary), + FuncName: ast.NewCIStr(InternalFuncFromBinary), RetType: tp, Function: f, } diff --git a/pkg/expression/builtin_other.go b/pkg/expression/builtin_other.go index bdc6afd3c9298..d910879d72027 100644 --- a/pkg/expression/builtin_other.go +++ b/pkg/expression/builtin_other.go @@ -22,7 +22,6 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/expression/expropt" "github.com/pingcap/tidb/pkg/parser/ast" - "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/chunk" @@ -1070,7 +1069,7 @@ func BuildGetVarFunction(ctx BuildContext, expr Expression, retType *types.Field retType = builtinRetTp } sf := &ScalarFunction{ - FuncName: model.NewCIStr(ast.GetVar), + FuncName: ast.NewCIStr(ast.GetVar), RetType: retType, Function: f, } diff --git a/pkg/expression/builtin_test.go b/pkg/expression/builtin_test.go index 5ea1f47c17fa9..9efd20e2088e5 100644 --- a/pkg/expression/builtin_test.go +++ b/pkg/expression/builtin_test.go @@ -24,7 +24,6 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/charset" - "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util" @@ -272,7 +271,7 @@ func newFunctionForTest(ctx BuildContext, funcName string, args ...Expression) ( return nil, err } return &ScalarFunction{ - FuncName: model.NewCIStr(funcName), + FuncName: ast.NewCIStr(funcName), RetType: f.getRetTp(), Function: f, }, nil diff --git a/pkg/expression/column_test.go b/pkg/expression/column_test.go index a1babed68ab9a..3d0119147221f 100644 --- a/pkg/expression/column_test.go +++ b/pkg/expression/column_test.go @@ -20,7 +20,6 @@ import ( "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/ast" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/cascades/base" "github.com/pingcap/tidb/pkg/types" @@ -135,9 +134,9 @@ func TestColInfo2Col(t *testing.T) { func TestIndexInfo2Cols(t *testing.T) { col0 := &Column{UniqueID: 0, ID: 0, RetType: types.NewFieldType(mysql.TypeLonglong)} col1 := &Column{UniqueID: 1, ID: 1, RetType: types.NewFieldType(mysql.TypeLonglong)} - colInfo0 := &model.ColumnInfo{ID: 0, Name: pmodel.NewCIStr("0")} - colInfo1 := &model.ColumnInfo{ID: 1, Name: pmodel.NewCIStr("1")} - indexCol0, indexCol1 := &model.IndexColumn{Name: pmodel.NewCIStr("0")}, &model.IndexColumn{Name: pmodel.NewCIStr("1")} + colInfo0 := &model.ColumnInfo{ID: 0, Name: ast.NewCIStr("0")} + colInfo1 := &model.ColumnInfo{ID: 1, Name: ast.NewCIStr("1")} + indexCol0, indexCol1 := &model.IndexColumn{Name: ast.NewCIStr("0")}, &model.IndexColumn{Name: ast.NewCIStr("1")} indexInfo := &model.IndexInfo{Columns: []*model.IndexColumn{indexCol0, indexCol1}} cols := []*Column{col0} diff --git a/pkg/expression/distsql_builtin.go b/pkg/expression/distsql_builtin.go index ae8d468a8366a..91c98723ee3d0 100644 --- a/pkg/expression/distsql_builtin.go +++ b/pkg/expression/distsql_builtin.go @@ -20,7 +20,6 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/parser/ast" - "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/codec" @@ -1163,7 +1162,7 @@ func newDistSQLFunctionBySig(ctx BuildContext, sigCode tipb.ScalarFuncSig, tp *t return nil, err } return &ScalarFunction{ - FuncName: model.NewCIStr(fmt.Sprintf("sig_%T", f)), + FuncName: ast.NewCIStr(fmt.Sprintf("sig_%T", f)), Function: f, RetType: f.getRetTp(), }, nil diff --git a/pkg/expression/evaluator_test.go b/pkg/expression/evaluator_test.go index a9988b35b2003..809cc186267db 100644 --- a/pkg/expression/evaluator_test.go +++ b/pkg/expression/evaluator_test.go @@ -22,7 +22,6 @@ import ( "github.com/pingcap/tidb/pkg/expression/exprctx" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/charset" - "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/chunk" @@ -619,7 +618,7 @@ func TestOptionalProp(t *testing.T) { arg1f, err := arg1fc.getFunction(ctx, nil) require.NoError(t, err) arg1 := &ScalarFunction{ - FuncName: model.NewCIStr(ast.CurrentUser), + FuncName: ast.NewCIStr(ast.CurrentUser), Function: arg1f, RetType: arg1f.getRetTp(), } @@ -627,7 +626,7 @@ func TestOptionalProp(t *testing.T) { arg2f, err := arg2fc.getFunction(ctx, nil) require.NoError(t, err) arg2 := &ScalarFunction{ - FuncName: model.NewCIStr(ast.TiDBIsDDLOwner), + FuncName: ast.NewCIStr(ast.TiDBIsDDLOwner), Function: arg2f, RetType: arg2f.getRetTp(), } @@ -635,7 +634,7 @@ func TestOptionalProp(t *testing.T) { f, err := fc.getFunction(ctx, []Expression{arg1, arg2}) require.NoError(t, err) fe := &ScalarFunction{ - FuncName: model.NewCIStr(ast.Plus), + FuncName: ast.NewCIStr(ast.Plus), Function: f, RetType: f.getRetTp(), } @@ -644,7 +643,7 @@ func TestOptionalProp(t *testing.T) { f2, err := fc2.getFunction(ctx, datumsToConstants(types.MakeDatums("tidb_distsql_scan_concurrency", 10))) require.NoError(t, err) fe2 := &ScalarFunction{ - FuncName: model.NewCIStr(ast.GetLock), + FuncName: ast.NewCIStr(ast.GetLock), Function: f2, RetType: f2.getRetTp(), } diff --git a/pkg/expression/expression.go b/pkg/expression/expression.go index 0eb654a0c7179..5e92ad4dbfa56 100644 --- a/pkg/expression/expression.go +++ b/pkg/expression/expression.go @@ -23,7 +23,6 @@ import ( "github.com/pingcap/tidb/pkg/expression/exprctx" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/ast" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/opcode" "github.com/pingcap/tidb/pkg/parser/terror" @@ -57,7 +56,7 @@ type BuildOptions struct { // InputNames is the input names for expression to build InputNames types.NameSlice // SourceTableDB is the database that the source table located - SourceTableDB pmodel.CIStr + SourceTableDB ast.CIStr // SourceTable is used to provide some extra column info. SourceTable *model.TableInfo // AllowCastArray specifies whether to allow casting to an array type. @@ -73,7 +72,7 @@ type BuildOption func(*BuildOptions) // When this option is specified, it will use the table meta to resolve column names. func WithTableInfo(db string, tblInfo *model.TableInfo) BuildOption { return func(options *BuildOptions) { - options.SourceTableDB = pmodel.NewCIStr(db) + options.SourceTableDB = ast.NewCIStr(db) options.SourceTable = tblInfo } } @@ -850,7 +849,7 @@ func FlattenCNFConditions(CNFCondition *ScalarFunction) []Expression { type Assignment struct { Col *Column // ColName indicates its original column name in table schema. It's used for outputting helping message when executing meets some errors. - ColName pmodel.CIStr + ColName ast.CIStr Expr Expression // LazyErr is used in statement like `INSERT INTO t1 (a) VALUES (1) ON DUPLICATE KEY UPDATE a= (SELECT b FROM source);`, ErrSubqueryMoreThan1Row // should be evaluated after the duplicate situation is detected in the executing procedure. @@ -1029,7 +1028,7 @@ func evaluateExprWithNullInNullRejectCheck(ctx BuildContext, schema *Schema, exp } // TableInfo2SchemaAndNames converts the TableInfo to the schema and name slice. -func TableInfo2SchemaAndNames(ctx BuildContext, dbName pmodel.CIStr, tbl *model.TableInfo) (*Schema, []*types.FieldName, error) { +func TableInfo2SchemaAndNames(ctx BuildContext, dbName ast.CIStr, tbl *model.TableInfo) (*Schema, []*types.FieldName, error) { cols, names, err := ColumnInfos2ColumnsAndNames(ctx, dbName, tbl.Name, tbl.Cols(), tbl) if err != nil { return nil, nil, err @@ -1076,7 +1075,7 @@ func TableInfo2SchemaAndNames(ctx BuildContext, dbName pmodel.CIStr, tbl *model. } // ColumnInfos2ColumnsAndNames converts the ColumnInfo to the *Column and NameSlice. -func ColumnInfos2ColumnsAndNames(ctx BuildContext, dbName, tblName pmodel.CIStr, colInfos []*model.ColumnInfo, tblInfo *model.TableInfo) ([]*Column, types.NameSlice, error) { +func ColumnInfos2ColumnsAndNames(ctx BuildContext, dbName, tblName ast.CIStr, colInfos []*model.ColumnInfo, tblInfo *model.TableInfo) ([]*Column, types.NameSlice, error) { columns := make([]*Column, 0, len(colInfos)) names := make([]*types.FieldName, 0, len(colInfos)) for i, col := range colInfos { @@ -1139,7 +1138,7 @@ func NewValuesFunc(ctx BuildContext, offset int, retTp *types.FieldType) *Scalar bt, err := fc.getFunction(ctx, nil) terror.Log(err) return &ScalarFunction{ - FuncName: pmodel.NewCIStr(ast.Values), + FuncName: ast.NewCIStr(ast.Values), RetType: retTp, Function: bt, } @@ -1180,12 +1179,12 @@ func wrapWithIsTrue(ctx BuildContext, keepNull bool, arg Expression, wrapForInt return nil, err } sf := &ScalarFunction{ - FuncName: pmodel.NewCIStr(ast.IsTruthWithoutNull), + FuncName: ast.NewCIStr(ast.IsTruthWithoutNull), Function: f, RetType: f.getRetTp(), } if keepNull { - sf.FuncName = pmodel.NewCIStr(ast.IsTruthWithNull) + sf.FuncName = ast.NewCIStr(ast.IsTruthWithNull) } return FoldConstant(ctx, sf), nil } diff --git a/pkg/expression/expression_test.go b/pkg/expression/expression_test.go index 80125595240cf..5555772c61ffe 100644 --- a/pkg/expression/expression_test.go +++ b/pkg/expression/expression_test.go @@ -21,7 +21,6 @@ import ( "github.com/pingcap/tidb/pkg/expression/exprstatic" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/ast" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/chunk" @@ -247,7 +246,7 @@ func (builder *testTableBuilder) add(name string, tp byte, flag uint) *testTable func (builder *testTableBuilder) build() *model.TableInfo { ti := &model.TableInfo{ ID: 1, - Name: pmodel.NewCIStr(builder.tableName), + Name: ast.NewCIStr(builder.tableName), State: model.StatePublic, } for i, colName := range builder.columnNames { @@ -262,7 +261,7 @@ func (builder *testTableBuilder) build() *model.TableInfo { fieldType.SetFlag(builder.flags[i]) ti.Columns = append(ti.Columns, &model.ColumnInfo{ ID: int64(i + 1), - Name: pmodel.NewCIStr(colName), + Name: ast.NewCIStr(colName), Offset: i, FieldType: *fieldType, State: model.StatePublic, diff --git a/pkg/expression/helper_test.go b/pkg/expression/helper_test.go index 3cdcb40aa0f4d..e55d2d36885b3 100644 --- a/pkg/expression/helper_test.go +++ b/pkg/expression/helper_test.go @@ -22,7 +22,6 @@ import ( "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/charset" - "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/types" driver "github.com/pingcap/tidb/pkg/types/parser_driver" @@ -92,7 +91,7 @@ func TestGetTimeValue(t *testing.T) { {ast.NewValueExpr("2012-12-12 00:00:00", charset.CharsetUTF8MB4, charset.CollationUTF8MB4), "2012-12-12 00:00:00"}, {ast.NewValueExpr(int64(0), "", ""), "0000-00-00 00:00:00"}, {ast.NewValueExpr(nil, "", ""), nil}, - {&ast.FuncCallExpr{FnName: model.NewCIStr(ast.CurrentTimestamp)}, strings.ToUpper(ast.CurrentTimestamp)}, + {&ast.FuncCallExpr{FnName: ast.NewCIStr(ast.CurrentTimestamp)}, strings.ToUpper(ast.CurrentTimestamp)}, // {&ast.UnaryOperationExpr{Op: opcode.Minus, V: ast.NewValueExpr(int64(0))}, "0000-00-00 00:00:00"}, } @@ -115,7 +114,7 @@ func TestGetTimeValue(t *testing.T) { {"2012-13-12 00:00:00"}, {ast.NewValueExpr("2012-13-12 00:00:00", charset.CharsetUTF8MB4, charset.CollationUTF8MB4)}, {ast.NewValueExpr(int64(1), "", "")}, - {&ast.FuncCallExpr{FnName: model.NewCIStr("xxx")}}, + {&ast.FuncCallExpr{FnName: ast.NewCIStr("xxx")}}, // {&ast.UnaryOperationExpr{Op: opcode.Minus, V: ast.NewValueExpr(int64(1))}}, } @@ -131,7 +130,7 @@ func TestIsCurrentTimestampExpr(t *testing.T) { if i != 0 { args = []ast.ExprNode{&driver.ValueExpr{Datum: types.NewIntDatum(i)}} } - return &ast.FuncCallExpr{FnName: model.NewCIStr("CURRENT_TIMESTAMP"), Args: args} + return &ast.FuncCallExpr{FnName: ast.NewCIStr("CURRENT_TIMESTAMP"), Args: args} } v := IsValidCurrentTimestampExpr(ast.NewValueExpr("abc", charset.CharsetUTF8MB4, charset.CollationUTF8MB4), nil) diff --git a/pkg/expression/integration_test/BUILD.bazel b/pkg/expression/integration_test/BUILD.bazel index fc10d03844631..139264b797cca 100644 --- a/pkg/expression/integration_test/BUILD.bazel +++ b/pkg/expression/integration_test/BUILD.bazel @@ -16,8 +16,8 @@ go_test( "//pkg/expression", "//pkg/kv", "//pkg/meta/model", + "//pkg/parser/ast", "//pkg/parser/auth", - "//pkg/parser/model", "//pkg/parser/mysql", "//pkg/parser/terror", "//pkg/planner/core", diff --git a/pkg/expression/integration_test/integration_test.go b/pkg/expression/integration_test/integration_test.go index 485d10677f9d2..941f6726e5054 100644 --- a/pkg/expression/integration_test/integration_test.go +++ b/pkg/expression/integration_test/integration_test.go @@ -37,8 +37,8 @@ import ( "github.com/pingcap/tidb/pkg/expression" "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/parser/auth" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" plannercore "github.com/pingcap/tidb/pkg/planner/core" @@ -1451,7 +1451,7 @@ func TestTiDBDecodeKeyFunc(t *testing.T) { tk.MustExec("create table t (a varchar(255), b int, c datetime, primary key (a, b, c));") dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) getTime := func(year, month, day int, timeType byte) types.Time { ret := types.NewTime(types.FromDate(year, month, day, 0, 0, 0, 0), timeType, types.DefaultFsp) @@ -1482,7 +1482,7 @@ func TestTiDBDecodeKeyFunc(t *testing.T) { tk.MustExec("create table t (a varchar(255), b int, c datetime, index idx(a, b, c));") dom = domain.GetDomain(tk.Session()) is = dom.InfoSchema() - tbl, err = is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err = is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) buildIndexKeyFromData := func(tableID, indexID int64, data []types.Datum) string { k, err := codec.EncodeKey(tk.Session().GetSessionVars().StmtCtx.TimeZone(), nil, data...) @@ -1521,7 +1521,7 @@ func TestTiDBDecodeKeyFunc(t *testing.T) { tk.MustExec("create table t (a int primary key nonclustered, b int, key bk (b));") dom = domain.GetDomain(tk.Session()) is = dom.InfoSchema() - tbl, err = is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err = is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) buildTableRowKey := func(tableID, rowID int64) string { return hex.EncodeToString( @@ -1540,7 +1540,7 @@ func TestTiDBDecodeKeyFunc(t *testing.T) { tk.MustExec("create table t (a int primary key clustered, b int, key bk (b));") dom = domain.GetDomain(tk.Session()) is = dom.InfoSchema() - tbl, err = is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err = is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) hexKey = buildTableRowKey(tbl.Meta().ID, rowID) sql = fmt.Sprintf("select tidb_decode_key( '%s' )", hexKey) @@ -1552,7 +1552,7 @@ func TestTiDBDecodeKeyFunc(t *testing.T) { tk.MustExec("create table t (a int primary key clustered, b int, key bk (b)) PARTITION BY RANGE (a) (PARTITION p0 VALUES LESS THAN (1), PARTITION p1 VALUES LESS THAN (2));") dom = domain.GetDomain(tk.Session()) is = dom.InfoSchema() - tbl, err = is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err = is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) require.NotNil(t, tbl.Meta().Partition) hexKey = buildTableRowKey(tbl.Meta().Partition.Definitions[0].ID, rowID) @@ -1653,7 +1653,7 @@ func TestShardIndexOnTiFlash(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ Count: 1, @@ -1689,7 +1689,7 @@ func TestExprPushdownBlacklist(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ Count: 1, diff --git a/pkg/expression/scalar_function.go b/pkg/expression/scalar_function.go index a8ba91018ca45..bda520d28aa81 100644 --- a/pkg/expression/scalar_function.go +++ b/pkg/expression/scalar_function.go @@ -22,7 +22,6 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/expression/exprctx" "github.com/pingcap/tidb/pkg/parser/ast" - "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/planner/cascades/base" @@ -39,7 +38,7 @@ var _ base.HashEquals = &ScalarFunction{} // ScalarFunction is the function that returns a value. type ScalarFunction struct { - FuncName model.CIStr + FuncName ast.CIStr // RetType is the type that ScalarFunction returns. // TODO: Implement type inference here, now we use ast's return type temporarily. RetType *types.FieldType `plan-cache-clone:"shallow"` @@ -270,7 +269,7 @@ func newFunctionImpl(ctx BuildContext, fold int, funcName string, retType *types retType = builtinRetTp } sf := &ScalarFunction{ - FuncName: model.NewCIStr(funcName), + FuncName: ast.NewCIStr(funcName), RetType: retType, Function: f, } diff --git a/pkg/expression/sessionexpr/BUILD.bazel b/pkg/expression/sessionexpr/BUILD.bazel index dab3390872cb0..99f5fb0e0a5dd 100644 --- a/pkg/expression/sessionexpr/BUILD.bazel +++ b/pkg/expression/sessionexpr/BUILD.bazel @@ -11,8 +11,8 @@ go_library( "//pkg/expression/expropt", "//pkg/expression/exprstatic", "//pkg/infoschema/context", + "//pkg/parser/ast", "//pkg/parser/auth", - "//pkg/parser/model", "//pkg/parser/mysql", "//pkg/privilege", "//pkg/sessionctx", diff --git a/pkg/expression/sessionexpr/sessionctx.go b/pkg/expression/sessionexpr/sessionctx.go index 0ad74841543b8..74162eb60ba08 100644 --- a/pkg/expression/sessionexpr/sessionctx.go +++ b/pkg/expression/sessionexpr/sessionctx.go @@ -24,8 +24,8 @@ import ( "github.com/pingcap/tidb/pkg/expression/expropt" "github.com/pingcap/tidb/pkg/expression/exprstatic" infoschema "github.com/pingcap/tidb/pkg/infoschema/context" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/auth" - "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/privilege" "github.com/pingcap/tidb/pkg/sessionctx" @@ -394,7 +394,7 @@ func (s *sequenceOperator) SetSequenceVal(newVal int64) (int64, bool, error) { func sequenceOperatorProp(sctx sessionctx.Context) expropt.SequenceOperatorProvider { return func(db, name string) (expropt.SequenceOperator, error) { - sequence, err := util.GetSequenceByName(sctx.GetInfoSchema(), model.NewCIStr(db), model.NewCIStr(name)) + sequence, err := util.GetSequenceByName(sctx.GetInfoSchema(), ast.NewCIStr(db), ast.NewCIStr(name)) if err != nil { return nil, err } diff --git a/pkg/expression/simple_rewriter_test.go b/pkg/expression/simple_rewriter_test.go index e30d6dba56264..5cabe3c93bf38 100644 --- a/pkg/expression/simple_rewriter_test.go +++ b/pkg/expression/simple_rewriter_test.go @@ -19,7 +19,6 @@ import ( "testing" "github.com/pingcap/tidb/pkg/parser/ast" - "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/types" "github.com/stretchr/testify/require" ) @@ -51,15 +50,15 @@ func generateTestData(size int) (types.NameSlice, *ast.ColumnName) { names := make(types.NameSlice, size) for i := 0; i < size; i++ { names[i] = &types.FieldName{ - DBName: model.NewCIStr("db"), - TblName: model.NewCIStr("tbl"), - ColName: model.NewCIStr("col" + string(rune('A'+i))), + DBName: ast.NewCIStr("db"), + TblName: ast.NewCIStr("tbl"), + ColName: ast.NewCIStr("col" + string(rune('A'+i))), } } astCol := &ast.ColumnName{ - Schema: model.NewCIStr("db"), - Table: model.NewCIStr("tbl"), - Name: model.NewCIStr("colZ"), + Schema: ast.NewCIStr("db"), + Table: ast.NewCIStr("tbl"), + Name: ast.NewCIStr("colZ"), } return names, astCol } @@ -75,89 +74,89 @@ func TestFindFieldName(t *testing.T) { { name: "Simple match", names: types.NameSlice{ - {DBName: model.NewCIStr("db"), TblName: model.NewCIStr("tbl"), ColName: model.NewCIStr("col")}, + {DBName: ast.NewCIStr("db"), TblName: ast.NewCIStr("tbl"), ColName: ast.NewCIStr("col")}, }, - astCol: &ast.ColumnName{Schema: model.NewCIStr("db"), Table: model.NewCIStr("tbl"), Name: model.NewCIStr("col")}, + astCol: &ast.ColumnName{Schema: ast.NewCIStr("db"), Table: ast.NewCIStr("tbl"), Name: ast.NewCIStr("col")}, expected: 0, }, { name: "Match with empty schema and table", names: types.NameSlice{ - {DBName: model.NewCIStr("db"), TblName: model.NewCIStr("tbl"), ColName: model.NewCIStr("col")}, + {DBName: ast.NewCIStr("db"), TblName: ast.NewCIStr("tbl"), ColName: ast.NewCIStr("col")}, }, - astCol: &ast.ColumnName{Schema: model.NewCIStr(""), Table: model.NewCIStr(""), Name: model.NewCIStr("col")}, + astCol: &ast.ColumnName{Schema: ast.NewCIStr(""), Table: ast.NewCIStr(""), Name: ast.NewCIStr("col")}, expected: 0, }, { name: "Match with empty schema, non-empty table", names: types.NameSlice{ - {DBName: model.NewCIStr("db"), TblName: model.NewCIStr("tbl"), ColName: model.NewCIStr("col")}, + {DBName: ast.NewCIStr("db"), TblName: ast.NewCIStr("tbl"), ColName: ast.NewCIStr("col")}, }, - astCol: &ast.ColumnName{Schema: model.NewCIStr(""), Table: model.NewCIStr("tbl"), Name: model.NewCIStr("col")}, + astCol: &ast.ColumnName{Schema: ast.NewCIStr(""), Table: ast.NewCIStr("tbl"), Name: ast.NewCIStr("col")}, expected: 0, }, { name: "Match with non-empty schema, empty table", names: types.NameSlice{ - {DBName: model.NewCIStr("db"), TblName: model.NewCIStr("tbl"), ColName: model.NewCIStr("col")}, + {DBName: ast.NewCIStr("db"), TblName: ast.NewCIStr("tbl"), ColName: ast.NewCIStr("col")}, }, - astCol: &ast.ColumnName{Schema: model.NewCIStr("db"), Table: model.NewCIStr(""), Name: model.NewCIStr("col")}, + astCol: &ast.ColumnName{Schema: ast.NewCIStr("db"), Table: ast.NewCIStr(""), Name: ast.NewCIStr("col")}, expected: 0, }, { name: "No match", names: types.NameSlice{ - {DBName: model.NewCIStr("db"), TblName: model.NewCIStr("tbl"), ColName: model.NewCIStr("col1")}, + {DBName: ast.NewCIStr("db"), TblName: ast.NewCIStr("tbl"), ColName: ast.NewCIStr("col1")}, }, - astCol: &ast.ColumnName{Schema: model.NewCIStr("db"), Table: model.NewCIStr("tbl"), Name: model.NewCIStr("col2")}, + astCol: &ast.ColumnName{Schema: ast.NewCIStr("db"), Table: ast.NewCIStr("tbl"), Name: ast.NewCIStr("col2")}, expected: -1, }, { name: "Match with redundant field", names: types.NameSlice{ - {DBName: model.NewCIStr("db"), TblName: model.NewCIStr("tbl"), ColName: model.NewCIStr("col"), Redundant: true}, - {DBName: model.NewCIStr("db"), TblName: model.NewCIStr("tbl"), ColName: model.NewCIStr("col"), Redundant: true}, - {DBName: model.NewCIStr("db"), TblName: model.NewCIStr("tbl"), ColName: model.NewCIStr("col")}, + {DBName: ast.NewCIStr("db"), TblName: ast.NewCIStr("tbl"), ColName: ast.NewCIStr("col"), Redundant: true}, + {DBName: ast.NewCIStr("db"), TblName: ast.NewCIStr("tbl"), ColName: ast.NewCIStr("col"), Redundant: true}, + {DBName: ast.NewCIStr("db"), TblName: ast.NewCIStr("tbl"), ColName: ast.NewCIStr("col")}, }, - astCol: &ast.ColumnName{Schema: model.NewCIStr("db"), Table: model.NewCIStr("tbl"), Name: model.NewCIStr("col")}, + astCol: &ast.ColumnName{Schema: ast.NewCIStr("db"), Table: ast.NewCIStr("tbl"), Name: ast.NewCIStr("col")}, expected: 2, }, { name: "Non-unique match", names: types.NameSlice{ - {DBName: model.NewCIStr("db"), TblName: model.NewCIStr("tbl"), ColName: model.NewCIStr("col")}, - {DBName: model.NewCIStr("db"), TblName: model.NewCIStr("tbl"), ColName: model.NewCIStr("col")}, + {DBName: ast.NewCIStr("db"), TblName: ast.NewCIStr("tbl"), ColName: ast.NewCIStr("col")}, + {DBName: ast.NewCIStr("db"), TblName: ast.NewCIStr("tbl"), ColName: ast.NewCIStr("col")}, }, - astCol: &ast.ColumnName{Schema: model.NewCIStr("db"), Table: model.NewCIStr("tbl"), Name: model.NewCIStr("col")}, + astCol: &ast.ColumnName{Schema: ast.NewCIStr("db"), Table: ast.NewCIStr("tbl"), Name: ast.NewCIStr("col")}, err: errNonUniq.GenWithStackByArgs("db.tbl.col", "field list"), }, { name: "Match with empty schema and table and redundant", names: types.NameSlice{ - {DBName: model.NewCIStr("db"), TblName: model.NewCIStr("tbl"), ColName: model.NewCIStr("col"), Redundant: true}, - {DBName: model.NewCIStr("db"), TblName: model.NewCIStr("tbl"), ColName: model.NewCIStr("col")}, + {DBName: ast.NewCIStr("db"), TblName: ast.NewCIStr("tbl"), ColName: ast.NewCIStr("col"), Redundant: true}, + {DBName: ast.NewCIStr("db"), TblName: ast.NewCIStr("tbl"), ColName: ast.NewCIStr("col")}, }, - astCol: &ast.ColumnName{Schema: model.NewCIStr(""), Table: model.NewCIStr(""), Name: model.NewCIStr("col")}, + astCol: &ast.ColumnName{Schema: ast.NewCIStr(""), Table: ast.NewCIStr(""), Name: ast.NewCIStr("col")}, expected: 1, }, { name: "Non-unique match with a redundant", names: types.NameSlice{ - {DBName: model.NewCIStr("db"), TblName: model.NewCIStr("tbl"), ColName: model.NewCIStr("col"), Redundant: true}, - {DBName: model.NewCIStr("db"), TblName: model.NewCIStr("tbl"), ColName: model.NewCIStr("col")}, - {DBName: model.NewCIStr("db"), TblName: model.NewCIStr("tbl"), ColName: model.NewCIStr("col")}, + {DBName: ast.NewCIStr("db"), TblName: ast.NewCIStr("tbl"), ColName: ast.NewCIStr("col"), Redundant: true}, + {DBName: ast.NewCIStr("db"), TblName: ast.NewCIStr("tbl"), ColName: ast.NewCIStr("col")}, + {DBName: ast.NewCIStr("db"), TblName: ast.NewCIStr("tbl"), ColName: ast.NewCIStr("col")}, }, - astCol: &ast.ColumnName{Schema: model.NewCIStr("db"), Table: model.NewCIStr("tbl"), Name: model.NewCIStr("col")}, + astCol: &ast.ColumnName{Schema: ast.NewCIStr("db"), Table: ast.NewCIStr("tbl"), Name: ast.NewCIStr("col")}, err: errNonUniq.GenWithStackByArgs("db.tbl.col", "field list"), }, { name: "Match with multiple redundant", names: types.NameSlice{ - {DBName: model.NewCIStr("db"), TblName: model.NewCIStr("tbl"), ColName: model.NewCIStr("col"), Redundant: true}, - {DBName: model.NewCIStr("db"), TblName: model.NewCIStr("tbl"), ColName: model.NewCIStr("col"), Redundant: true}, - {DBName: model.NewCIStr("db"), TblName: model.NewCIStr("tbl"), ColName: model.NewCIStr("col"), Redundant: true}, + {DBName: ast.NewCIStr("db"), TblName: ast.NewCIStr("tbl"), ColName: ast.NewCIStr("col"), Redundant: true}, + {DBName: ast.NewCIStr("db"), TblName: ast.NewCIStr("tbl"), ColName: ast.NewCIStr("col"), Redundant: true}, + {DBName: ast.NewCIStr("db"), TblName: ast.NewCIStr("tbl"), ColName: ast.NewCIStr("col"), Redundant: true}, }, - astCol: &ast.ColumnName{Schema: model.NewCIStr("db"), Table: model.NewCIStr("tbl"), Name: model.NewCIStr("col")}, + astCol: &ast.ColumnName{Schema: ast.NewCIStr("db"), Table: ast.NewCIStr("tbl"), Name: ast.NewCIStr("col")}, expected: 0, }, } diff --git a/pkg/expression/test/multivaluedindex/BUILD.bazel b/pkg/expression/test/multivaluedindex/BUILD.bazel index 98443c141a278..beb311cf6e7e0 100644 --- a/pkg/expression/test/multivaluedindex/BUILD.bazel +++ b/pkg/expression/test/multivaluedindex/BUILD.bazel @@ -13,7 +13,7 @@ go_test( "//pkg/config", "//pkg/errno", "//pkg/kv", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/sessiontxn", "//pkg/table", "//pkg/tablecodec", diff --git a/pkg/expression/test/multivaluedindex/multi_valued_index_test.go b/pkg/expression/test/multivaluedindex/multi_valued_index_test.go index e84613b4ab0d5..a49d85b146270 100644 --- a/pkg/expression/test/multivaluedindex/multi_valued_index_test.go +++ b/pkg/expression/test/multivaluedindex/multi_valued_index_test.go @@ -20,7 +20,7 @@ import ( "github.com/pingcap/tidb/pkg/errno" "github.com/pingcap/tidb/pkg/kv" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/sessiontxn" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/tablecodec" @@ -42,7 +42,7 @@ func TestWriteMultiValuedIndex(t *testing.T) { tk.MustExec("insert into t1 values (5, null)") tk.MustExec("insert into t1 values (6, '1')") - t1, err := dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t1")) + t1, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t1")) require.NoError(t, err) for _, index := range t1.Indices() { if index.Meta().MVIndex { @@ -76,7 +76,7 @@ func TestWriteMultiValuedIndex(t *testing.T) { tk.MustExec("insert into t1 values (3, '[\"b \"]')") tk.MustQuery("select pk from t1 where 'b ' member of (a)").Check(testkit.Rows("3")) - t1, err = dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t1")) + t1, err = dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t1")) require.NoError(t, err) for _, index := range t1.Indices() { if index.Meta().MVIndex { @@ -122,7 +122,7 @@ func TestWriteMultiValuedIndex(t *testing.T) { tk.MustExec("insert into t1 values (5, null)") tk.MustExec("insert into t1 values (6, '1')") - t1, err = dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t1")) + t1, err = dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t1")) require.NoError(t, err) for _, index := range t1.Indices() { if index.Meta().MVIndex { @@ -167,7 +167,7 @@ func TestWriteMultiValuedIndexPartitionTable(t *testing.T) { tk.MustExec("insert into t1 values (3, null)") tk.MustExec("insert into t1 values (13, null)") - t1, err := dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t1")) + t1, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t1")) require.NoError(t, err) expect := map[string]struct { @@ -221,7 +221,7 @@ func TestWriteMultiValuedIndexUnique(t *testing.T) { tk.MustGetErrCode("insert into t1 values (2, '[1]')", errno.ErrDupEntry) tk.MustExec("insert into t1 values (3, '[3,3,4]')") - t1, err := dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t1")) + t1, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t1")) require.NoError(t, err) for _, index := range t1.Indices() { if index.Meta().MVIndex { @@ -247,7 +247,7 @@ func TestWriteMultiValuedIndexComposite(t *testing.T) { tk.MustExec("insert into t1 values (4, null, 4, 4)") tk.MustExec("insert into t1 values (5, '[]', 5, 5)") - t1, err := dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t1")) + t1, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t1")) require.NoError(t, err) for _, index := range t1.Indices() { if index.Meta().MVIndex { diff --git a/pkg/expression/util_test.go b/pkg/expression/util_test.go index 2ce8177f36686..fc4b6cf0d9d91 100644 --- a/pkg/expression/util_test.go +++ b/pkg/expression/util_test.go @@ -20,7 +20,6 @@ import ( "time" "github.com/pingcap/tidb/pkg/parser/ast" - "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/cascades/base" "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" @@ -205,7 +204,7 @@ func TestPopRowFirstArg(t *testing.T) { c1, c2, c3 := &Column{RetType: newIntFieldType()}, &Column{RetType: newIntFieldType()}, &Column{RetType: newIntFieldType()} f, err := funcs[ast.RowFunc].getFunction(ctx, []Expression{c1, c2, c3}) require.NoError(t, err) - fun := &ScalarFunction{Function: f, FuncName: model.NewCIStr(ast.RowFunc), RetType: newIntFieldType()} + fun := &ScalarFunction{Function: f, FuncName: ast.NewCIStr(ast.RowFunc), RetType: newIntFieldType()} fun2, err := PopRowFirstArg(mock.NewContext(), fun) require.NoError(t, err) require.Len(t, fun2.(*ScalarFunction).GetArgs(), 2) diff --git a/pkg/expression/vs_helper.go b/pkg/expression/vs_helper.go index 921dc73d4b6c5..d5fbdb811568d 100644 --- a/pkg/expression/vs_helper.go +++ b/pkg/expression/vs_helper.go @@ -18,7 +18,6 @@ import ( "strings" "github.com/pingcap/tidb/pkg/parser/ast" - "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/intest" @@ -38,7 +37,7 @@ var ( // NOTE: not all VectorSearch functions are supported by the index. The caller // needs to check the distance function name. type VSInfo struct { - DistanceFnName model.CIStr + DistanceFnName ast.CIStr FnPbCode tipb.ScalarFuncSig Vec types.VectorFloat32 Column *Column diff --git a/pkg/infoschema/BUILD.bazel b/pkg/infoschema/BUILD.bazel index 55f8b03cb8746..8d9a7e70e82ae 100644 --- a/pkg/infoschema/BUILD.bazel +++ b/pkg/infoschema/BUILD.bazel @@ -34,9 +34,9 @@ go_library( "//pkg/meta/autoid", "//pkg/meta/model", "//pkg/metrics", + "//pkg/parser/ast", "//pkg/parser/auth", "//pkg/parser/charset", - "//pkg/parser/model", "//pkg/parser/mysql", "//pkg/parser/terror", "//pkg/privilege", @@ -104,7 +104,7 @@ go_test( "//pkg/meta", "//pkg/meta/autoid", "//pkg/meta/model", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/parser/mysql", "//pkg/sessionctx/variable", "//pkg/store/driver", diff --git a/pkg/infoschema/builder.go b/pkg/infoschema/builder.go index 5656ff9f76349..a1826ff5c41a4 100644 --- a/pkg/infoschema/builder.go +++ b/pkg/infoschema/builder.go @@ -30,8 +30,8 @@ import ( "github.com/pingcap/tidb/pkg/meta" "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/parser/charset" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/table" @@ -961,7 +961,7 @@ func (b *Builder) createSchemaTablesForDB(di *model.DBInfo, tableFromMeta tableF item := tableItem{ dbName: di.Name, dbID: di.ID, - tableName: pmodel.NewCIStr(name), + tableName: ast.NewCIStr(name), tableID: id, schemaVersion: schemaVersion, } diff --git a/pkg/infoschema/context/BUILD.bazel b/pkg/infoschema/context/BUILD.bazel index 32f386dd6deea..db3be7d8ec3af 100644 --- a/pkg/infoschema/context/BUILD.bazel +++ b/pkg/infoschema/context/BUILD.bazel @@ -8,6 +8,6 @@ go_library( deps = [ "//pkg/ddl/placement", "//pkg/meta/model", - "//pkg/parser/model", + "//pkg/parser/ast", ], ) diff --git a/pkg/infoschema/context/infoschema.go b/pkg/infoschema/context/infoschema.go index 17b73faf19f15..46cfc885f6f82 100644 --- a/pkg/infoschema/context/infoschema.go +++ b/pkg/infoschema/context/infoschema.go @@ -19,7 +19,7 @@ import ( "github.com/pingcap/tidb/pkg/ddl/placement" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" ) // SpecialAttributeFilter is used to filter tables with special attributes. @@ -91,7 +91,7 @@ var AllSpecialAttribute SpecialAttributeFilter = HasSpecialAttributes // TableInfoResult is used to store the result of ListTablesWithSpecialAttribute. type TableInfoResult struct { - DBName pmodel.CIStr + DBName ast.CIStr TableInfos []*model.TableInfo } @@ -100,16 +100,16 @@ type TableInfoResult struct { // But MetaOnlyInfoSchema is widely used for scenes that require meta only, so we give a convenience for that. type MetaOnlyInfoSchema interface { SchemaMetaVersion() int64 - SchemaByName(schema pmodel.CIStr) (*model.DBInfo, bool) - SchemaExists(schema pmodel.CIStr) bool - TableInfoByName(schema, table pmodel.CIStr) (*model.TableInfo, error) + SchemaByName(schema ast.CIStr) (*model.DBInfo, bool) + SchemaExists(schema ast.CIStr) bool + TableInfoByName(schema, table ast.CIStr) (*model.TableInfo, error) TableInfoByID(id int64) (*model.TableInfo, bool) FindTableInfoByPartitionID(partitionID int64) (*model.TableInfo, *model.DBInfo, *model.PartitionDefinition) - TableExists(schema, table pmodel.CIStr) bool + TableExists(schema, table ast.CIStr) bool SchemaByID(id int64) (*model.DBInfo, bool) SchemaAndTable - AllSchemaNames() []pmodel.CIStr - SchemaSimpleTableInfos(ctx stdctx.Context, schema pmodel.CIStr) ([]*model.TableNameInfo, error) + AllSchemaNames() []ast.CIStr + SchemaSimpleTableInfos(ctx stdctx.Context, schema ast.CIStr) ([]*model.TableNameInfo, error) ListTablesWithSpecialAttribute(filter SpecialAttributeFilter) []TableInfoResult Misc } @@ -117,13 +117,13 @@ type MetaOnlyInfoSchema interface { // SchemaAndTable is define for iterating all the schemas and tables in the infoschema. type SchemaAndTable interface { AllSchemas() []*model.DBInfo - SchemaTableInfos(ctx stdctx.Context, schema pmodel.CIStr) ([]*model.TableInfo, error) + SchemaTableInfos(ctx stdctx.Context, schema ast.CIStr) ([]*model.TableInfo, error) } // Misc contains the methods that are not closely related to InfoSchema. type Misc interface { - PolicyByName(name pmodel.CIStr) (*model.PolicyInfo, bool) - ResourceGroupByName(name pmodel.CIStr) (*model.ResourceGroupInfo, bool) + PolicyByName(name ast.CIStr) (*model.PolicyInfo, bool) + ResourceGroupByName(name ast.CIStr) (*model.ResourceGroupInfo, bool) // PlacementBundleByPhysicalTableID is used to get a rule bundle. PlacementBundleByPhysicalTableID(id int64) (*placement.Bundle, bool) // AllPlacementBundles is used to get all placement bundles @@ -151,7 +151,7 @@ func (d DBInfoAsInfoSchema) AllSchemas() []*model.DBInfo { } // SchemaTableInfos implement infoschema.SchemaAndTable interface. -func (d DBInfoAsInfoSchema) SchemaTableInfos(ctx stdctx.Context, schema pmodel.CIStr) ([]*model.TableInfo, error) { +func (d DBInfoAsInfoSchema) SchemaTableInfos(ctx stdctx.Context, schema ast.CIStr) ([]*model.TableInfo, error) { for _, db := range d { if db.Name == schema { return db.Deprecated.Tables, nil diff --git a/pkg/infoschema/infoschema.go b/pkg/infoschema/infoschema.go index 145a4cabc2126..7ab058d98309a 100644 --- a/pkg/infoschema/infoschema.go +++ b/pkg/infoschema/infoschema.go @@ -27,7 +27,7 @@ import ( "github.com/pingcap/tidb/pkg/infoschema/context" "github.com/pingcap/tidb/pkg/meta/autoid" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/table" @@ -101,7 +101,7 @@ type SchemaAndTableName struct { // MockInfoSchema only serves for test. func MockInfoSchema(tbList []*model.TableInfo) InfoSchema { result := newInfoSchema() - dbInfo := &model.DBInfo{ID: 1, Name: pmodel.NewCIStr("test")} + dbInfo := &model.DBInfo{ID: 1, Name: ast.NewCIStr("test")} dbInfo.Deprecated.Tables = tbList tableNames := &schemaTables{ dbInfo: dbInfo, @@ -130,19 +130,19 @@ func MockInfoSchema(tbList []*model.TableInfo) InfoSchema { { // Use a very big ID to avoid conflict with normal tables. ID: 9999, - Name: pmodel.NewCIStr("stats_meta"), + Name: ast.NewCIStr("stats_meta"), Columns: []*model.ColumnInfo{ { State: model.StatePublic, Offset: 0, - Name: pmodel.NewCIStr("a"), + Name: ast.NewCIStr("a"), ID: 1, }, }, State: model.StatePublic, }, } - mysqlDBInfo := &model.DBInfo{ID: 2, Name: pmodel.NewCIStr("mysql")} + mysqlDBInfo := &model.DBInfo{ID: 2, Name: ast.NewCIStr("mysql")} mysqlDBInfo.Deprecated.Tables = tables tableNames = &schemaTables{ dbInfo: mysqlDBInfo, @@ -167,7 +167,7 @@ func MockInfoSchema(tbList []*model.TableInfo) InfoSchema { // MockInfoSchemaWithSchemaVer only serves for test. func MockInfoSchemaWithSchemaVer(tbList []*model.TableInfo, schemaVer int64) InfoSchema { result := newInfoSchema() - dbInfo := &model.DBInfo{ID: 1, Name: pmodel.NewCIStr("test")} + dbInfo := &model.DBInfo{ID: 1, Name: ast.NewCIStr("test")} dbInfo.Deprecated.Tables = tbList tableNames := &schemaTables{ dbInfo: dbInfo, @@ -210,7 +210,7 @@ func newInfoSchema() *infoSchema { } } -func (is *infoSchema) SchemaByName(schema pmodel.CIStr) (val *model.DBInfo, ok bool) { +func (is *infoSchema) SchemaByName(schema ast.CIStr) (val *model.DBInfo, ok bool) { return is.schemaByName(schema.L) } @@ -222,12 +222,12 @@ func (is *infoSchema) schemaByName(name string) (val *model.DBInfo, ok bool) { return tableNames.dbInfo, true } -func (is *infoSchema) SchemaExists(schema pmodel.CIStr) bool { +func (is *infoSchema) SchemaExists(schema ast.CIStr) bool { _, ok := is.schemaMap[schema.L] return ok } -func (is *infoSchema) TableByName(ctx stdctx.Context, schema, table pmodel.CIStr) (t table.Table, err error) { +func (is *infoSchema) TableByName(ctx stdctx.Context, schema, table ast.CIStr) (t table.Table, err error) { if tbNames, ok := is.schemaMap[schema.L]; ok { if t, ok = tbNames.tables[table.L]; ok { return @@ -237,13 +237,13 @@ func (is *infoSchema) TableByName(ctx stdctx.Context, schema, table pmodel.CIStr } // TableInfoByName implements InfoSchema.TableInfoByName -func (is *infoSchema) TableInfoByName(schema, table pmodel.CIStr) (*model.TableInfo, error) { +func (is *infoSchema) TableInfoByName(schema, table ast.CIStr) (*model.TableInfo, error) { tbl, err := is.TableByName(stdctx.Background(), schema, table) return getTableInfo(tbl), err } // TableIsView indicates whether the schema.table is a view. -func TableIsView(is InfoSchema, schema, table pmodel.CIStr) bool { +func TableIsView(is InfoSchema, schema, table ast.CIStr) bool { tbl, err := is.TableByName(stdctx.Background(), schema, table) if err == nil { return tbl.Meta().IsView() @@ -252,7 +252,7 @@ func TableIsView(is InfoSchema, schema, table pmodel.CIStr) bool { } // TableIsSequence indicates whether the schema.table is a sequence. -func TableIsSequence(is InfoSchema, schema, table pmodel.CIStr) bool { +func TableIsSequence(is InfoSchema, schema, table ast.CIStr) bool { tbl, err := is.TableByName(stdctx.Background(), schema, table) if err == nil { return tbl.Meta().IsSequence() @@ -260,7 +260,7 @@ func TableIsSequence(is InfoSchema, schema, table pmodel.CIStr) bool { return false } -func (is *infoSchema) TableExists(schema, table pmodel.CIStr) bool { +func (is *infoSchema) TableExists(schema, table ast.CIStr) bool { if tbNames, ok := is.schemaMap[schema.L]; ok { if _, ok = tbNames.tables[table.L]; ok { return true @@ -315,7 +315,7 @@ func (is *infoSchema) TableByID(_ stdctx.Context, id int64) (val table.Table, ok return slice[idx], true } -func (is *infoSchema) SchemaNameByTableID(tableID int64) (schemaName pmodel.CIStr, ok bool) { +func (is *infoSchema) SchemaNameByTableID(tableID int64) (schemaName ast.CIStr, ok bool) { tbl, ok := is.TableByID(stdctx.Background(), tableID) if !ok { return @@ -343,7 +343,7 @@ func (is *infoSchema) FindTableInfoByPartitionID( } // SchemaTableInfos implements MetaOnlyInfoSchema. -func (is *infoSchema) SchemaTableInfos(ctx stdctx.Context, schema pmodel.CIStr) ([]*model.TableInfo, error) { +func (is *infoSchema) SchemaTableInfos(ctx stdctx.Context, schema ast.CIStr) ([]*model.TableInfo, error) { schemaTables, ok := is.schemaMap[schema.L] if !ok { return nil, nil @@ -356,7 +356,7 @@ func (is *infoSchema) SchemaTableInfos(ctx stdctx.Context, schema pmodel.CIStr) } // SchemaSimpleTableInfos implements MetaOnlyInfoSchema. -func (is *infoSchema) SchemaSimpleTableInfos(ctx stdctx.Context, schema pmodel.CIStr) ([]*model.TableNameInfo, error) { +func (is *infoSchema) SchemaSimpleTableInfos(ctx stdctx.Context, schema ast.CIStr) ([]*model.TableNameInfo, error) { schemaTables, ok := is.schemaMap[schema.L] if !ok { return nil, nil @@ -404,8 +404,8 @@ func (is *infoSchema) AllSchemas() (schemas []*model.DBInfo) { return } -func (is *infoSchema) AllSchemaNames() (schemas []pmodel.CIStr) { - rs := make([]pmodel.CIStr, 0, len(is.schemaMap)) +func (is *infoSchema) AllSchemaNames() (schemas []ast.CIStr) { + rs := make([]ast.CIStr, 0, len(is.schemaMap)) for _, v := range is.schemaMap { rs = append(rs, v.dbInfo.Name) } @@ -453,7 +453,7 @@ func (is *infoSchemaMisc) SchemaMetaVersion() int64 { } // GetSequenceByName gets the sequence by name. -func GetSequenceByName(is InfoSchema, schema, sequence pmodel.CIStr) (util.SequenceTable, error) { +func GetSequenceByName(is InfoSchema, schema, sequence ast.CIStr) (util.SequenceTable, error) { tbl, err := is.TableByName(stdctx.Background(), schema, sequence) if err != nil { return nil, err @@ -491,7 +491,7 @@ func init() { } infoSchemaDB.Deprecated.Tables = infoSchemaTables RegisterVirtualTable(infoSchemaDB, createInfoSchemaTable) - util.GetSequenceByName = func(is context.MetaOnlyInfoSchema, schema, sequence pmodel.CIStr) (util.SequenceTable, error) { + util.GetSequenceByName = func(is context.MetaOnlyInfoSchema, schema, sequence ast.CIStr) (util.SequenceTable, error) { return GetSequenceByName(is.(InfoSchema), schema, sequence) } mock.MockInfoschema = func(tbList []*model.TableInfo) context.MetaOnlyInfoSchema { @@ -510,7 +510,7 @@ func HasAutoIncrementColumn(tbInfo *model.TableInfo) (bool, string) { } // PolicyByName is used to find the policy. -func (is *infoSchemaMisc) PolicyByName(name pmodel.CIStr) (*model.PolicyInfo, bool) { +func (is *infoSchemaMisc) PolicyByName(name ast.CIStr) (*model.PolicyInfo, bool) { is.policyMutex.RLock() defer is.policyMutex.RUnlock() t, r := is.policyMap[name.L] @@ -518,7 +518,7 @@ func (is *infoSchemaMisc) PolicyByName(name pmodel.CIStr) (*model.PolicyInfo, bo } // ResourceGroupByName is used to find the resource group. -func (is *infoSchemaMisc) ResourceGroupByName(name pmodel.CIStr) (*model.ResourceGroupInfo, bool) { +func (is *infoSchemaMisc) ResourceGroupByName(name ast.CIStr) (*model.ResourceGroupInfo, bool) { is.resourceGroupMutex.RLock() defer is.resourceGroupMutex.RUnlock() t, r := is.resourceGroupMap[name.L] @@ -608,7 +608,7 @@ func (is *infoSchemaMisc) deletePolicy(name string) { delete(is.policyMap, name) } -func (is *infoSchemaMisc) addReferredForeignKeys(schema pmodel.CIStr, tbInfo *model.TableInfo) { +func (is *infoSchemaMisc) addReferredForeignKeys(schema ast.CIStr, tbInfo *model.TableInfo) { for _, fk := range tbInfo.ForeignKeys { if fk.Version < model.FKVersion1 { continue @@ -648,7 +648,7 @@ func (is *infoSchemaMisc) addReferredForeignKeys(schema pmodel.CIStr, tbInfo *mo } } -func (is *infoSchemaMisc) deleteReferredForeignKeys(schema pmodel.CIStr, tbInfo *model.TableInfo) { +func (is *infoSchemaMisc) deleteReferredForeignKeys(schema ast.CIStr, tbInfo *model.TableInfo) { for _, fk := range tbInfo.ForeignKeys { if fk.Version < model.FKVersion1 { continue @@ -693,7 +693,7 @@ func NewSessionTables() *SessionTables { } // TableByName get table by name -func (is *SessionTables) TableByName(ctx stdctx.Context, schema, table pmodel.CIStr) (table.Table, bool) { +func (is *SessionTables) TableByName(ctx stdctx.Context, schema, table ast.CIStr) (table.Table, bool) { if tbNames, ok := is.schemaMap[schema.L]; ok { if t, ok := tbNames.tables[table.L]; ok { return t, true @@ -703,7 +703,7 @@ func (is *SessionTables) TableByName(ctx stdctx.Context, schema, table pmodel.CI } // TableExists check if table with the name exists -func (is *SessionTables) TableExists(schema, table pmodel.CIStr) (ok bool) { +func (is *SessionTables) TableExists(schema, table ast.CIStr) (ok bool) { _, ok = is.TableByName(stdctx.Background(), schema, table) return } @@ -734,7 +734,7 @@ func (is *SessionTables) AddTable(db *model.DBInfo, tbl table.Table) error { } // RemoveTable remove a table -func (is *SessionTables) RemoveTable(schema, table pmodel.CIStr) (exist bool) { +func (is *SessionTables) RemoveTable(schema, table ast.CIStr) (exist bool) { tbls := is.schemaTables(schema) if tbls == nil { return false @@ -779,7 +779,7 @@ func (is *SessionTables) ensureSchema(db *model.DBInfo) *schemaTables { return tbls } -func (is *SessionTables) schemaTables(schema pmodel.CIStr) *schemaTables { +func (is *SessionTables) schemaTables(schema ast.CIStr) *schemaTables { if is.schemaMap == nil { return nil } @@ -802,7 +802,7 @@ type SessionExtendedInfoSchema struct { } // TableByName implements InfoSchema.TableByName -func (ts *SessionExtendedInfoSchema) TableByName(ctx stdctx.Context, schema, table pmodel.CIStr) (table.Table, error) { +func (ts *SessionExtendedInfoSchema) TableByName(ctx stdctx.Context, schema, table ast.CIStr) (table.Table, error) { if ts.LocalTemporaryTables != nil { if tbl, ok := ts.LocalTemporaryTables.TableByName(ctx, schema, table); ok { return tbl, nil @@ -819,7 +819,7 @@ func (ts *SessionExtendedInfoSchema) TableByName(ctx stdctx.Context, schema, tab } // TableInfoByName implements InfoSchema.TableInfoByName -func (ts *SessionExtendedInfoSchema) TableInfoByName(schema, table pmodel.CIStr) (*model.TableInfo, error) { +func (ts *SessionExtendedInfoSchema) TableInfoByName(schema, table ast.CIStr) (*model.TableInfo, error) { tbl, err := ts.TableByName(stdctx.Background(), schema, table) return getTableInfo(tbl), err } diff --git a/pkg/infoschema/infoschema_nokit_test.go b/pkg/infoschema/infoschema_nokit_test.go index 35aefe7961680..cfa4c99b0f97c 100644 --- a/pkg/infoschema/infoschema_nokit_test.go +++ b/pkg/infoschema/infoschema_nokit_test.go @@ -18,7 +18,7 @@ import ( "testing" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/stretchr/testify/require" ) @@ -32,11 +32,11 @@ func (is *infoschemaV2) HasCache(tableID int64, schemaVersion int64) bool { func TestInfoSchemaAddDel(t *testing.T) { is := newInfoSchema() is.addSchema(&schemaTables{ - dbInfo: &model.DBInfo{ID: 1, Name: pmodel.NewCIStr("test")}, + dbInfo: &model.DBInfo{ID: 1, Name: ast.NewCIStr("test")}, }) require.Contains(t, is.schemaMap, "test") require.Contains(t, is.schemaID2Name, int64(1)) - is.delSchema(&model.DBInfo{ID: 1, Name: pmodel.NewCIStr("test")}) + is.delSchema(&model.DBInfo{ID: 1, Name: ast.NewCIStr("test")}) require.Empty(t, is.schemaMap) require.Empty(t, is.schemaID2Name) } diff --git a/pkg/infoschema/infoschema_test.go b/pkg/infoschema/infoschema_test.go index ac96ed62db92f..89a9c78abea9d 100644 --- a/pkg/infoschema/infoschema_test.go +++ b/pkg/infoschema/infoschema_test.go @@ -31,7 +31,7 @@ import ( "github.com/pingcap/tidb/pkg/meta" "github.com/pingcap/tidb/pkg/meta/autoid" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/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/table" @@ -49,11 +49,11 @@ func TestBasic(t *testing.T) { require.NoError(t, err) }() - dbName := pmodel.NewCIStr("Test") - tbName := pmodel.NewCIStr("T") - colName := pmodel.NewCIStr("A") - idxName := pmodel.NewCIStr("idx") - noexist := pmodel.NewCIStr("noexist") + dbName := ast.NewCIStr("Test") + tbName := ast.NewCIStr("T") + colName := ast.NewCIStr("A") + idxName := ast.NewCIStr("idx") + noexist := ast.NewCIStr("noexist") colID, err := internal.GenGlobalID(re.Store()) require.NoError(t, err) @@ -230,7 +230,7 @@ func TestBasic(t *testing.T) { require.Len(t, tblInfos, 0) // Make sure partitions table exists - tb, err = is.TableByName(context.Background(), pmodel.NewCIStr("information_schema"), pmodel.NewCIStr("partitions")) + tb, err = is.TableByName(context.Background(), ast.NewCIStr("information_schema"), ast.NewCIStr("partitions")) require.NoError(t, err) require.NotNil(t, tb) @@ -258,7 +258,7 @@ func TestBasic(t *testing.T) { tests := []struct { name string tableID int64 - wantSchema pmodel.CIStr + wantSchema ast.CIStr wantOK bool }{ { @@ -270,13 +270,13 @@ func TestBasic(t *testing.T) { { name: "non-existent table ID", tableID: tbID + 1, - wantSchema: pmodel.CIStr{}, + wantSchema: ast.CIStr{}, wantOK: false, }, { name: "invalid table ID (negative)", tableID: -1, - wantSchema: pmodel.CIStr{}, + wantSchema: ast.CIStr{}, wantOK: false, }, } @@ -292,7 +292,7 @@ func TestBasic(t *testing.T) { func TestMockInfoSchema(t *testing.T) { tblID := int64(1234) - tblName := pmodel.NewCIStr("tbl_m") + tblName := ast.NewCIStr("tbl_m") tableInfo := &model.TableInfo{ ID: tblID, Name: tblName, @@ -301,7 +301,7 @@ func TestMockInfoSchema(t *testing.T) { colInfo := &model.ColumnInfo{ State: model.StatePublic, Offset: 0, - Name: pmodel.NewCIStr("h"), + Name: ast.NewCIStr("h"), FieldType: *types.NewFieldType(mysql.TypeLong), ID: 1, } @@ -374,7 +374,7 @@ func TestInfoTables(t *testing.T) { "VIEWS", } for _, tbl := range infoTables { - tb, err1 := is.TableByName(context.Background(), util.InformationSchemaName, pmodel.NewCIStr(tbl)) + tb, err1 := is.TableByName(context.Background(), util.InformationSchemaName, ast.NewCIStr(tbl)) require.Nil(t, err1) require.NotNil(t, tb) } @@ -389,7 +389,7 @@ func TestBuildSchemaWithGlobalTemporaryTable(t *testing.T) { dbInfo := &model.DBInfo{ ID: 1, - Name: pmodel.NewCIStr("test"), + Name: ast.NewCIStr("test"), State: model.StatePublic, } dbInfo.Deprecated.Tables = []*model.TableInfo{} @@ -400,7 +400,7 @@ func TestBuildSchemaWithGlobalTemporaryTable(t *testing.T) { require.NoError(t, err) is := builder.Build(math.MaxUint64) require.False(t, is.HasTemporaryTable()) - db, ok := is.SchemaByName(pmodel.NewCIStr("test")) + db, ok := is.SchemaByName(ast.NewCIStr("test")) require.True(t, ok) ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnDDL) err = kv.RunInNewTxn(ctx, re.Store(), true, func(ctx context.Context, txn kv.Transaction) error { @@ -495,7 +495,7 @@ func TestBuildSchemaWithGlobalTemporaryTable(t *testing.T) { // full load data = infoschema.NewData() - newDB, ok := newIS.SchemaByName(pmodel.NewCIStr("test")) + newDB, ok := newIS.SchemaByName(ast.NewCIStr("test")) tblInfos, err := newIS.SchemaTableInfos(context.Background(), newDB.Name) require.NoError(t, err) newDB.Deprecated.Tables = tblInfos @@ -575,13 +575,13 @@ func TestBuildBundle(t *testing.T) { }() is := domain.GetDomain(tk.Session()).InfoSchema() - db, ok := is.SchemaByName(pmodel.NewCIStr("test")) + db, ok := is.SchemaByName(ast.NewCIStr("test")) require.True(t, ok) - tbl1, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t1")) + tbl1, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t1")) require.NoError(t, err) - tbl2, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t2")) + tbl2, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t2")) require.NoError(t, err) var p1 model.PartitionDefinition @@ -649,7 +649,7 @@ func TestWithRefillOption(t *testing.T) { tk.MustExec("create table t1 (id int)") tk.MustQuery("select * from t1").Check(testkit.Rows()) is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t1")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t1")) require.NoError(t, err) tblInfo := tbl.Meta() ok, v2 := infoschema.IsV2(is) @@ -672,7 +672,7 @@ func TestWithRefillOption(t *testing.T) { for i, testCase := range testCases { // Mock t1 schema cache been evicted. - v2.EvictTable(pmodel.NewCIStr("test"), pmodel.NewCIStr("t1")) + v2.EvictTable(ast.NewCIStr("test"), ast.NewCIStr("t1")) // Test the API switch testCase.OP { @@ -680,7 +680,7 @@ func TestWithRefillOption(t *testing.T) { _, found := is.TableByID(testCase.ctx, tblInfo.ID) require.True(t, found) case "TableByName": - _, err := is.TableByName(testCase.ctx, pmodel.NewCIStr("test"), pmodel.NewCIStr("t1")) + _, err := is.TableByName(testCase.ctx, ast.NewCIStr("test"), ast.NewCIStr("t1")) require.NoError(t, err) } @@ -702,7 +702,7 @@ func TestLocalTemporaryTables(t *testing.T) { require.NoError(t, err) return &model.DBInfo{ ID: schemaID, - Name: pmodel.NewCIStr(schemaName), + Name: ast.NewCIStr(schemaName), State: model.StatePublic, } } @@ -713,7 +713,7 @@ func TestLocalTemporaryTables(t *testing.T) { colInfo := &model.ColumnInfo{ ID: colID, - Name: pmodel.NewCIStr("col1"), + Name: ast.NewCIStr("col1"), Offset: 0, FieldType: *types.NewFieldType(mysql.TypeLonglong), State: model.StatePublic, @@ -724,7 +724,7 @@ func TestLocalTemporaryTables(t *testing.T) { tblInfo := &model.TableInfo{ ID: tbID, - Name: pmodel.NewCIStr(tbName), + Name: ast.NewCIStr(tbName), Columns: []*model.ColumnInfo{colInfo}, Indices: []*model.IndexInfo{}, State: model.StatePublic, @@ -739,7 +739,7 @@ func TestLocalTemporaryTables(t *testing.T) { } assertTableByName := func(sc *infoschema.SessionTables, schemaName, tableName string, schema *model.DBInfo, tb table.Table) { - got, ok := sc.TableByName(context.Background(), pmodel.NewCIStr(schemaName), pmodel.NewCIStr(tableName)) + got, ok := sc.TableByName(context.Background(), ast.NewCIStr(schemaName), ast.NewCIStr(tableName)) if tb == nil { require.Nil(t, schema) require.False(t, ok) @@ -752,7 +752,7 @@ func TestLocalTemporaryTables(t *testing.T) { } assertTableExists := func(sc *infoschema.SessionTables, schemaName, tableName string, exists bool) { - got := sc.TableExists(pmodel.NewCIStr(schemaName), pmodel.NewCIStr(tableName)) + got := sc.TableExists(ast.NewCIStr(schemaName), ast.NewCIStr(tableName)) require.Equal(t, exists, got) } @@ -857,11 +857,11 @@ func TestLocalTemporaryTables(t *testing.T) { assertTableByName(sc, db1.Name.L, tb11.Meta().Name.L, db1, tb11) // delete some tables - require.True(t, sc.RemoveTable(pmodel.NewCIStr("db1"), pmodel.NewCIStr("tb1"))) - require.True(t, sc.RemoveTable(pmodel.NewCIStr("Db2"), pmodel.NewCIStr("tB2"))) + require.True(t, sc.RemoveTable(ast.NewCIStr("db1"), ast.NewCIStr("tb1"))) + require.True(t, sc.RemoveTable(ast.NewCIStr("Db2"), ast.NewCIStr("tB2"))) tb22.Meta().DBID = 0 // SchemaByTable will get incorrect result if not reset here. - require.False(t, sc.RemoveTable(pmodel.NewCIStr("db1"), pmodel.NewCIStr("tbx"))) - require.False(t, sc.RemoveTable(pmodel.NewCIStr("dbx"), pmodel.NewCIStr("tbx"))) + require.False(t, sc.RemoveTable(ast.NewCIStr("db1"), ast.NewCIStr("tbx"))) + require.False(t, sc.RemoveTable(ast.NewCIStr("dbx"), ast.NewCIStr("tbx"))) // test non exist tables by name for _, c := range []struct{ dbName, tbName string }{ @@ -1014,14 +1014,14 @@ func TestInfoSchemaCreateTableLike(t *testing.T) { tk.MustExec("create table t1 like information_schema.variables_info;") tk.MustExec("alter table t1 add column c varchar(32);") is := domain.GetDomain(tk.Session()).InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t1")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t1")) require.NoError(t, err) tblInfo := tbl.Meta() require.Equal(t, tblInfo.Columns[8].Name.O, "c") require.Equal(t, tblInfo.Columns[8].ID, int64(9)) tk.MustExec("alter table t1 add index idx(c);") is = domain.GetDomain(tk.Session()).InfoSchema() - tbl, err = is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t1")) + tbl, err = is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t1")) require.NoError(t, err) tblInfo = tbl.Meta() require.Equal(t, tblInfo.Indices[0].Name.O, "idx") @@ -1031,14 +1031,14 @@ func TestInfoSchemaCreateTableLike(t *testing.T) { tk.MustExec("create table t2 like metrics_schema.up;") tk.MustExec("alter table t2 add column c varchar(32);") is = domain.GetDomain(tk.Session()).InfoSchema() - tbl, err = is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t2")) + tbl, err = is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t2")) require.NoError(t, err) tblInfo = tbl.Meta() require.Equal(t, tblInfo.Columns[4].Name.O, "c") require.Equal(t, tblInfo.Columns[4].ID, int64(5)) tk.MustExec("alter table t2 add index idx(c);") is = domain.GetDomain(tk.Session()).InfoSchema() - tbl, err = is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t2")) + tbl, err = is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t2")) require.NoError(t, err) tblInfo = tbl.Meta() require.Equal(t, tblInfo.Indices[0].Name.O, "idx") @@ -1207,7 +1207,7 @@ func (tc *infoschemaTestContext) runModifyTable(tblName string, tp model.ActionT } func (tc *infoschemaTestContext) runAddColumn(tblName string) { - tbl, err := tc.is.TableByName(context.Background(), tc.dbInfo.Name, pmodel.NewCIStr(tblName)) + tbl, err := tc.is.TableByName(context.Background(), tc.dbInfo.Name, ast.NewCIStr(tblName)) require.NoError(tc.t, err) tc.addColumn(tbl.Meta()) @@ -1219,7 +1219,7 @@ func (tc *infoschemaTestContext) runAddColumn(tblName string) { } func (tc *infoschemaTestContext) addColumn(tblInfo *model.TableInfo) { - colName := pmodel.NewCIStr("b") + colName := ast.NewCIStr("b") colID, err := internal.GenGlobalID(tc.re.Store()) require.NoError(tc.t, err) colInfo := &model.ColumnInfo{ @@ -1240,7 +1240,7 @@ func (tc *infoschemaTestContext) addColumn(tblInfo *model.TableInfo) { } func (tc *infoschemaTestContext) runModifyColumn(tblName string) { - tbl, err := tc.is.TableByName(context.Background(), tc.dbInfo.Name, pmodel.NewCIStr(tblName)) + tbl, err := tc.is.TableByName(context.Background(), tc.dbInfo.Name, ast.NewCIStr(tblName)) require.NoError(tc.t, err) tc.modifyColumn(tbl.Meta()) @@ -1341,7 +1341,7 @@ func TestApplyDiff(t *testing.T) { tc.runModifySchemaCharsetAndCollate("utf8mb4", "utf8mb4_general_ci") tc.runModifySchemaCharsetAndCollate("utf8", "utf8_unicode_ci") tc.runModifySchemaDefaultPlacement(&model.PolicyRefInfo{ - Name: pmodel.NewCIStr("test"), + Name: ast.NewCIStr("test"), }) tc.runCreateTables([]string{"test1", "test2"}) } diff --git a/pkg/infoschema/infoschema_v2.go b/pkg/infoschema/infoschema_v2.go index a53fd6658603e..5512ee35c97b6 100644 --- a/pkg/infoschema/infoschema_v2.go +++ b/pkg/infoschema/infoschema_v2.go @@ -34,7 +34,7 @@ import ( "github.com/pingcap/tidb/pkg/meta/autoid" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/metrics" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/util" @@ -45,9 +45,9 @@ import ( // tableItem is the btree item sorted by name or by id. type tableItem struct { - dbName pmodel.CIStr + dbName ast.CIStr dbID int64 - tableName pmodel.CIStr + tableName ast.CIStr tableID int64 schemaVersion int64 tomb bool @@ -62,7 +62,7 @@ type schemaItem struct { type schemaIDName struct { schemaVersion int64 id int64 - name pmodel.CIStr + name ast.CIStr tomb bool } @@ -142,7 +142,7 @@ type Data struct { } type tableInfoItem struct { - dbName pmodel.CIStr + dbName ast.CIStr tableID int64 schemaVersion int64 tableInfo *model.TableInfo @@ -722,7 +722,7 @@ func (is *infoschemaV2) TableByID(ctx context.Context, id int64) (val table.Tabl return ret, true } -func (is *infoschemaV2) SchemaNameByTableID(tableID int64) (schemaName pmodel.CIStr, ok bool) { +func (is *infoschemaV2) SchemaNameByTableID(tableID int64) (schemaName ast.CIStr, ok bool) { if !tableIDIsValid(tableID) { return } @@ -737,8 +737,8 @@ func (is *infoschemaV2) SchemaNameByTableID(tableID int64) (schemaName pmodel.CI // TableItem is exported from tableItem. type TableItem struct { - DBName pmodel.CIStr - TableName pmodel.CIStr + DBName ast.CIStr + TableName ast.CIStr } // IterateAllTableItems is used for special performance optimization. @@ -800,7 +800,7 @@ func IsSpecialDB(dbName string) bool { } // EvictTable is exported for testing only. -func (is *infoschemaV2) EvictTable(schema, tbl pmodel.CIStr) { +func (is *infoschemaV2) EvictTable(schema, tbl ast.CIStr) { eq := func(a, b *tableItem) bool { return a.dbName == b.dbName && a.tableName == b.tableName } itm, ok := search(is.byName.Load(), is.infoSchema.schemaMetaVersion, tableItem{dbName: schema, tableName: tbl, schemaVersion: math.MaxInt64}, eq) if !ok { @@ -834,7 +834,7 @@ func (h *tableByNameHelper) onItem(item *tableItem) bool { // TableByName implements the InfoSchema interface. // When schema cache miss, it will fetch the TableInfo from TikV and refill cache. -func (is *infoschemaV2) TableByName(ctx context.Context, schema, tbl pmodel.CIStr) (t table.Table, err error) { +func (is *infoschemaV2) TableByName(ctx context.Context, schema, tbl ast.CIStr) (t table.Table, err error) { if IsSpecialDB(schema.L) { if raw, ok := is.specials.Load(schema.L); ok { tbNames := raw.(*schemaTables) @@ -884,7 +884,7 @@ func (is *infoschemaV2) TableByName(ctx context.Context, schema, tbl pmodel.CISt } // TableInfoByName implements InfoSchema.TableInfoByName -func (is *infoschemaV2) TableInfoByName(schema, table pmodel.CIStr) (*model.TableInfo, error) { +func (is *infoschemaV2) TableInfoByName(schema, table ast.CIStr) (*model.TableInfo, error) { tbl, err := is.TableByName(context.Background(), schema, table) return getTableInfo(tbl), err } @@ -912,7 +912,7 @@ func (is *infoschemaV2) keepAlive() { } // SchemaTableInfos implements MetaOnlyInfoSchema. -func (is *infoschemaV2) SchemaTableInfos(ctx context.Context, schema pmodel.CIStr) ([]*model.TableInfo, error) { +func (is *infoschemaV2) SchemaTableInfos(ctx context.Context, schema ast.CIStr) ([]*model.TableInfo, error) { if IsSpecialDB(schema.L) { raw, ok := is.Data.specials.Load(schema.L) if ok { @@ -959,7 +959,7 @@ retry: } // SchemaSimpleTableInfos implements MetaOnlyInfoSchema. -func (is *infoschemaV2) SchemaSimpleTableInfos(ctx context.Context, schema pmodel.CIStr) ([]*model.TableNameInfo, error) { +func (is *infoschemaV2) SchemaSimpleTableInfos(ctx context.Context, schema ast.CIStr) ([]*model.TableNameInfo, error) { if IsSpecialDB(schema.L) { raw, ok := is.Data.specials.Load(schema.L) if ok { @@ -1016,7 +1016,7 @@ func (is *infoschemaV2) FindTableInfoByPartitionID( return getTableInfo(tbl), db, partDef } -func (is *infoschemaV2) SchemaByName(schema pmodel.CIStr) (val *model.DBInfo, ok bool) { +func (is *infoschemaV2) SchemaByName(schema ast.CIStr) (val *model.DBInfo, ok bool) { if IsSpecialDB(schema.L) { raw, ok := is.Data.specials.Load(schema.L) if !ok { @@ -1081,15 +1081,15 @@ func (is *infoschemaV2) AllSchemas() (schemas []*model.DBInfo) { return } -func (is *infoschemaV2) AllSchemaNames() []pmodel.CIStr { - rs := make([]pmodel.CIStr, 0, is.Data.schemaMap.Load().Len()) +func (is *infoschemaV2) AllSchemaNames() []ast.CIStr { + rs := make([]ast.CIStr, 0, is.Data.schemaMap.Load().Len()) is.allSchemas(func(di *model.DBInfo) { rs = append(rs, di.Name) }) return rs } -func (is *infoschemaV2) SchemaExists(schema pmodel.CIStr) bool { +func (is *infoschemaV2) SchemaExists(schema ast.CIStr) bool { _, ok := is.SchemaByName(schema) return ok } @@ -1143,7 +1143,7 @@ func (is *infoschemaV2) FindTableByPartitionID(partitionID int64) (table.Table, return tbl, dbInfo, def } -func (is *infoschemaV2) TableExists(schema, table pmodel.CIStr) bool { +func (is *infoschemaV2) TableExists(schema, table ast.CIStr) bool { _, err := is.TableByName(context.Background(), schema, table) return err == nil } @@ -1165,7 +1165,7 @@ func (is *infoschemaV2) SchemaByID(id int64) (*model.DBInfo, bool) { return st.dbInfo, true } var ok bool - var name pmodel.CIStr + var name ast.CIStr is.Data.schemaID2Name.Load().DescendLessOrEqual(schemaIDName{ id: id, schemaVersion: math.MaxInt64, diff --git a/pkg/infoschema/infoschema_v2_test.go b/pkg/infoschema/infoschema_v2_test.go index 0f7a4fa8f856a..498681536349b 100644 --- a/pkg/infoschema/infoschema_v2_test.go +++ b/pkg/infoschema/infoschema_v2_test.go @@ -25,7 +25,7 @@ import ( "github.com/pingcap/tidb/pkg/meta" "github.com/pingcap/tidb/pkg/meta/autoid" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/stretchr/testify/require" ) @@ -37,8 +37,8 @@ func TestV2Basic(t *testing.T) { }() is := NewInfoSchemaV2(r, nil, NewData()) - schemaName := pmodel.NewCIStr("testDB") - tableName := pmodel.NewCIStr("test") + schemaName := ast.NewCIStr("testDB") + tableName := ast.NewCIStr("test") dbInfo := internal.MockDBInfo(t, r.Store(), schemaName.O) is.Data.addDB(1, dbInfo) @@ -72,7 +72,7 @@ func TestV2Basic(t *testing.T) { require.NoError(t, err) require.Same(t, gotTblInfo, getTableInfo.Meta()) - gotTblInfo, err = is.TableInfoByName(schemaName, pmodel.NewCIStr("notexist")) + gotTblInfo, err = is.TableInfoByName(schemaName, ast.NewCIStr("notexist")) require.Error(t, err) require.Nil(t, gotTblInfo) @@ -113,11 +113,11 @@ func TestV2Basic(t *testing.T) { require.Equal(t, 1, len(tblInfos)) require.Equal(t, tables[0], tblInfos[0]) - tables, err = is.SchemaTableInfos(context.Background(), pmodel.NewCIStr("notexist")) + tables, err = is.SchemaTableInfos(context.Background(), ast.NewCIStr("notexist")) require.NoError(t, err) require.Equal(t, 0, len(tables)) - tblInfos, err = is.SchemaTableInfos(context.Background(), pmodel.NewCIStr("notexist")) + tblInfos, err = is.SchemaTableInfos(context.Background(), ast.NewCIStr("notexist")) require.NoError(t, err) require.Equal(t, 0, len(tblInfos)) @@ -127,7 +127,7 @@ func TestV2Basic(t *testing.T) { schemaNameByTableIDTests := []struct { name string tableID int64 - wantSchema pmodel.CIStr + wantSchema ast.CIStr wantOK bool }{ { @@ -139,13 +139,13 @@ func TestV2Basic(t *testing.T) { { name: "non-existent table ID", tableID: tblInfo.ID + 1, - wantSchema: pmodel.CIStr{}, + wantSchema: ast.CIStr{}, wantOK: false, }, { name: "invalid table ID (negative)", tableID: -1, - wantSchema: pmodel.CIStr{}, + wantSchema: ast.CIStr{}, wantOK: false, }, } @@ -289,8 +289,8 @@ func TestBundles(t *testing.T) { r.Store().Close() }() - schemaName := pmodel.NewCIStr("testDB") - tableName := pmodel.NewCIStr("test") + schemaName := ast.NewCIStr("testDB") + tableName := ast.NewCIStr("test") builder := NewBuilder(r, nil, NewData(), variable.SchemaCacheSize.Load() > 0) err := builder.InitWithDBInfos(nil, nil, nil, 1) require.NoError(t, err) @@ -410,8 +410,8 @@ func TestReferredFKInfo(t *testing.T) { r.Store().Close() }() - schemaName := pmodel.NewCIStr("testDB") - tableName := pmodel.NewCIStr("testTable") + schemaName := ast.NewCIStr("testDB") + tableName := ast.NewCIStr("testTable") builder := NewBuilder(r, nil, NewData(), variable.SchemaCacheSize.Load() > 0) err := builder.InitWithDBInfos(nil, nil, nil, 1) require.NoError(t, err) @@ -431,9 +431,9 @@ func TestReferredFKInfo(t *testing.T) { tblInfo := internal.MockTableInfo(t, r.Store(), tableName.O) tblInfo.ForeignKeys = []*model.FKInfo{{ ID: 1, - Name: pmodel.NewCIStr("fk_1"), - RefSchema: pmodel.NewCIStr("t1"), - RefTable: pmodel.NewCIStr("parent"), + Name: ast.NewCIStr("fk_1"), + RefSchema: ast.NewCIStr("t1"), + RefTable: ast.NewCIStr("parent"), Version: 1, }} internal.AddTable(t, r.Store(), dbInfo.ID, tblInfo) @@ -450,9 +450,9 @@ func TestReferredFKInfo(t *testing.T) { // check ReferredFKInfo after add foreign key tblInfo.ForeignKeys = append(tblInfo.ForeignKeys, &model.FKInfo{ ID: 2, - Name: pmodel.NewCIStr("fk_2"), - RefSchema: pmodel.NewCIStr("t1"), - RefTable: pmodel.NewCIStr("parent"), + Name: ast.NewCIStr("fk_2"), + RefSchema: ast.NewCIStr("t1"), + RefTable: ast.NewCIStr("parent"), Version: 1, }) internal.UpdateTable(t, r.Store(), dbInfo, tblInfo) @@ -516,8 +516,8 @@ func TestSpecialAttributeCorrectnessInSchemaChange(t *testing.T) { r.Store().Close() }() - schemaName := pmodel.NewCIStr("testDB") - tableName := pmodel.NewCIStr("testTable") + schemaName := ast.NewCIStr("testDB") + tableName := ast.NewCIStr("testTable") builder := NewBuilder(r, nil, NewData(), variable.SchemaCacheSize.Load() > 0) err := builder.InitWithDBInfos(nil, nil, nil, 1) require.NoError(t, err) @@ -551,12 +551,12 @@ func TestSpecialAttributeCorrectnessInSchemaChange(t *testing.T) { // tests partition info correctness in schema change tblInfo.Partition = &model.PartitionInfo{ Expr: "aa+1", - Columns: []pmodel.CIStr{ - pmodel.NewCIStr("aa"), + Columns: []ast.CIStr{ + ast.NewCIStr("aa"), }, Definitions: []model.PartitionDefinition{ - {ID: 1, Name: pmodel.NewCIStr("p1")}, - {ID: 2, Name: pmodel.NewCIStr("p2")}, + {ID: 1, Name: ast.NewCIStr("p1")}, + {ID: 2, Name: ast.NewCIStr("p2")}, }, Enable: true, DDLState: model.StatePublic, @@ -572,7 +572,7 @@ func TestSpecialAttributeCorrectnessInSchemaChange(t *testing.T) { // test placement policy correctness in schema change tblInfo.PlacementPolicyRef = &model.PolicyRefInfo{ ID: 1, - Name: pmodel.NewCIStr("p3"), + Name: ast.NewCIStr("p3"), } tblInfo1 = updateTableSpecialAttribute(t, dbInfo, tblInfo, builder, r, model.ActionAlterTablePlacement, 5, infoschemacontext.PlacementPolicyAttribute, true) require.Equal(t, tblInfo.PlacementPolicyRef, tblInfo1.PlacementPolicyRef) @@ -592,7 +592,7 @@ func TestSpecialAttributeCorrectnessInSchemaChange(t *testing.T) { // test table lock correctness in schema change tblInfo.Lock = &model.TableLockInfo{ - Tp: pmodel.TableLockRead, + Tp: ast.TableLockRead, State: model.TableLockStatePublic, TS: 1, } @@ -604,11 +604,11 @@ func TestSpecialAttributeCorrectnessInSchemaChange(t *testing.T) { // test foreign key correctness in schema change tblInfo.ForeignKeys = []*model.FKInfo{{ ID: 1, - Name: pmodel.NewCIStr("fk_1"), - RefSchema: pmodel.NewCIStr("t"), - RefTable: pmodel.NewCIStr("t"), - RefCols: []pmodel.CIStr{pmodel.NewCIStr("a")}, - Cols: []pmodel.CIStr{pmodel.NewCIStr("t_a")}, + Name: ast.NewCIStr("fk_1"), + RefSchema: ast.NewCIStr("t"), + RefTable: ast.NewCIStr("t"), + RefCols: []ast.CIStr{ast.NewCIStr("a")}, + Cols: []ast.CIStr{ast.NewCIStr("t_a")}, State: model.StateWriteOnly, }} tblInfo1 = updateTableSpecialAttribute(t, dbInfo, tblInfo, builder, r, model.ActionAddForeignKey, 11, infoschemacontext.ForeignKeysAttribute, true) @@ -623,8 +623,8 @@ func TestDataStructFieldsCorrectnessInSchemaChange(t *testing.T) { r.Store().Close() }() - schemaName := pmodel.NewCIStr("testDB") - tableName := pmodel.NewCIStr("testTable") + schemaName := ast.NewCIStr("testDB") + tableName := ast.NewCIStr("testTable") builder := NewBuilder(r, nil, NewData(), variable.SchemaCacheSize.Load() > 0) err := builder.InitWithDBInfos(nil, nil, nil, 1) require.NoError(t, err) @@ -667,8 +667,8 @@ func TestDataStructFieldsCorrectnessInSchemaChange(t *testing.T) { require.Equal(t, v2.Data.pid2tid.Load().Len(), 0) tblInfo.Partition = &model.PartitionInfo{ Definitions: []model.PartitionDefinition{ - {ID: 1, Name: pmodel.NewCIStr("p1")}, - {ID: 2, Name: pmodel.NewCIStr("p2")}, + {ID: 1, Name: ast.NewCIStr("p1")}, + {ID: 2, Name: ast.NewCIStr("p2")}, }, Enable: true, DDLState: model.StatePublic, diff --git a/pkg/infoschema/infoschemav2_cache_test.go b/pkg/infoschema/infoschemav2_cache_test.go index b9d655ed40dcf..14b2fb2928bb9 100644 --- a/pkg/infoschema/infoschemav2_cache_test.go +++ b/pkg/infoschema/infoschemav2_cache_test.go @@ -22,7 +22,7 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta/autoid" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/types" @@ -185,7 +185,7 @@ func (tc *testCase) runAction(action testAction) { func (tc *testCase) createSchema(args []any) { dbInfo := &model.DBInfo{ ID: int64(args[0].(int)), - Name: pmodel.NewCIStr(args[1].(string)), + Name: ast.NewCIStr(args[1].(string)), State: model.StatePublic, } dbInfo.Deprecated.Tables = []*model.TableInfo{} @@ -197,7 +197,7 @@ func (tc *testCase) createSchema(args []any) { func (tc *testCase) createTable(args []any) { colInfo := &model.ColumnInfo{ ID: 1, - Name: pmodel.NewCIStr("a"), + Name: ast.NewCIStr("a"), Offset: 0, FieldType: *types.NewFieldType(mysql.TypeLonglong), State: model.StatePublic, @@ -206,7 +206,7 @@ func (tc *testCase) createTable(args []any) { tblInfo := &model.TableInfo{ DBID: int64(args[0].(int)), ID: int64(args[2].(int)), - Name: pmodel.NewCIStr(args[3].(string)), + Name: ast.NewCIStr(args[3].(string)), Columns: []*model.ColumnInfo{colInfo}, State: model.StatePublic, } @@ -217,7 +217,7 @@ func (tc *testCase) createTable(args []any) { tc.statusHook.On("onEvict").Return().Once() } } - tc.is.Data.add(tableItem{pmodel.NewCIStr(args[1].(string)), tblInfo.DBID, tblInfo.Name, tblInfo.ID, int64(args[4].(int)), false}, internal.MockTable(tc.t, tc.r.Store(), tblInfo)) + tc.is.Data.add(tableItem{ast.NewCIStr(args[1].(string)), tblInfo.DBID, tblInfo.Name, tblInfo.ID, int64(args[4].(int)), false}, internal.MockTable(tc.t, tc.r.Store(), tblInfo)) internal.AddTable(tc.t, tc.r.Store(), tblInfo.DBID, tblInfo) } @@ -260,5 +260,5 @@ func (tc *testCase) tableByName(args []any) { } } - tc.is.TableByName(tc.ctx, pmodel.NewCIStr(args[0].(string)), pmodel.NewCIStr(args[1].(string))) + tc.is.TableByName(tc.ctx, ast.NewCIStr(args[0].(string)), ast.NewCIStr(args[1].(string))) } diff --git a/pkg/infoschema/interface.go b/pkg/infoschema/interface.go index 4a4fb9a265bee..8a52259dd53e5 100644 --- a/pkg/infoschema/interface.go +++ b/pkg/infoschema/interface.go @@ -19,7 +19,7 @@ import ( "github.com/pingcap/tidb/pkg/infoschema/context" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/table" ) @@ -28,9 +28,9 @@ import ( // InfoSchema is read-only, and the returned value is a copy. type InfoSchema interface { context.MetaOnlyInfoSchema - TableByName(ctx stdctx.Context, schema, table pmodel.CIStr) (table.Table, error) + TableByName(ctx stdctx.Context, schema, table ast.CIStr) (table.Table, error) TableByID(ctx stdctx.Context, id int64) (table.Table, bool) - SchemaNameByTableID(tableID int64) (pmodel.CIStr, bool) + SchemaNameByTableID(tableID int64) (ast.CIStr, bool) FindTableByPartitionID(partitionID int64) (table.Table, *model.DBInfo, *model.PartitionDefinition) base() *infoSchema } diff --git a/pkg/infoschema/internal/BUILD.bazel b/pkg/infoschema/internal/BUILD.bazel index 978699d88c52d..370ec0c55481c 100644 --- a/pkg/infoschema/internal/BUILD.bazel +++ b/pkg/infoschema/internal/BUILD.bazel @@ -13,7 +13,7 @@ go_library( "//pkg/meta", "//pkg/meta/autoid", "//pkg/meta/model", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/parser/mysql", "//pkg/store/mockstore", "//pkg/table", diff --git a/pkg/infoschema/internal/testkit.go b/pkg/infoschema/internal/testkit.go index 9d0e9058aa5ae..eb213604c1213 100644 --- a/pkg/infoschema/internal/testkit.go +++ b/pkg/infoschema/internal/testkit.go @@ -24,7 +24,7 @@ import ( "github.com/pingcap/tidb/pkg/meta" "github.com/pingcap/tidb/pkg/meta/autoid" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/store/mockstore" "github.com/pingcap/tidb/pkg/table" @@ -155,7 +155,7 @@ func MockDBInfo(t testing.TB, store kv.Storage, DBName string) *model.DBInfo { require.NoError(t, err) dbInfo := &model.DBInfo{ ID: id, - Name: pmodel.NewCIStr(DBName), + Name: ast.NewCIStr(DBName), State: model.StatePublic, } dbInfo.Deprecated.Tables = []*model.TableInfo{} @@ -168,7 +168,7 @@ func MockTableInfo(t testing.TB, store kv.Storage, tblName string) *model.TableI require.NoError(t, err) colInfo := &model.ColumnInfo{ ID: colID, - Name: pmodel.NewCIStr("a"), + Name: ast.NewCIStr("a"), Offset: 0, FieldType: *types.NewFieldType(mysql.TypeLonglong), State: model.StatePublic, @@ -179,7 +179,7 @@ func MockTableInfo(t testing.TB, store kv.Storage, tblName string) *model.TableI return &model.TableInfo{ ID: tblID, - Name: pmodel.NewCIStr(tblName), + Name: ast.NewCIStr(tblName), Columns: []*model.ColumnInfo{colInfo}, State: model.StatePublic, } @@ -198,7 +198,7 @@ func MockResourceGroupInfo(t *testing.T, store kv.Storage, groupName string) *mo require.NoError(t, err) return &model.ResourceGroupInfo{ ID: id, - Name: pmodel.NewCIStr(groupName), + Name: ast.NewCIStr(groupName), } } @@ -208,7 +208,7 @@ func MockPolicyInfo(t *testing.T, store kv.Storage, policyName string) *model.Po require.NoError(t, err) return &model.PolicyInfo{ ID: id, - Name: pmodel.NewCIStr(policyName), + Name: ast.NewCIStr(policyName), } } @@ -218,7 +218,7 @@ func MockPolicyRefInfo(t *testing.T, store kv.Storage, policyName string) *model require.NoError(t, err) return &model.PolicyRefInfo{ ID: id, - Name: pmodel.NewCIStr(policyName), + Name: ast.NewCIStr(policyName), } } diff --git a/pkg/infoschema/metrics_schema.go b/pkg/infoschema/metrics_schema.go index 4c3f8b99e0d6f..870d681343b0b 100644 --- a/pkg/infoschema/metrics_schema.go +++ b/pkg/infoschema/metrics_schema.go @@ -25,7 +25,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/meta/autoid" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/util" @@ -64,7 +64,7 @@ func init() { dbInfo := &model.DBInfo{ ID: dbID, - Name: pmodel.NewCIStr(util.MetricSchemaName.O), + Name: ast.NewCIStr(util.MetricSchemaName.O), Charset: mysql.DefaultCharset, Collate: mysql.DefaultCollationName, } diff --git a/pkg/infoschema/tables.go b/pkg/infoschema/tables.go index 6119fe96b7602..2b923f51250c1 100644 --- a/pkg/infoschema/tables.go +++ b/pkg/infoschema/tables.go @@ -41,9 +41,9 @@ import ( "github.com/pingcap/tidb/pkg/kv" "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/parser/auth" "github.com/pingcap/tidb/pkg/parser/charset" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/privilege" @@ -384,7 +384,7 @@ func buildColumnInfo(col columnInfo) *model.ColumnInfo { fieldType.SetFlag(col.flag) fieldType.SetElems(col.enumElems) return &model.ColumnInfo{ - Name: pmodel.NewCIStr(col.name), + Name: ast.NewCIStr(col.name), FieldType: fieldType, State: model.StatePublic, DefaultValue: col.deflt, @@ -396,7 +396,7 @@ func buildTableMeta(tableName string, cs []columnInfo) *model.TableInfo { cols := make([]*model.ColumnInfo, 0, len(cs)) primaryIndices := make([]*model.IndexInfo, 0, 1) tblInfo := &model.TableInfo{ - Name: pmodel.NewCIStr(tableName), + Name: ast.NewCIStr(tableName), State: model.StatePublic, Charset: mysql.DefaultCharset, Collate: mysql.DefaultCollationName, @@ -411,12 +411,12 @@ func buildTableMeta(tableName string, cs []columnInfo) *model.TableInfo { tblInfo.IsCommonHandle = true tblInfo.CommonHandleVersion = 1 index := &model.IndexInfo{ - Name: pmodel.NewCIStr("primary"), + Name: ast.NewCIStr("primary"), State: model.StatePublic, Primary: true, Unique: true, Columns: []*model.IndexColumn{ - {Name: pmodel.NewCIStr(c.name), Offset: offset, Length: types.UnspecifiedLength}}, + {Name: ast.NewCIStr(c.name), Offset: offset, Length: types.UnspecifiedLength}}, } primaryIndices = append(primaryIndices, index) tblInfo.Indices = primaryIndices @@ -1826,7 +1826,7 @@ var tablePlanCache = []columnInfo{ // // The returned nil indicates that sharding information is not suitable for the table(for example, when the table is a View). // This function is exported for unit test. -func GetShardingInfo(dbInfo pmodel.CIStr, tableInfo *model.TableInfo) any { +func GetShardingInfo(dbInfo ast.CIStr, tableInfo *model.TableInfo) any { if tableInfo == nil || tableInfo.IsView() || util.IsMemOrSysDB(dbInfo.L) { return nil } diff --git a/pkg/infoschema/test/clustertablestest/BUILD.bazel b/pkg/infoschema/test/clustertablestest/BUILD.bazel index d61f60d5edf02..d3cebd8410c2b 100644 --- a/pkg/infoschema/test/clustertablestest/BUILD.bazel +++ b/pkg/infoschema/test/clustertablestest/BUILD.bazel @@ -22,8 +22,8 @@ go_test( "//pkg/meta/autoid", "//pkg/meta/model", "//pkg/parser", + "//pkg/parser/ast", "//pkg/parser/auth", - "//pkg/parser/model", "//pkg/parser/mysql", "//pkg/parser/terror", "//pkg/planner/core", diff --git a/pkg/infoschema/test/clustertablestest/cluster_tables_test.go b/pkg/infoschema/test/clustertablestest/cluster_tables_test.go index 52c1e31aaebb2..f3a55b41af35a 100644 --- a/pkg/infoschema/test/clustertablestest/cluster_tables_test.go +++ b/pkg/infoschema/test/clustertablestest/cluster_tables_test.go @@ -40,8 +40,8 @@ import ( "github.com/pingcap/tidb/pkg/infoschema/internal" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/auth" - "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/privilege/privileges" "github.com/pingcap/tidb/pkg/server" @@ -1901,7 +1901,7 @@ func TestMDLViewIDConflict(t *testing.T) { tk.MustExec("use test") tk.MustExec("create table t(a int);") - tbl, err := s.dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := s.dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tk.MustExec("insert into t values (1)") @@ -1912,7 +1912,7 @@ func TestMDLViewIDConflict(t *testing.T) { bigTableName = fmt.Sprintf("t%d", i) tk.MustExec(fmt.Sprintf("create table %s(a int);", bigTableName)) - tbl, err := s.dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr(bigTableName)) + tbl, err := s.dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr(bigTableName)) require.NoError(t, err) require.LessOrEqual(t, tbl.Meta().ID, bigID) diff --git a/pkg/infoschema/test/clustertablestest/tables_test.go b/pkg/infoschema/test/clustertablestest/tables_test.go index 14eca9a4cf085..6a83a1645df26 100644 --- a/pkg/infoschema/test/clustertablestest/tables_test.go +++ b/pkg/infoschema/test/clustertablestest/tables_test.go @@ -35,8 +35,8 @@ import ( "github.com/pingcap/tidb/pkg/meta/autoid" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/auth" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" plannercore "github.com/pingcap/tidb/pkg/planner/core" @@ -343,7 +343,7 @@ func TestTableRowIDShardingInfo(t *testing.T) { testFunc := func(dbName string, expectInfo any) { tableInfo := model.TableInfo{} - info := infoschema.GetShardingInfo(pmodel.NewCIStr(dbName), &tableInfo) + info := infoschema.GetShardingInfo(ast.NewCIStr(dbName), &tableInfo) require.Equal(t, expectInfo, info) } @@ -614,11 +614,11 @@ func TestReloadDropDatabase(t *testing.T) { tk.MustExec("create table t2 (a int)") tk.MustExec("create table t3 (a int)") is := domain.GetDomain(tk.Session()).InfoSchema() - t2, err := is.TableByName(context.Background(), pmodel.NewCIStr("test_dbs"), pmodel.NewCIStr("t2")) + t2, err := is.TableByName(context.Background(), ast.NewCIStr("test_dbs"), ast.NewCIStr("t2")) require.NoError(t, err) tk.MustExec("drop database test_dbs") is = domain.GetDomain(tk.Session()).InfoSchema() - _, err = is.TableByName(context.Background(), pmodel.NewCIStr("test_dbs"), pmodel.NewCIStr("t2")) + _, err = is.TableByName(context.Background(), ast.NewCIStr("test_dbs"), ast.NewCIStr("t2")) require.True(t, terror.ErrorEqual(infoschema.ErrTableNotExists, err)) _, ok := is.TableByID(context.Background(), t2.Meta().ID) require.False(t, ok) @@ -636,11 +636,11 @@ func TestSystemSchemaID(t *testing.T) { func checkSystemSchemaTableID(t *testing.T, dom *domain.Domain, dbName string, dbID, start, end int64, uniqueIDMap map[int64]string) { is := dom.InfoSchema() require.NotNil(t, is) - db, ok := is.SchemaByName(pmodel.NewCIStr(dbName)) + db, ok := is.SchemaByName(ast.NewCIStr(dbName)) require.True(t, ok) require.Equal(t, dbID, db.ID) // Test for information_schema table id. - tables, err := is.SchemaTableInfos(context.Background(), pmodel.NewCIStr(dbName)) + tables, err := is.SchemaTableInfos(context.Background(), ast.NewCIStr(dbName)) require.NoError(t, err) require.Greater(t, len(tables), 0) for _, tbl := range tables { @@ -673,7 +673,7 @@ func TestSelectHiddenColumn(t *testing.T) { tk.MustExec("USE test_hidden;") tk.MustExec("CREATE TABLE hidden (a int , b int, c int);") tk.MustQuery("select count(*) from INFORMATION_SCHEMA.COLUMNS where table_name = 'hidden'").Check(testkit.Rows("3")) - tb, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test_hidden"), pmodel.NewCIStr("hidden")) + tb, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test_hidden"), ast.NewCIStr("hidden")) require.NoError(t, err) tbInfo := tb.Meta() colInfo := tbInfo.Columns diff --git a/pkg/infoschema/test/infoschemav2test/BUILD.bazel b/pkg/infoschema/test/infoschemav2test/BUILD.bazel index 584c70f93a791..43768aed9468a 100644 --- a/pkg/infoschema/test/infoschemav2test/BUILD.bazel +++ b/pkg/infoschema/test/infoschemav2test/BUILD.bazel @@ -15,8 +15,8 @@ go_test( "//pkg/infoschema", "//pkg/infoschema/context", "//pkg/meta/autoid", + "//pkg/parser/ast", "//pkg/parser/auth", - "//pkg/parser/model", "//pkg/sessionctx/variable", "//pkg/table", "//pkg/testkit", diff --git a/pkg/infoschema/test/infoschemav2test/v2_test.go b/pkg/infoschema/test/infoschemav2test/v2_test.go index d0e2f315e1cbb..244a65853df0b 100644 --- a/pkg/infoschema/test/infoschemav2test/v2_test.go +++ b/pkg/infoschema/test/infoschemav2test/v2_test.go @@ -30,8 +30,8 @@ import ( "github.com/pingcap/tidb/pkg/infoschema" infoschemacontext "github.com/pingcap/tidb/pkg/infoschema/context" "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/parser/model" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/testkit" @@ -92,7 +92,7 @@ func checkPIDNotExist(t *testing.T, dom *domain.Domain, pid int64) { func getPIDForP3(t *testing.T, dom *domain.Domain) (int64, table.Table) { is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("pt")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("pt")) require.NoError(t, err) pi := tbl.Meta().GetPartitionInfo() pid := pi.GetPartitionIDByName("p3") @@ -150,12 +150,12 @@ PARTITION p5 VALUES LESS THAN (1980))`) // Test FindTableByPartitionID after exchange partition. tk.MustExec("create table nt (id int)") is = dom.InfoSchema() - ntbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("nt")) + ntbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("nt")) require.NoError(t, err) tk.MustExec("alter table pt exchange partition p3 with table nt") is = dom.InfoSchema() - ptbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("pt")) + ptbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("pt")) require.NoError(t, err) pi := ptbl.Meta().GetPartitionInfo() pid = pi.GetPartitionIDByName("p3") @@ -281,7 +281,7 @@ func TestUnrelatedDDLTriggerReload(t *testing.T) { is := dom.InfoSchema() ok, v2 := infoschema.IsV2(is) require.True(t, ok) - v2.EvictTable(model.NewCIStr("test"), model.NewCIStr("t1")) + v2.EvictTable(ast.NewCIStr("test"), ast.NewCIStr("t1")) tk.MustExec("create table t2 (id int)") @@ -312,7 +312,7 @@ func TestTrace(t *testing.T) { require.True(t, ok) // Evict the table cache and check the trace information can catch this calling. - raw.EvictTable(model.NewCIStr("test"), model.NewCIStr("t_trace")) + raw.EvictTable(ast.NewCIStr("test"), ast.NewCIStr("t_trace")) tk.MustQuery("trace select * from information_schema.tables where table_schema='test' and table_name='t_trace'").CheckContain("infoschema.loadTableInfo") } @@ -329,7 +329,7 @@ func TestCachedTable(t *testing.T) { require.True(t, ok) // Cover a case that after cached table evict and load, table.Table goes wrong. - raw.EvictTable(model.NewCIStr("test"), model.NewCIStr("t_cache")) + raw.EvictTable(ast.NewCIStr("test"), ast.NewCIStr("t_cache")) tk.MustExec("insert into t_cache values (2)") // no panic here tk.MustQuery("select * from t_cache").Check(testkit.Rows("1", "2")) } @@ -343,8 +343,8 @@ func BenchmarkTableByName(t *testing.B) { tk.MustExec(fmt.Sprintf("create table t%d (id int)", i)) } is := dom.InfoSchema() - db := model.NewCIStr("test") - tbl := model.NewCIStr("t123") + db := ast.NewCIStr("test") + tbl := ast.NewCIStr("t123") t.ResetTimer() for i := 0; i < t.N; i++ { _, err := is.TableByName(context.Background(), db, tbl) @@ -482,7 +482,7 @@ func TestSchemaSimpleTableInfos(t *testing.T) { is := tk.Session().GetInfoSchema() // Cover special schema - tblInfos, err := is.SchemaSimpleTableInfos(context.Background(), model.NewCIStr("INFORMATION_SCHEMA")) + tblInfos, err := is.SchemaSimpleTableInfos(context.Background(), ast.NewCIStr("INFORMATION_SCHEMA")) require.NoError(t, err) res := make([]string, 0, len(tblInfos)) for _, tbl := range tblInfos { @@ -493,7 +493,7 @@ func TestSchemaSimpleTableInfos(t *testing.T) { Sort().Check(testkit.Rows(res...)) // Cover normal schema - tblInfos, err = is.SchemaSimpleTableInfos(context.Background(), model.NewCIStr("simple")) + tblInfos, err = is.SchemaSimpleTableInfos(context.Background(), ast.NewCIStr("simple")) require.NoError(t, err) require.Len(t, tblInfos, 1) require.Equal(t, tblInfos[0].Name.L, "t1") @@ -501,7 +501,7 @@ func TestSchemaSimpleTableInfos(t *testing.T) { // Cover snapshot infoschema tk.MustExec(fmt.Sprintf(`set @@tidb_snapshot="%s"`, time1.Format("2006-1-2 15:04:05.000"))) is = tk.Session().GetInfoSchema() - tblInfos, err = is.SchemaSimpleTableInfos(context.Background(), model.NewCIStr("simple")) + tblInfos, err = is.SchemaSimpleTableInfos(context.Background(), ast.NewCIStr("simple")) require.NoError(t, err) require.Len(t, tblInfos, 2) require.Equal(t, tblInfos[0].Name.L, "t2") diff --git a/pkg/lightning/backend/kv/BUILD.bazel b/pkg/lightning/backend/kv/BUILD.bazel index 6b66a7289c486..97925831f94f3 100644 --- a/pkg/lightning/backend/kv/BUILD.bazel +++ b/pkg/lightning/backend/kv/BUILD.bazel @@ -78,7 +78,6 @@ go_test( "//pkg/meta/model", "//pkg/parser", "//pkg/parser/ast", - "//pkg/parser/model", "//pkg/parser/mysql", "//pkg/planner/core", "//pkg/sessionctx/stmtctx", diff --git a/pkg/lightning/backend/kv/base_test.go b/pkg/lightning/backend/kv/base_test.go index 3fba4ca6cc7cc..55bd522c0ccc5 100644 --- a/pkg/lightning/backend/kv/base_test.go +++ b/pkg/lightning/backend/kv/base_test.go @@ -23,7 +23,7 @@ import ( "github.com/pingcap/tidb/pkg/lightning/backend/encode" "github.com/pingcap/tidb/pkg/lightning/log" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/table/tables" @@ -37,7 +37,7 @@ func TestLogKVConvertFailed(t *testing.T) { err := log.InitLogger(logCfg, "info") require.NoError(t, err) - modelName := pmodel.NewCIStr("c1") + modelName := ast.NewCIStr("c1") modelState := model.StatePublic modelFieldType := *types.NewFieldType(mysql.TypeTiny) c1 := &model.ColumnInfo{ID: 1, Name: modelName, State: modelState, Offset: 0, FieldType: modelFieldType} diff --git a/pkg/lightning/backend/kv/sql2kv_test.go b/pkg/lightning/backend/kv/sql2kv_test.go index a21e7198fe4ad..0a774bd24aa85 100644 --- a/pkg/lightning/backend/kv/sql2kv_test.go +++ b/pkg/lightning/backend/kv/sql2kv_test.go @@ -31,7 +31,6 @@ import ( "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser" "github.com/pingcap/tidb/pkg/parser/ast" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" _ "github.com/pingcap/tidb/pkg/planner/core" // to setup expression.EvalAstExpr. Otherwise we cannot parse the default value "github.com/pingcap/tidb/pkg/table" @@ -77,7 +76,7 @@ func (mockTable) AddRecord(ctx table.MutateContext, txn kv.Transaction, r []type } func TestEncode(t *testing.T) { - c1 := &model.ColumnInfo{ID: 1, Name: pmodel.NewCIStr("c1"), State: model.StatePublic, Offset: 0, FieldType: *types.NewFieldType(mysql.TypeTiny)} + c1 := &model.ColumnInfo{ID: 1, Name: ast.NewCIStr("c1"), State: model.StatePublic, Offset: 0, FieldType: *types.NewFieldType(mysql.TypeTiny)} cols := []*model.ColumnInfo{c1} tblInfo := &model.TableInfo{ID: 1, Columns: cols, PKIsHandle: false, State: model.StatePublic} tbl, err := tables.TableFromMeta(lkv.NewPanickingAllocators(tblInfo.SepAutoInc()), tblInfo) @@ -160,7 +159,7 @@ func TestEncode(t *testing.T) { } func TestDecode(t *testing.T) { - c1 := &model.ColumnInfo{ID: 1, Name: pmodel.NewCIStr("c1"), State: model.StatePublic, Offset: 0, FieldType: *types.NewFieldType(mysql.TypeTiny)} + c1 := &model.ColumnInfo{ID: 1, Name: ast.NewCIStr("c1"), State: model.StatePublic, Offset: 0, FieldType: *types.NewFieldType(mysql.TypeTiny)} cols := []*model.ColumnInfo{c1} tblInfo := &model.TableInfo{ID: 1, Columns: cols, PKIsHandle: false, State: model.StatePublic} tbl, err := tables.TableFromMeta(lkv.NewPanickingAllocators(tblInfo.SepAutoInc()), tblInfo) @@ -201,7 +200,7 @@ func TestDecodeIndex(t *testing.T) { Indices: []*model.IndexInfo{ { ID: 2, - Name: pmodel.NewCIStr("test"), + Name: ast.NewCIStr("test"), Columns: []*model.IndexColumn{ {Offset: 0}, {Offset: 1}, @@ -211,8 +210,8 @@ func TestDecodeIndex(t *testing.T) { }, }, Columns: []*model.ColumnInfo{ - {ID: 1, Name: pmodel.NewCIStr("c1"), State: model.StatePublic, Offset: 0, FieldType: *types.NewFieldType(mysql.TypeInt24)}, - {ID: 2, Name: pmodel.NewCIStr("c2"), State: model.StatePublic, Offset: 1, FieldType: *types.NewFieldType(mysql.TypeString)}, + {ID: 1, Name: ast.NewCIStr("c1"), State: model.StatePublic, Offset: 0, FieldType: *types.NewFieldType(mysql.TypeInt24)}, + {ID: 2, Name: ast.NewCIStr("c2"), State: model.StatePublic, Offset: 1, FieldType: *types.NewFieldType(mysql.TypeString)}, }, State: model.StatePublic, PKIsHandle: false, @@ -259,7 +258,7 @@ func TestDecodeIndex(t *testing.T) { func TestEncodeRowFormatV2(t *testing.T) { // Test encoding in row format v2, as described in . - c1 := &model.ColumnInfo{ID: 1, Name: pmodel.NewCIStr("c1"), State: model.StatePublic, Offset: 0, FieldType: *types.NewFieldType(mysql.TypeTiny)} + c1 := &model.ColumnInfo{ID: 1, Name: ast.NewCIStr("c1"), State: model.StatePublic, Offset: 0, FieldType: *types.NewFieldType(mysql.TypeTiny)} cols := []*model.ColumnInfo{c1} tblInfo := &model.TableInfo{ID: 1, Columns: cols, PKIsHandle: false, State: model.StatePublic} tbl, err := tables.TableFromMeta(lkv.NewPanickingAllocators(tblInfo.SepAutoInc()), tblInfo) @@ -304,7 +303,7 @@ func TestEncodeTimestamp(t *testing.T) { ty.AddFlag(mysql.NotNullFlag) c1 := &model.ColumnInfo{ ID: 1, - Name: pmodel.NewCIStr("c1"), + Name: ast.NewCIStr("c1"), State: model.StatePublic, Offset: 0, FieldType: ty, diff --git a/pkg/lightning/backend/local/BUILD.bazel b/pkg/lightning/backend/local/BUILD.bazel index 2cb9dbc4bbeeb..dc2e5a19518df 100644 --- a/pkg/lightning/backend/local/BUILD.bazel +++ b/pkg/lightning/backend/local/BUILD.bazel @@ -148,7 +148,6 @@ go_test( "//pkg/meta/model", "//pkg/parser", "//pkg/parser/ast", - "//pkg/parser/model", "//pkg/parser/mysql", "//pkg/sessionctx/stmtctx", "//pkg/store/pdtypes", diff --git a/pkg/lightning/backend/local/checksum_test.go b/pkg/lightning/backend/local/checksum_test.go index fb24a7e630f56..399866bf0fa2f 100644 --- a/pkg/lightning/backend/local/checksum_test.go +++ b/pkg/lightning/backend/local/checksum_test.go @@ -30,7 +30,7 @@ import ( "github.com/pingcap/tidb/pkg/kv" . "github.com/pingcap/tidb/pkg/lightning/checkpoints" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" pmysql "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util" @@ -172,11 +172,11 @@ func TestDoChecksumWithTikv(t *testing.T) { tableInfo := &model.TableInfo{ ID: 999, - Name: pmodel.NewCIStr("t1"), + Name: ast.NewCIStr("t1"), Columns: []*model.ColumnInfo{ { ID: 1, - Name: pmodel.NewCIStr("c1"), + Name: ast.NewCIStr("c1"), FieldType: *fieldType, }, }, diff --git a/pkg/lightning/backend/tidb/BUILD.bazel b/pkg/lightning/backend/tidb/BUILD.bazel index 1416344efab06..8a39bc88e0a56 100644 --- a/pkg/lightning/backend/tidb/BUILD.bazel +++ b/pkg/lightning/backend/tidb/BUILD.bazel @@ -16,7 +16,7 @@ go_library( "//pkg/lightning/log", "//pkg/lightning/verification", "//pkg/meta/model", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/parser/mysql", "//pkg/table", "//pkg/types", @@ -52,8 +52,8 @@ go_test( "//pkg/lightning/log", "//pkg/lightning/verification", "//pkg/meta/model", + "//pkg/parser/ast", "//pkg/parser/charset", - "//pkg/parser/model", "//pkg/parser/mysql", "//pkg/table", "//pkg/table/tables", diff --git a/pkg/lightning/backend/tidb/tidb.go b/pkg/lightning/backend/tidb/tidb.go index 535dc2cea1961..72ac4655f4714 100644 --- a/pkg/lightning/backend/tidb/tidb.go +++ b/pkg/lightning/backend/tidb/tidb.go @@ -38,7 +38,7 @@ import ( "github.com/pingcap/tidb/pkg/lightning/log" "github.com/pingcap/tidb/pkg/lightning/verification" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/types" @@ -162,7 +162,7 @@ func (b *targetInfoGetter) FetchRemoteDBModels(ctx context.Context) ([]*model.DB return e } dbInfo := &model.DBInfo{ - Name: pmodel.NewCIStr(dbName), + Name: ast.NewCIStr(dbName), } results = append(results, dbInfo) } @@ -233,7 +233,7 @@ func (b *targetInfoGetter) FetchRemoteTableModels( if tableName != curTableName { tableIdx++ curTable = &model.TableInfo{ - Name: pmodel.NewCIStr(tableName), + Name: ast.NewCIStr(tableName), State: model.StatePublic, PKIsHandle: true, } @@ -254,7 +254,7 @@ func (b *targetInfoGetter) FetchRemoteTableModels( ft := types.FieldType{} ft.SetFlag(flag) curTable.Columns = append(curTable.Columns, &model.ColumnInfo{ - Name: pmodel.NewCIStr(columnName), + Name: ast.NewCIStr(columnName), Offset: curColOffset, State: model.StatePublic, FieldType: ft, diff --git a/pkg/lightning/backend/tidb/tidb_test.go b/pkg/lightning/backend/tidb/tidb_test.go index 2170a65df00c8..0340313906ffd 100644 --- a/pkg/lightning/backend/tidb/tidb_test.go +++ b/pkg/lightning/backend/tidb/tidb_test.go @@ -35,8 +35,8 @@ import ( "github.com/pingcap/tidb/pkg/lightning/log" "github.com/pingcap/tidb/pkg/lightning/verification" "github.com/pingcap/tidb/pkg/meta/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/charset" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/table/tables" @@ -65,7 +65,7 @@ func createMysqlSuite(t *testing.T) *mysqlSuite { } cols := make([]*model.ColumnInfo, 0, len(tys)) for i, ty := range tys { - col := &model.ColumnInfo{ID: int64(i + 1), Name: pmodel.NewCIStr(fmt.Sprintf("c%d", i)), State: model.StatePublic, Offset: i, FieldType: *types.NewFieldType(ty)} + col := &model.ColumnInfo{ID: int64(i + 1), Name: ast.NewCIStr(fmt.Sprintf("c%d", i)), State: model.StatePublic, Offset: i, FieldType: *types.NewFieldType(ty)} cols = append(cols, col) } tblInfo := &model.TableInfo{ID: 1, Columns: cols, PKIsHandle: false, State: model.StatePublic} @@ -289,10 +289,10 @@ func testStrictMode(t *testing.T) { defer s.TearDownTest(t) ft := *types.NewFieldType(mysql.TypeVarchar) ft.SetCharset(charset.CharsetUTF8MB4) - col0 := &model.ColumnInfo{ID: 1, Name: pmodel.NewCIStr("s0"), State: model.StatePublic, Offset: 0, FieldType: ft} + col0 := &model.ColumnInfo{ID: 1, Name: ast.NewCIStr("s0"), State: model.StatePublic, Offset: 0, FieldType: ft} ft = *types.NewFieldType(mysql.TypeString) ft.SetCharset(charset.CharsetASCII) - col1 := &model.ColumnInfo{ID: 2, Name: pmodel.NewCIStr("s1"), State: model.StatePublic, Offset: 1, FieldType: ft} + col1 := &model.ColumnInfo{ID: 2, Name: ast.NewCIStr("s1"), State: model.StatePublic, Offset: 1, FieldType: ft} tblInfo := &model.TableInfo{ID: 1, Columns: []*model.ColumnInfo{col0, col1}, PKIsHandle: false, State: model.StatePublic} tbl, err := tables.TableFromMeta(kv.NewPanickingAllocators(tblInfo.SepAutoInc()), tblInfo) require.NoError(t, err) @@ -354,12 +354,12 @@ func TestFetchRemoteTableModels_4_0(t *testing.T) { ft.SetFlag(mysql.AutoIncrementFlag | mysql.UnsignedFlag) require.Equal(t, map[string]*model.TableInfo{ "t": { - Name: pmodel.NewCIStr("t"), + Name: ast.NewCIStr("t"), State: model.StatePublic, PKIsHandle: true, Columns: []*model.ColumnInfo{ { - Name: pmodel.NewCIStr("id"), + Name: ast.NewCIStr("id"), Offset: 0, State: model.StatePublic, FieldType: ft, @@ -389,12 +389,12 @@ func TestFetchRemoteTableModels_4_x_auto_increment(t *testing.T) { ft.SetFlag(mysql.AutoIncrementFlag) require.Equal(t, map[string]*model.TableInfo{ "t": { - Name: pmodel.NewCIStr("t"), + Name: ast.NewCIStr("t"), State: model.StatePublic, PKIsHandle: true, Columns: []*model.ColumnInfo{ { - Name: pmodel.NewCIStr("id"), + Name: ast.NewCIStr("id"), Offset: 0, State: model.StatePublic, FieldType: ft, @@ -424,13 +424,13 @@ func TestFetchRemoteTableModels_4_x_auto_random(t *testing.T) { ft.SetFlag(mysql.PriKeyFlag) require.Equal(t, map[string]*model.TableInfo{ "t": { - Name: pmodel.NewCIStr("t"), + Name: ast.NewCIStr("t"), State: model.StatePublic, PKIsHandle: true, AutoRandomBits: 1, Columns: []*model.ColumnInfo{ { - Name: pmodel.NewCIStr("id"), + Name: ast.NewCIStr("id"), Offset: 0, State: model.StatePublic, FieldType: ft, @@ -471,18 +471,18 @@ func TestFetchRemoteTableModelsDropTableHalfway(t *testing.T) { ft.SetFlag(mysql.AutoIncrementFlag) require.Equal(t, map[string]*model.TableInfo{ "tbl01": { - Name: pmodel.NewCIStr("tbl01"), + Name: ast.NewCIStr("tbl01"), State: model.StatePublic, PKIsHandle: true, Columns: []*model.ColumnInfo{ { - Name: pmodel.NewCIStr("id"), + Name: ast.NewCIStr("id"), Offset: 0, State: model.StatePublic, FieldType: ft, }, { - Name: pmodel.NewCIStr("val"), + Name: ast.NewCIStr("val"), Offset: 1, State: model.StatePublic, }, diff --git a/pkg/lightning/checkpoints/BUILD.bazel b/pkg/lightning/checkpoints/BUILD.bazel index cf08c81f1b592..5a7f99a76fb9b 100644 --- a/pkg/lightning/checkpoints/BUILD.bazel +++ b/pkg/lightning/checkpoints/BUILD.bazel @@ -44,7 +44,7 @@ go_test( "//pkg/lightning/mydump", "//pkg/lightning/verification", "//pkg/meta/model", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/testkit/testsetup", "@com_github_data_dog_go_sqlmock//:go-sqlmock", "@com_github_pingcap_errors//:errors", diff --git a/pkg/lightning/checkpoints/checkpoints_file_test.go b/pkg/lightning/checkpoints/checkpoints_file_test.go index 17d9fceb1e122..c2eb2628cd78b 100644 --- a/pkg/lightning/checkpoints/checkpoints_file_test.go +++ b/pkg/lightning/checkpoints/checkpoints_file_test.go @@ -26,7 +26,7 @@ import ( "github.com/pingcap/tidb/pkg/lightning/mydump" "github.com/pingcap/tidb/pkg/lightning/verification" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/stretchr/testify/require" ) @@ -65,7 +65,7 @@ func newFileCheckpointsDB(t *testing.T, addIndexBySQL bool) *checkpoints.FileChe "t3": { Name: "t3", Desired: &model.TableInfo{ - Name: pmodel.NewCIStr("t3"), + Name: ast.NewCIStr("t3"), }, }, }, @@ -223,7 +223,7 @@ func TestGet(t *testing.T) { }, }, TableInfo: &model.TableInfo{ - Name: pmodel.NewCIStr("t3"), + Name: ast.NewCIStr("t3"), }, } diff --git a/pkg/lightning/checkpoints/checkpoints_sql_test.go b/pkg/lightning/checkpoints/checkpoints_sql_test.go index 20420cbc2a50c..4d3f60247cc18 100644 --- a/pkg/lightning/checkpoints/checkpoints_sql_test.go +++ b/pkg/lightning/checkpoints/checkpoints_sql_test.go @@ -29,7 +29,7 @@ import ( "github.com/pingcap/tidb/pkg/lightning/mydump" "github.com/pingcap/tidb/pkg/lightning/verification" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/stretchr/testify/require" ) @@ -102,7 +102,7 @@ func TestNormalOperations(t *testing.T) { Name: "t2", ID: 2, Desired: &model.TableInfo{ - Name: pmodel.NewCIStr("t2"), + Name: ast.NewCIStr("t2"), }, }, }, @@ -193,15 +193,15 @@ func TestNormalOperationsWithAddIndexBySQL(t *testing.T) { // 2. initialize with checkpoint data. t1Info, err := json.Marshal(&model.TableInfo{ - Name: pmodel.NewCIStr("t1"), + Name: ast.NewCIStr("t1"), }) require.NoError(t, err) t2Info, err := json.Marshal(&model.TableInfo{ - Name: pmodel.NewCIStr("t2"), + Name: ast.NewCIStr("t2"), }) require.NoError(t, err) t3Info, err := json.Marshal(&model.TableInfo{ - Name: pmodel.NewCIStr("t3"), + Name: ast.NewCIStr("t3"), }) require.NoError(t, err) @@ -235,14 +235,14 @@ func TestNormalOperationsWithAddIndexBySQL(t *testing.T) { Name: "t1", ID: 1, Desired: &model.TableInfo{ - Name: pmodel.NewCIStr("t1"), + Name: ast.NewCIStr("t1"), }, }, "t2": { Name: "t2", ID: 2, Desired: &model.TableInfo{ - Name: pmodel.NewCIStr("t2"), + Name: ast.NewCIStr("t2"), }, }, }, @@ -254,7 +254,7 @@ func TestNormalOperationsWithAddIndexBySQL(t *testing.T) { Name: "t3", ID: 3, Desired: &model.TableInfo{ - Name: pmodel.NewCIStr("t3"), + Name: ast.NewCIStr("t3"), }, }, }, @@ -432,7 +432,7 @@ func TestNormalOperationsWithAddIndexBySQL(t *testing.T) { AutoRowIDBase: 132863, TableID: int64(2), TableInfo: &model.TableInfo{ - Name: pmodel.NewCIStr("t2"), + Name: ast.NewCIStr("t2"), }, Engines: map[int32]*checkpoints.EngineCheckpoint{ -1: {Status: checkpoints.CheckpointStatusLoaded}, diff --git a/pkg/lightning/errormanager/BUILD.bazel b/pkg/lightning/errormanager/BUILD.bazel index e6d342044572c..f6f4234007c34 100644 --- a/pkg/lightning/errormanager/BUILD.bazel +++ b/pkg/lightning/errormanager/BUILD.bazel @@ -50,7 +50,6 @@ go_test( "//pkg/meta/model", "//pkg/parser", "//pkg/parser/ast", - "//pkg/parser/model", "//pkg/parser/mysql", "//pkg/sessionctx/variable", "//pkg/table/tables", diff --git a/pkg/lightning/errormanager/errormanager_test.go b/pkg/lightning/errormanager/errormanager_test.go index 7b02b2772a77e..b81843961dea3 100644 --- a/pkg/lightning/errormanager/errormanager_test.go +++ b/pkg/lightning/errormanager/errormanager_test.go @@ -28,7 +28,7 @@ import ( "github.com/pingcap/tidb/pkg/lightning/config" "github.com/pingcap/tidb/pkg/lightning/log" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/table/tables" "github.com/pingcap/tidb/pkg/types" @@ -163,7 +163,7 @@ func (c mockConn) QueryContext(_ context.Context, query string, args []driver.Na func TestReplaceConflictOneKey(t *testing.T) { column1 := &model.ColumnInfo{ ID: 1, - Name: pmodel.NewCIStr("a"), + Name: ast.NewCIStr("a"), Offset: 0, DefaultValue: 0, FieldType: *types.NewFieldType(mysql.TypeLong), @@ -174,7 +174,7 @@ func TestReplaceConflictOneKey(t *testing.T) { column2 := &model.ColumnInfo{ ID: 2, - Name: pmodel.NewCIStr("b"), + Name: ast.NewCIStr("b"), Offset: 1, DefaultValue: 0, FieldType: *types.NewFieldType(mysql.TypeLong), @@ -184,7 +184,7 @@ func TestReplaceConflictOneKey(t *testing.T) { column3 := &model.ColumnInfo{ ID: 3, - Name: pmodel.NewCIStr("c"), + Name: ast.NewCIStr("c"), Offset: 2, DefaultValue: 0, FieldType: *types.NewFieldType(mysql.TypeBlob), @@ -194,11 +194,11 @@ func TestReplaceConflictOneKey(t *testing.T) { index := &model.IndexInfo{ ID: 1, - Name: pmodel.NewCIStr("key_b"), - Table: pmodel.NewCIStr(""), + Name: ast.NewCIStr("key_b"), + Table: ast.NewCIStr(""), Columns: []*model.IndexColumn{ { - Name: pmodel.NewCIStr("b"), + Name: ast.NewCIStr("b"), Offset: 1, Length: -1, }}, @@ -209,7 +209,7 @@ func TestReplaceConflictOneKey(t *testing.T) { table := &model.TableInfo{ ID: 104, - Name: pmodel.NewCIStr("a"), + Name: ast.NewCIStr("a"), Charset: "utf8mb4", Collate: "utf8mb4_bin", Columns: []*model.ColumnInfo{column1, column2, column3}, @@ -351,7 +351,7 @@ func TestReplaceConflictOneKey(t *testing.T) { func TestReplaceConflictOneUniqueKey(t *testing.T) { column1 := &model.ColumnInfo{ ID: 1, - Name: pmodel.NewCIStr("a"), + Name: ast.NewCIStr("a"), Offset: 0, DefaultValue: 0, FieldType: *types.NewFieldType(mysql.TypeLong), @@ -362,7 +362,7 @@ func TestReplaceConflictOneUniqueKey(t *testing.T) { column2 := &model.ColumnInfo{ ID: 2, - Name: pmodel.NewCIStr("b"), + Name: ast.NewCIStr("b"), Offset: 1, DefaultValue: 0, FieldType: *types.NewFieldType(mysql.TypeLong), @@ -373,7 +373,7 @@ func TestReplaceConflictOneUniqueKey(t *testing.T) { column3 := &model.ColumnInfo{ ID: 3, - Name: pmodel.NewCIStr("c"), + Name: ast.NewCIStr("c"), Offset: 2, DefaultValue: 0, FieldType: *types.NewFieldType(mysql.TypeBlob), @@ -383,11 +383,11 @@ func TestReplaceConflictOneUniqueKey(t *testing.T) { index := &model.IndexInfo{ ID: 1, - Name: pmodel.NewCIStr("uni_b"), - Table: pmodel.NewCIStr(""), + Name: ast.NewCIStr("uni_b"), + Table: ast.NewCIStr(""), Columns: []*model.IndexColumn{ { - Name: pmodel.NewCIStr("b"), + Name: ast.NewCIStr("b"), Offset: 1, Length: -1, }}, @@ -398,7 +398,7 @@ func TestReplaceConflictOneUniqueKey(t *testing.T) { table := &model.TableInfo{ ID: 104, - Name: pmodel.NewCIStr("a"), + Name: ast.NewCIStr("a"), Charset: "utf8mb4", Collate: "utf8mb4_bin", Columns: []*model.ColumnInfo{column1, column2, column3}, diff --git a/pkg/lightning/mydump/BUILD.bazel b/pkg/lightning/mydump/BUILD.bazel index e9538040836d6..c03d4224ca7e7 100644 --- a/pkg/lightning/mydump/BUILD.bazel +++ b/pkg/lightning/mydump/BUILD.bazel @@ -29,7 +29,6 @@ go_library( "//pkg/parser", "//pkg/parser/ast", "//pkg/parser/format", - "//pkg/parser/model", "//pkg/parser/mysql", "//pkg/types", "//pkg/util", diff --git a/pkg/lightning/mydump/schema_import.go b/pkg/lightning/mydump/schema_import.go index ea6da63854665..acc01b6246090 100644 --- a/pkg/lightning/mydump/schema_import.go +++ b/pkg/lightning/mydump/schema_import.go @@ -29,7 +29,6 @@ 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/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/util" "github.com/pingcap/tidb/pkg/util/set" @@ -388,21 +387,21 @@ func createIfNotExistsStmt(p *parser.Parser, createTable, dbName, tblName string for _, stmt := range stmts { switch node := stmt.(type) { case *ast.CreateDatabaseStmt: - node.Name = model.NewCIStr(dbName) + node.Name = ast.NewCIStr(dbName) node.IfNotExists = true case *ast.DropDatabaseStmt: - node.Name = model.NewCIStr(dbName) + node.Name = ast.NewCIStr(dbName) node.IfExists = true case *ast.CreateTableStmt: - node.Table.Schema = model.NewCIStr(dbName) - node.Table.Name = model.NewCIStr(tblName) + node.Table.Schema = ast.NewCIStr(dbName) + node.Table.Name = ast.NewCIStr(tblName) node.IfNotExists = true case *ast.CreateViewStmt: - node.ViewName.Schema = model.NewCIStr(dbName) - node.ViewName.Name = model.NewCIStr(tblName) + node.ViewName.Schema = ast.NewCIStr(dbName) + node.ViewName.Name = ast.NewCIStr(tblName) case *ast.DropTableStmt: - node.Tables[0].Schema = model.NewCIStr(dbName) - node.Tables[0].Name = model.NewCIStr(tblName) + node.Tables[0].Schema = ast.NewCIStr(dbName) + node.Tables[0].Name = ast.NewCIStr(tblName) node.IfExists = true } if err := stmt.Restore(ctx); err != nil { diff --git a/pkg/lock/BUILD.bazel b/pkg/lock/BUILD.bazel index a3df51d2adc9c..d64b775267cdf 100644 --- a/pkg/lock/BUILD.bazel +++ b/pkg/lock/BUILD.bazel @@ -9,7 +9,7 @@ go_library( "//pkg/infoschema", "//pkg/infoschema/context", "//pkg/lock/context", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/parser/mysql", "//pkg/table", "//pkg/util", diff --git a/pkg/lock/context/BUILD.bazel b/pkg/lock/context/BUILD.bazel index a38d9a52696b6..67c6107f8f25d 100644 --- a/pkg/lock/context/BUILD.bazel +++ b/pkg/lock/context/BUILD.bazel @@ -7,6 +7,6 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/meta/model", - "//pkg/parser/model", + "//pkg/parser/ast", ], ) diff --git a/pkg/lock/context/lockcontext.go b/pkg/lock/context/lockcontext.go index 7500a8178f170..55d4e74685bff 100644 --- a/pkg/lock/context/lockcontext.go +++ b/pkg/lock/context/lockcontext.go @@ -16,13 +16,13 @@ package context import ( "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" ) // TableLockReadContext is the interface to get table lock information. type TableLockReadContext interface { // CheckTableLocked checks the table lock. - CheckTableLocked(tblID int64) (bool, pmodel.TableLockType) + CheckTableLocked(tblID int64) (bool, ast.TableLockType) // GetAllTableLocks gets all table locks table id and db id hold by the session. GetAllTableLocks() []model.TableLockTpInfo // HasLockedTables uses to check whether this session locked any tables. diff --git a/pkg/lock/lock.go b/pkg/lock/lock.go index 86ebc88d9630d..1586b0365abc6 100644 --- a/pkg/lock/lock.go +++ b/pkg/lock/lock.go @@ -21,7 +21,7 @@ import ( "github.com/pingcap/tidb/pkg/infoschema" infoschemacontext "github.com/pingcap/tidb/pkg/infoschema/context" "github.com/pingcap/tidb/pkg/lock/context" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/util" @@ -68,7 +68,7 @@ func (c *Checker) CheckTableLock(db, table string, privilege mysql.PrivilegeType return nil } // TODO: try to remove this get for speed up. - tb, err := c.is.TableByName(stdctx.Background(), model.NewCIStr(db), model.NewCIStr(table)) + tb, err := c.is.TableByName(stdctx.Background(), ast.NewCIStr(db), ast.NewCIStr(table)) // Ignore this error for "drop table if not exists t1" when t1 doesn't exists. if infoschema.ErrTableNotExists.Equal(err) { return nil @@ -84,9 +84,9 @@ func (c *Checker) CheckTableLock(db, table string, privilege mysql.PrivilegeType for _, lockT := range lockTables { if lockT.TableID == tb.Meta().ID { switch tb.Meta().Lock.Tp { - case model.TableLockWrite: + case ast.TableLockWrite: return ErrLockedTableDropped - case model.TableLockRead, model.TableLockWriteLocal, model.TableLockReadOnly: + case ast.TableLockRead, ast.TableLockWriteLocal, ast.TableLockReadOnly: return infoschema.ErrTableNotLockedForWrite.GenWithStackByArgs(tb.Meta().Name) } } @@ -105,23 +105,23 @@ func (c *Checker) CheckTableLock(db, table string, privilege mysql.PrivilegeType if privilege == mysql.SelectPriv { switch tb.Meta().Lock.Tp { - case model.TableLockRead, model.TableLockWriteLocal, model.TableLockReadOnly: + case ast.TableLockRead, ast.TableLockWriteLocal, ast.TableLockReadOnly: return nil } } - if alterWriteable && tb.Meta().Lock.Tp == model.TableLockReadOnly { + if alterWriteable && tb.Meta().Lock.Tp == ast.TableLockReadOnly { return nil } return infoschema.ErrTableLocked.GenWithStackByArgs(tb.Meta().Name.L, tb.Meta().Lock.Tp, tb.Meta().Lock.Sessions[0]) } -func checkLockTpMeetPrivilege(tp model.TableLockType, privilege mysql.PrivilegeType) bool { +func checkLockTpMeetPrivilege(tp ast.TableLockType, privilege mysql.PrivilegeType) bool { // TableLockReadOnly doesn't need to check in this, because it is session unrelated. switch tp { - case model.TableLockWrite, model.TableLockWriteLocal: + case ast.TableLockWrite, ast.TableLockWriteLocal: return true - case model.TableLockRead: + case ast.TableLockRead: // ShowDBPriv, AllPrivMask, CreatePriv, CreateViewPriv already checked before. // The other privilege in read lock was not allowed. if privilege == mysql.SelectPriv { diff --git a/pkg/meta/BUILD.bazel b/pkg/meta/BUILD.bazel index 074561331472d..2b0b1a82aecc8 100644 --- a/pkg/meta/BUILD.bazel +++ b/pkg/meta/BUILD.bazel @@ -14,7 +14,7 @@ go_library( "//pkg/kv", "//pkg/meta/model", "//pkg/metrics", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/parser/mysql", "//pkg/resourcegroup", "//pkg/store/helper", @@ -47,7 +47,6 @@ go_test( "//pkg/meta/model", "//pkg/parser", "//pkg/parser/ast", - "//pkg/parser/model", "//pkg/parser/terror", "//pkg/planner/core", "//pkg/session", diff --git a/pkg/meta/autoid/BUILD.bazel b/pkg/meta/autoid/BUILD.bazel index 0ccb2c2ff2c8d..9af33da723522 100644 --- a/pkg/meta/autoid/BUILD.bazel +++ b/pkg/meta/autoid/BUILD.bazel @@ -53,7 +53,7 @@ go_test( "//pkg/kv", "//pkg/meta", "//pkg/meta/model", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/parser/mysql", "//pkg/parser/terror", "//pkg/store/mockstore", diff --git a/pkg/meta/autoid/autoid_test.go b/pkg/meta/autoid/autoid_test.go index 0128ba6199ae9..e135019b82147 100644 --- a/pkg/meta/autoid/autoid_test.go +++ b/pkg/meta/autoid/autoid_test.go @@ -30,7 +30,7 @@ import ( "github.com/pingcap/tidb/pkg/meta" "github.com/pingcap/tidb/pkg/meta/autoid" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/store/mockstore" "github.com/pingcap/tidb/pkg/util" "github.com/stretchr/testify/require" @@ -64,17 +64,17 @@ func TestSignedAutoid(t *testing.T) { ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnMeta) err = kv.RunInNewTxn(ctx, store, false, func(ctx context.Context, txn kv.Transaction) error { m := meta.NewMutator(txn) - err = m.CreateDatabase(&model.DBInfo{ID: 1, Name: pmodel.NewCIStr("a")}) + err = m.CreateDatabase(&model.DBInfo{ID: 1, Name: ast.NewCIStr("a")}) require.NoError(t, err) - err = m.CreateTableOrView(1, &model.TableInfo{ID: 1, Name: pmodel.NewCIStr("t")}) + err = m.CreateTableOrView(1, &model.TableInfo{ID: 1, Name: ast.NewCIStr("t")}) require.NoError(t, err) - err = m.CreateTableOrView(1, &model.TableInfo{ID: 2, Name: pmodel.NewCIStr("t1")}) + err = m.CreateTableOrView(1, &model.TableInfo{ID: 2, Name: ast.NewCIStr("t1")}) require.NoError(t, err) - err = m.CreateTableOrView(1, &model.TableInfo{ID: 3, Name: pmodel.NewCIStr("t1")}) + err = m.CreateTableOrView(1, &model.TableInfo{ID: 3, Name: ast.NewCIStr("t1")}) require.NoError(t, err) - err = m.CreateTableOrView(1, &model.TableInfo{ID: 4, Name: pmodel.NewCIStr("t2")}) + err = m.CreateTableOrView(1, &model.TableInfo{ID: 4, Name: ast.NewCIStr("t2")}) require.NoError(t, err) - err = m.CreateTableOrView(1, &model.TableInfo{ID: 5, Name: pmodel.NewCIStr("t3")}) + err = m.CreateTableOrView(1, &model.TableInfo{ID: 5, Name: ast.NewCIStr("t3")}) require.NoError(t, err) return nil }) @@ -269,17 +269,17 @@ func TestUnsignedAutoid(t *testing.T) { ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnMeta) err = kv.RunInNewTxn(ctx, store, false, func(ctx context.Context, txn kv.Transaction) error { m := meta.NewMutator(txn) - err = m.CreateDatabase(&model.DBInfo{ID: 1, Name: pmodel.NewCIStr("a")}) + err = m.CreateDatabase(&model.DBInfo{ID: 1, Name: ast.NewCIStr("a")}) require.NoError(t, err) - err = m.CreateTableOrView(1, &model.TableInfo{ID: 1, Name: pmodel.NewCIStr("t")}) + err = m.CreateTableOrView(1, &model.TableInfo{ID: 1, Name: ast.NewCIStr("t")}) require.NoError(t, err) - err = m.CreateTableOrView(1, &model.TableInfo{ID: 2, Name: pmodel.NewCIStr("t1")}) + err = m.CreateTableOrView(1, &model.TableInfo{ID: 2, Name: ast.NewCIStr("t1")}) require.NoError(t, err) - err = m.CreateTableOrView(1, &model.TableInfo{ID: 3, Name: pmodel.NewCIStr("t1")}) + err = m.CreateTableOrView(1, &model.TableInfo{ID: 3, Name: ast.NewCIStr("t1")}) require.NoError(t, err) - err = m.CreateTableOrView(1, &model.TableInfo{ID: 4, Name: pmodel.NewCIStr("t2")}) + err = m.CreateTableOrView(1, &model.TableInfo{ID: 4, Name: ast.NewCIStr("t2")}) require.NoError(t, err) - err = m.CreateTableOrView(1, &model.TableInfo{ID: 5, Name: pmodel.NewCIStr("t3")}) + err = m.CreateTableOrView(1, &model.TableInfo{ID: 5, Name: ast.NewCIStr("t3")}) require.NoError(t, err) return nil }) @@ -433,9 +433,9 @@ func TestConcurrentAlloc(t *testing.T) { ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnMeta) err = kv.RunInNewTxn(ctx, store, false, func(ctx context.Context, txn kv.Transaction) error { m := meta.NewMutator(txn) - err = m.CreateDatabase(&model.DBInfo{ID: dbID, Name: pmodel.NewCIStr("a")}) + err = m.CreateDatabase(&model.DBInfo{ID: dbID, Name: ast.NewCIStr("a")}) require.NoError(t, err) - err = m.CreateTableOrView(dbID, &model.TableInfo{ID: tblID, Name: pmodel.NewCIStr("t")}) + err = m.CreateTableOrView(dbID, &model.TableInfo{ID: tblID, Name: ast.NewCIStr("t")}) require.NoError(t, err) return nil }) @@ -519,9 +519,9 @@ func TestRollbackAlloc(t *testing.T) { ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnMeta) err = kv.RunInNewTxn(ctx, store, false, func(ctx context.Context, txn kv.Transaction) error { m := meta.NewMutator(txn) - err = m.CreateDatabase(&model.DBInfo{ID: dbID, Name: pmodel.NewCIStr("a")}) + err = m.CreateDatabase(&model.DBInfo{ID: dbID, Name: ast.NewCIStr("a")}) require.NoError(t, err) - err = m.CreateTableOrView(dbID, &model.TableInfo{ID: tblID, Name: pmodel.NewCIStr("t")}) + err = m.CreateTableOrView(dbID, &model.TableInfo{ID: tblID, Name: ast.NewCIStr("t")}) require.NoError(t, err) return nil }) @@ -569,11 +569,11 @@ func TestAllocComputationIssue(t *testing.T) { ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnMeta) err = kv.RunInNewTxn(ctx, store, false, func(ctx context.Context, txn kv.Transaction) error { m := meta.NewMutator(txn) - err = m.CreateDatabase(&model.DBInfo{ID: 1, Name: pmodel.NewCIStr("a")}) + err = m.CreateDatabase(&model.DBInfo{ID: 1, Name: ast.NewCIStr("a")}) require.NoError(t, err) - err = m.CreateTableOrView(1, &model.TableInfo{ID: 1, Name: pmodel.NewCIStr("t")}) + err = m.CreateTableOrView(1, &model.TableInfo{ID: 1, Name: ast.NewCIStr("t")}) require.NoError(t, err) - err = m.CreateTableOrView(1, &model.TableInfo{ID: 2, Name: pmodel.NewCIStr("t1")}) + err = m.CreateTableOrView(1, &model.TableInfo{ID: 2, Name: ast.NewCIStr("t1")}) require.NoError(t, err) return nil }) @@ -620,9 +620,9 @@ func TestIssue40584(t *testing.T) { ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnMeta) err = kv.RunInNewTxn(ctx, store, false, func(ctx context.Context, txn kv.Transaction) error { m := meta.NewMutator(txn) - err = m.CreateDatabase(&model.DBInfo{ID: 1, Name: pmodel.NewCIStr("a")}) + err = m.CreateDatabase(&model.DBInfo{ID: 1, Name: ast.NewCIStr("a")}) require.NoError(t, err) - err = m.CreateTableOrView(1, &model.TableInfo{ID: 1, Name: pmodel.NewCIStr("t")}) + err = m.CreateTableOrView(1, &model.TableInfo{ID: 1, Name: ast.NewCIStr("t")}) require.NoError(t, err) return nil }) diff --git a/pkg/meta/autoid/bench_test.go b/pkg/meta/autoid/bench_test.go index a4dbec927bb26..dcac8e76554cc 100644 --- a/pkg/meta/autoid/bench_test.go +++ b/pkg/meta/autoid/bench_test.go @@ -24,7 +24,7 @@ import ( "github.com/pingcap/tidb/pkg/meta" "github.com/pingcap/tidb/pkg/meta/autoid" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/store/mockstore" ) @@ -45,11 +45,11 @@ func BenchmarkAllocator_Alloc(b *testing.B) { ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnMeta) err = kv.RunInNewTxn(ctx, store, false, func(ctx context.Context, txn kv.Transaction) error { m := meta.NewMutator(txn) - err = m.CreateDatabase(&model.DBInfo{ID: dbID, Name: pmodel.NewCIStr("a")}) + err = m.CreateDatabase(&model.DBInfo{ID: dbID, Name: ast.NewCIStr("a")}) if err != nil { return err } - err = m.CreateTableOrView(dbID, &model.TableInfo{ID: tblID, Name: pmodel.NewCIStr("t")}) + err = m.CreateTableOrView(dbID, &model.TableInfo{ID: tblID, Name: ast.NewCIStr("t")}) if err != nil { return err } @@ -85,7 +85,7 @@ func BenchmarkAllocator_SequenceAlloc(b *testing.B) { ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnMeta) err = kv.RunInNewTxn(ctx, store, false, func(ctx context.Context, txn kv.Transaction) error { m := meta.NewMutator(txn) - err = m.CreateDatabase(&model.DBInfo{ID: 1, Name: pmodel.NewCIStr("a")}) + err = m.CreateDatabase(&model.DBInfo{ID: 1, Name: ast.NewCIStr("a")}) if err != nil { return err } @@ -100,7 +100,7 @@ func BenchmarkAllocator_SequenceAlloc(b *testing.B) { } seqTable := &model.TableInfo{ ID: 1, - Name: pmodel.NewCIStr("seq"), + Name: ast.NewCIStr("seq"), Sequence: seq, } sequenceBase = seq.Start - 1 diff --git a/pkg/meta/autoid/seq_autoid_test.go b/pkg/meta/autoid/seq_autoid_test.go index 0bcc907614b04..4e2143fa4fdd3 100644 --- a/pkg/meta/autoid/seq_autoid_test.go +++ b/pkg/meta/autoid/seq_autoid_test.go @@ -25,7 +25,7 @@ import ( "github.com/pingcap/tidb/pkg/meta" "github.com/pingcap/tidb/pkg/meta/autoid" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/store/mockstore" "github.com/pingcap/tidb/pkg/util" "github.com/stretchr/testify/require" @@ -44,7 +44,7 @@ func TestSequenceAutoid(t *testing.T) { ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnMeta) err = kv.RunInNewTxn(ctx, store, false, func(ctx context.Context, txn kv.Transaction) error { m := meta.NewMutator(txn) - err = m.CreateDatabase(&model.DBInfo{ID: 1, Name: pmodel.NewCIStr("a")}) + err = m.CreateDatabase(&model.DBInfo{ID: 1, Name: ast.NewCIStr("a")}) require.NoError(t, err) seq = &model.SequenceInfo{ Start: 1, @@ -57,7 +57,7 @@ func TestSequenceAutoid(t *testing.T) { } seqTable := &model.TableInfo{ ID: 1, - Name: pmodel.NewCIStr("seq"), + Name: ast.NewCIStr("seq"), Sequence: seq, } sequenceBase = seq.Start - 1 @@ -169,7 +169,7 @@ func TestConcurrentAllocSequence(t *testing.T) { ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnMeta) err = kv.RunInNewTxn(ctx, store, false, func(ctx context.Context, txn kv.Transaction) error { m := meta.NewMutator(txn) - err1 := m.CreateDatabase(&model.DBInfo{ID: 2, Name: pmodel.NewCIStr("a")}) + err1 := m.CreateDatabase(&model.DBInfo{ID: 2, Name: ast.NewCIStr("a")}) require.NoError(t, err1) seq = &model.SequenceInfo{ Start: 100, @@ -182,7 +182,7 @@ func TestConcurrentAllocSequence(t *testing.T) { } seqTable := &model.TableInfo{ ID: 2, - Name: pmodel.NewCIStr("seq"), + Name: ast.NewCIStr("seq"), Sequence: seq, } if seq.Increment >= 0 { diff --git a/pkg/meta/meta.go b/pkg/meta/meta.go index cc70646b00159..d7ea87b7ca817 100644 --- a/pkg/meta/meta.go +++ b/pkg/meta/meta.go @@ -34,7 +34,7 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/metrics" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/resourcegroup" "github.com/pingcap/tidb/pkg/store/helper" @@ -104,10 +104,10 @@ var ( ResourceGroupSettings: &model.ResourceGroupSettings{ RURate: math.MaxInt32, BurstLimit: -1, - Priority: pmodel.MediumPriorityValue, + Priority: ast.MediumPriorityValue, }, ID: defaultGroupID, - Name: pmodel.NewCIStr(resourcegroup.DefaultResourceGroupName), + Name: ast.NewCIStr(resourcegroup.DefaultResourceGroupName), State: model.StatePublic, } ) @@ -764,7 +764,7 @@ func (m *Mutator) CreateMySQLDatabaseIfNotExists() (int64, error) { } db := model.DBInfo{ ID: id, - Name: pmodel.NewCIStr(mysql.SystemDB), + Name: ast.NewCIStr(mysql.SystemDB), Charset: mysql.UTF8MB4Charset, Collate: mysql.UTF8MB4DefaultCollation, State: model.StatePublic, @@ -1230,7 +1230,7 @@ func FastUnmarshalTableNameInfo(data []byte) (*model.TableNameInfo, error) { return &model.TableNameInfo{ ID: id, - Name: pmodel.NewCIStr(name), + Name: ast.NewCIStr(name), }, nil } diff --git a/pkg/meta/meta_test.go b/pkg/meta/meta_test.go index a30f513198d09..6b2c1c0068697 100644 --- a/pkg/meta/meta_test.go +++ b/pkg/meta/meta_test.go @@ -32,7 +32,6 @@ import ( "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser" "github.com/pingcap/tidb/pkg/parser/ast" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/terror" _ "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/session" @@ -62,7 +61,7 @@ func TestPlacementPolicy(t *testing.T) { // test the meta storage of placemnt policy. policy := &model.PolicyInfo{ ID: 1, - Name: pmodel.NewCIStr("aa"), + Name: ast.NewCIStr("aa"), PlacementSettings: &model.PlacementSettings{ PrimaryRegion: "my primary", Regions: "my regions", @@ -87,7 +86,7 @@ func TestPlacementPolicy(t *testing.T) { require.Equal(t, policy, val) // mock updating the placement policy. - policy.Name = pmodel.NewCIStr("bb") + policy.Name = ast.NewCIStr("bb") policy.LearnerConstraints = "+zone=nanjing" err = m.UpdatePolicy(policy) require.NoError(t, err) @@ -142,7 +141,7 @@ func TestResourceGroup(t *testing.T) { rg := &model.ResourceGroupInfo{ ID: groupID, - Name: pmodel.NewCIStr("aa"), + Name: ast.NewCIStr("aa"), ResourceGroupSettings: &model.ResourceGroupSettings{ RURate: 100, }, @@ -213,7 +212,7 @@ func TestMeta(t *testing.T) { dbInfo := &model.DBInfo{ ID: 1, - Name: pmodel.NewCIStr("a"), + Name: ast.NewCIStr("a"), } err = m.CreateDatabase(dbInfo) require.NoError(t, err) @@ -226,7 +225,7 @@ func TestMeta(t *testing.T) { require.NoError(t, err) require.Equal(t, dbInfo, v) - dbInfo.Name = pmodel.NewCIStr("aa") + dbInfo.Name = ast.NewCIStr("aa") err = m.UpdateDatabase(dbInfo) require.NoError(t, err) @@ -240,7 +239,7 @@ func TestMeta(t *testing.T) { tbInfo := &model.TableInfo{ ID: 1, - Name: pmodel.NewCIStr("t"), + Name: ast.NewCIStr("t"), DBID: dbInfo.ID, } err = m.CreateTableOrView(1, tbInfo) @@ -258,7 +257,7 @@ func TestMeta(t *testing.T) { require.NotNil(t, err) require.True(t, meta.ErrTableExists.Equal(err)) - tbInfo.Name = pmodel.NewCIStr("tt") + tbInfo.Name = ast.NewCIStr("tt") err = m.UpdateTable(1, tbInfo) require.NoError(t, err) @@ -278,7 +277,7 @@ func TestMeta(t *testing.T) { tbInfo2 := &model.TableInfo{ ID: 2, - Name: pmodel.NewCIStr("bb"), + Name: ast.NewCIStr("bb"), DBID: dbInfo.ID, } err = m.CreateTableOrView(1, tbInfo2) @@ -344,7 +343,7 @@ func TestMeta(t *testing.T) { tid := int64(100) tbInfo100 := &model.TableInfo{ ID: tid, - Name: pmodel.NewCIStr("t_rename"), + Name: ast.NewCIStr("t_rename"), } // Create table. err = m.CreateTableOrView(1, tbInfo100) @@ -371,7 +370,7 @@ func TestMeta(t *testing.T) { // Test case for CreateTableAndSetAutoID. tbInfo3 := &model.TableInfo{ ID: 3, - Name: pmodel.NewCIStr("tbl3"), + Name: ast.NewCIStr("tbl3"), } err = m.CreateTableAndSetAutoID(1, tbInfo3, model.AutoIDGroup{RowID: 123, IncrementID: 0}) require.NoError(t, err) @@ -745,7 +744,7 @@ func TestIsTableInfoMustLoadSubStringsOrder(t *testing.T) { func TestTableNameExtract(t *testing.T) { var tbl model.TableInfo - tbl.Name = pmodel.NewCIStr(`a`) + tbl.Name = ast.NewCIStr(`a`) b, err := json.Marshal(tbl) require.NoError(t, err) @@ -754,28 +753,28 @@ func TestTableNameExtract(t *testing.T) { require.Len(t, nameLMatch, 2) require.Equal(t, "a", nameLMatch[1]) - tbl.Name = pmodel.NewCIStr(`"a"`) + tbl.Name = ast.NewCIStr(`"a"`) b, err = json.Marshal(tbl) require.NoError(t, err) nameLMatch = nameLRegex.FindStringSubmatch(string(b)) require.Len(t, nameLMatch, 2) require.Equal(t, `"a"`, meta.Unescape(nameLMatch[1])) - tbl.Name = pmodel.NewCIStr(`""a"`) + tbl.Name = ast.NewCIStr(`""a"`) b, err = json.Marshal(tbl) require.NoError(t, err) nameLMatch = nameLRegex.FindStringSubmatch(string(b)) require.Len(t, nameLMatch, 2) require.Equal(t, `""a"`, meta.Unescape(nameLMatch[1])) - tbl.Name = pmodel.NewCIStr(`"\"a"`) + tbl.Name = ast.NewCIStr(`"\"a"`) b, err = json.Marshal(tbl) require.NoError(t, err) nameLMatch = nameLRegex.FindStringSubmatch(string(b)) require.Len(t, nameLMatch, 2) require.Equal(t, `"\"a"`, meta.Unescape(nameLMatch[1])) - tbl.Name = pmodel.NewCIStr(`"\"啊"`) + tbl.Name = ast.NewCIStr(`"\"啊"`) b, err = json.Marshal(tbl) require.NoError(t, err) nameLMatch = nameLRegex.FindStringSubmatch(string(b)) @@ -931,28 +930,28 @@ func TestInfoSchemaV2SpecialAttributeCorrectnessAfterBootstrap(t *testing.T) { // create database dbInfo := &model.DBInfo{ ID: 10001, - Name: pmodel.NewCIStr("sc"), + Name: ast.NewCIStr("sc"), State: model.StatePublic, } // create table with special attributes tblInfo := &model.TableInfo{ ID: 10002, - Name: pmodel.NewCIStr("cs"), + Name: ast.NewCIStr("cs"), State: model.StatePublic, Partition: &model.PartitionInfo{ Definitions: []model.PartitionDefinition{ - {ID: 11, Name: pmodel.NewCIStr("p1")}, - {ID: 22, Name: pmodel.NewCIStr("p2")}, + {ID: 11, Name: ast.NewCIStr("p1")}, + {ID: 22, Name: ast.NewCIStr("p2")}, }, Enable: true, }, ForeignKeys: []*model.FKInfo{{ ID: 1, - Name: pmodel.NewCIStr("fk"), - RefTable: pmodel.NewCIStr("t"), - RefCols: []pmodel.CIStr{pmodel.NewCIStr("a")}, - Cols: []pmodel.CIStr{pmodel.NewCIStr("t_a")}, + Name: ast.NewCIStr("fk"), + RefTable: ast.NewCIStr("t"), + RefCols: []ast.CIStr{ast.NewCIStr("a")}, + Cols: []ast.CIStr{ast.NewCIStr("t_a")}, }}, TiFlashReplica: &model.TiFlashReplicaInfo{ Count: 0, @@ -960,13 +959,13 @@ func TestInfoSchemaV2SpecialAttributeCorrectnessAfterBootstrap(t *testing.T) { Available: true, }, Lock: &model.TableLockInfo{ - Tp: pmodel.TableLockRead, + Tp: ast.TableLockRead, State: model.TableLockStatePreLock, TS: 0, }, PlacementPolicyRef: &model.PolicyRefInfo{ ID: 1, - Name: pmodel.NewCIStr("r1"), + Name: ast.NewCIStr("r1"), }, TTLInfo: &model.TTLInfo{ IntervalExprStr: "1", @@ -1027,7 +1026,7 @@ func TestInfoSchemaV2DataFieldsCorrectnessAfterBootstrap(t *testing.T) { // create database dbInfo := &model.DBInfo{ ID: 10001, - Name: pmodel.NewCIStr("sc"), + Name: ast.NewCIStr("sc"), Charset: "utf8", Collate: "utf8_general_ci", State: model.StatePublic, @@ -1036,13 +1035,13 @@ func TestInfoSchemaV2DataFieldsCorrectnessAfterBootstrap(t *testing.T) { // create table with partition info tblInfo := &model.TableInfo{ ID: 10002, - Name: pmodel.NewCIStr("cs"), + Name: ast.NewCIStr("cs"), Charset: "latin1", Collate: "latin1_bin", State: model.StatePublic, Partition: &model.PartitionInfo{ Definitions: []model.PartitionDefinition{ - {ID: 1, Name: pmodel.NewCIStr("p1")}, + {ID: 1, Name: ast.NewCIStr("p1")}, }, Enable: true, }, @@ -1070,7 +1069,7 @@ func TestInfoSchemaV2DataFieldsCorrectnessAfterBootstrap(t *testing.T) { require.Equal(t, tbl.Meta().ID, tblInfo.ID) //byName, traverse byName and load from store, - tbl, err = is.TableByName(context.Background(), pmodel.NewCIStr("sc"), pmodel.NewCIStr("cs")) + tbl, err = is.TableByName(context.Background(), ast.NewCIStr("sc"), ast.NewCIStr("cs")) require.NoError(t, err) require.Equal(t, tbl.Meta().ID, tblInfo.ID) @@ -1080,7 +1079,7 @@ func TestInfoSchemaV2DataFieldsCorrectnessAfterBootstrap(t *testing.T) { require.Equal(t, tbl.Meta().ID, tblInfo.ID) //schemaMap, traverse schemaMap find dbInfo - db, ok := is.SchemaByName(pmodel.NewCIStr("sc")) + db, ok := is.SchemaByName(ast.NewCIStr("sc")) require.True(t, ok) require.Equal(t, db.ID, dbInfo.ID) @@ -1109,12 +1108,12 @@ func TestInfoSchemaMiscFieldsCorrectnessAfterBootstrap(t *testing.T) { dbInfo := &model.DBInfo{ ID: 10001, - Name: pmodel.NewCIStr("sc"), + Name: ast.NewCIStr("sc"), State: model.StatePublic, } policy := &model.PolicyInfo{ ID: 2, - Name: pmodel.NewCIStr("policy_1"), + Name: ast.NewCIStr("policy_1"), PlacementSettings: &model.PlacementSettings{ PrimaryRegion: "r1", Regions: "r1,r2", @@ -1122,17 +1121,17 @@ func TestInfoSchemaMiscFieldsCorrectnessAfterBootstrap(t *testing.T) { } group := &model.ResourceGroupInfo{ ID: 3, - Name: pmodel.NewCIStr("groupName_1"), + Name: ast.NewCIStr("groupName_1"), } tblInfo := &model.TableInfo{ ID: 10002, - Name: pmodel.NewCIStr("cs"), + Name: ast.NewCIStr("cs"), State: model.StatePublic, ForeignKeys: []*model.FKInfo{{ ID: 1, - Name: pmodel.NewCIStr("fk_1"), - RefSchema: pmodel.NewCIStr("t1"), - RefTable: pmodel.NewCIStr("parent"), + Name: ast.NewCIStr("fk_1"), + RefSchema: ast.NewCIStr("t1"), + RefTable: ast.NewCIStr("parent"), Version: 1, }}, PlacementPolicyRef: &model.PolicyRefInfo{ @@ -1142,7 +1141,7 @@ func TestInfoSchemaMiscFieldsCorrectnessAfterBootstrap(t *testing.T) { } tblInfo1 := &model.TableInfo{ ID: 10003, - Name: pmodel.NewCIStr("cs"), + Name: ast.NewCIStr("cs"), State: model.StatePublic, TempTableType: model.TempTableLocal, } diff --git a/pkg/meta/model/BUILD.bazel b/pkg/meta/model/BUILD.bazel index e9b16e0813c85..eae2284e24980 100644 --- a/pkg/meta/model/BUILD.bazel +++ b/pkg/meta/model/BUILD.bazel @@ -22,7 +22,6 @@ go_library( "//pkg/parser/auth", "//pkg/parser/charset", "//pkg/parser/duration", - "//pkg/parser/model", "//pkg/parser/mysql", "//pkg/parser/terror", "//pkg/parser/types", @@ -54,7 +53,6 @@ go_test( "//pkg/parser/ast", "//pkg/parser/charset", "//pkg/parser/duration", - "//pkg/parser/model", "//pkg/parser/mysql", "//pkg/parser/terror", "//pkg/parser/types", diff --git a/pkg/meta/model/column.go b/pkg/meta/model/column.go index 8d510f0c7f06b..bb09dbf8a19b4 100644 --- a/pkg/meta/model/column.go +++ b/pkg/meta/model/column.go @@ -18,8 +18,8 @@ import ( "strings" "unsafe" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/charset" - "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/types" ) @@ -48,13 +48,13 @@ type ChangeStateInfo struct { // ColumnInfo provides meta data describing of a table column. type ColumnInfo struct { - ID int64 `json:"id"` - Name model.CIStr `json:"name"` - Offset int `json:"offset"` - OriginDefaultValue any `json:"origin_default"` - OriginDefaultValueBit []byte `json:"origin_default_bit"` - DefaultValue any `json:"default"` - DefaultValueBit []byte `json:"default_bit"` + ID int64 `json:"id"` + Name ast.CIStr `json:"name"` + Offset int `json:"offset"` + OriginDefaultValue any `json:"origin_default"` + OriginDefaultValueBit []byte `json:"origin_default_bit"` + DefaultValue any `json:"default"` + DefaultValueBit []byte `json:"default_bit"` // DefaultIsExpr is indicates the default value string is expr. DefaultIsExpr bool `json:"default_is_expr"` GeneratedExprString string `json:"generated_expr_string"` diff --git a/pkg/meta/model/column_test.go b/pkg/meta/model/column_test.go index 5644be0799a4b..bd46ccbf01cf5 100644 --- a/pkg/meta/model/column_test.go +++ b/pkg/meta/model/column_test.go @@ -19,7 +19,7 @@ import ( "fmt" "testing" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/types" "github.com/stretchr/testify/require" @@ -32,13 +32,13 @@ func TestDefaultValue(t *testing.T) { randPlainStr := "random_plain_string" oldPlainCol := srcCol.Clone() - oldPlainCol.Name = model.NewCIStr("oldPlainCol") + oldPlainCol.Name = ast.NewCIStr("oldPlainCol") oldPlainCol.FieldType = *types.NewFieldType(mysql.TypeLong) oldPlainCol.DefaultValue = randPlainStr oldPlainCol.OriginDefaultValue = randPlainStr newPlainCol := srcCol.Clone() - newPlainCol.Name = model.NewCIStr("newPlainCol") + newPlainCol.Name = ast.NewCIStr("newPlainCol") newPlainCol.FieldType = *types.NewFieldType(mysql.TypeLong) err := newPlainCol.SetDefaultValue(1) require.NoError(t, err) @@ -50,13 +50,13 @@ func TestDefaultValue(t *testing.T) { randBitStr := string([]byte{25, 185}) oldBitCol := srcCol.Clone() - oldBitCol.Name = model.NewCIStr("oldBitCol") + oldBitCol.Name = ast.NewCIStr("oldBitCol") oldBitCol.FieldType = *types.NewFieldType(mysql.TypeBit) oldBitCol.DefaultValue = randBitStr oldBitCol.OriginDefaultValue = randBitStr newBitCol := srcCol.Clone() - newBitCol.Name = model.NewCIStr("newBitCol") + newBitCol.Name = ast.NewCIStr("newBitCol") newBitCol.FieldType = *types.NewFieldType(mysql.TypeBit) err = newBitCol.SetDefaultValue(1) // Only string type is allowed in BIT column. @@ -68,7 +68,7 @@ func TestDefaultValue(t *testing.T) { require.Equal(t, randBitStr, newBitCol.GetDefaultValue()) nullBitCol := srcCol.Clone() - nullBitCol.Name = model.NewCIStr("nullBitCol") + nullBitCol.Name = ast.NewCIStr("nullBitCol") nullBitCol.FieldType = *types.NewFieldType(mysql.TypeBit) err = nullBitCol.SetOriginDefaultValue(nil) require.NoError(t, err) diff --git a/pkg/meta/model/db.go b/pkg/meta/model/db.go index ece71f17c815b..ee550b02dd311 100644 --- a/pkg/meta/model/db.go +++ b/pkg/meta/model/db.go @@ -17,16 +17,16 @@ package model import ( "strings" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" ) // DBInfo provides meta data describing a DB. type DBInfo struct { - ID int64 `json:"id"` // Database ID - Name model.CIStr `json:"db_name"` // DB name. - Charset string `json:"charset"` - Collate string `json:"collate"` - Deprecated struct { // Tables is not set in infoschema v2, use infoschema SchemaTableInfos() instead. + ID int64 `json:"id"` // Database ID + Name ast.CIStr `json:"db_name"` // DB name. + Charset string `json:"charset"` + Collate string `json:"collate"` + Deprecated struct { // Tables is not set in infoschema v2, use infoschema SchemaTableInfos() instead. Tables []*TableInfo `json:"-"` // Tables in the DB. } State SchemaState `json:"state"` diff --git a/pkg/meta/model/index.go b/pkg/meta/model/index.go index da8518aebc000..4ec0f3ae05c0a 100644 --- a/pkg/meta/model/index.go +++ b/pkg/meta/model/index.go @@ -16,7 +16,6 @@ package model import ( "github.com/pingcap/tidb/pkg/parser/ast" - "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/types" "github.com/pingcap/tidb/pkg/planner/cascades/base" ) @@ -64,13 +63,13 @@ type VectorIndexInfo struct { // See https://dev.mysql.com/doc/refman/5.7/en/create-index.html type IndexInfo struct { ID int64 `json:"id"` - Name model.CIStr `json:"idx_name"` // Index name. - Table model.CIStr `json:"tbl_name"` // Table name. + Name ast.CIStr `json:"idx_name"` // Index name. + Table ast.CIStr `json:"tbl_name"` // Table name. Columns []*IndexColumn `json:"idx_cols"` // Index columns. State SchemaState `json:"state"` BackfillState BackfillState `json:"backfill_state"` Comment string `json:"comment"` // Comment - Tp model.IndexType `json:"index_type"` // Index type: Btree, Hash, Rtree or HNSW + Tp ast.IndexType `json:"index_type"` // Index type: Btree, Hash, Rtree or HNSW Unique bool `json:"is_unique"` // Whether the index is unique. Primary bool `json:"is_primary"` // Whether the index is primary key. Invisible bool `json:"is_invisible"` // Whether the index is invisible. @@ -145,7 +144,7 @@ func (index *IndexInfo) IsPublic() bool { } // FindIndexByColumns find IndexInfo in indices which is cover the specified columns. -func FindIndexByColumns(tbInfo *TableInfo, indices []*IndexInfo, cols ...model.CIStr) *IndexInfo { +func FindIndexByColumns(tbInfo *TableInfo, indices []*IndexInfo, cols ...ast.CIStr) *IndexInfo { for _, index := range indices { if IsIndexPrefixCovered(tbInfo, index, cols...) { return index @@ -155,7 +154,7 @@ func FindIndexByColumns(tbInfo *TableInfo, indices []*IndexInfo, cols ...model.C } // IsIndexPrefixCovered checks the index's columns beginning with the cols. -func IsIndexPrefixCovered(tbInfo *TableInfo, index *IndexInfo, cols ...model.CIStr) bool { +func IsIndexPrefixCovered(tbInfo *TableInfo, index *IndexInfo, cols ...ast.CIStr) bool { if len(index.Columns) < len(cols) { return false } @@ -184,8 +183,8 @@ func FindIndexInfoByID(indices []*IndexInfo, id int64) *IndexInfo { // IndexColumn provides index column info. type IndexColumn struct { - Name model.CIStr `json:"name"` // Index name - Offset int `json:"offset"` // Index offset + Name ast.CIStr `json:"name"` // Index name + Offset int `json:"offset"` // Index offset // Length of prefix when using column prefix // for indexing; // UnspecifedLength if not using prefix indexing diff --git a/pkg/meta/model/index_test.go b/pkg/meta/model/index_test.go index 1c65da0ea7e36..251b1009f8b32 100644 --- a/pkg/meta/model/index_test.go +++ b/pkg/meta/model/index_test.go @@ -18,14 +18,14 @@ import ( "fmt" "testing" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/stretchr/testify/require" ) func newColumnForTest(id int64, offset int) *ColumnInfo { return &ColumnInfo{ ID: id, - Name: model.NewCIStr(fmt.Sprintf("c_%d", id)), + Name: ast.NewCIStr(fmt.Sprintf("c_%d", id)), Offset: offset, } } @@ -37,7 +37,7 @@ func newIndexForTest(id int64, cols ...*ColumnInfo) *IndexInfo { } return &IndexInfo{ ID: id, - Name: model.NewCIStr(fmt.Sprintf("i_%d", id)), + Name: ast.NewCIStr(fmt.Sprintf("i_%d", id)), Columns: idxCols, } } @@ -54,18 +54,18 @@ func TestIsIndexPrefixCovered(t *testing.T) { tbl := &TableInfo{ ID: 1, - Name: model.NewCIStr("t"), + Name: ast.NewCIStr("t"), Columns: []*ColumnInfo{c0, c1, c2, c3, c4}, Indices: []*IndexInfo{i0, i1}, } - require.Equal(t, true, IsIndexPrefixCovered(tbl, i0, model.NewCIStr("c_0"))) - require.Equal(t, true, IsIndexPrefixCovered(tbl, i0, model.NewCIStr("c_0"), model.NewCIStr("c_1"), model.NewCIStr("c_2"))) - require.Equal(t, false, IsIndexPrefixCovered(tbl, i0, model.NewCIStr("c_1"))) - require.Equal(t, false, IsIndexPrefixCovered(tbl, i0, model.NewCIStr("c_2"))) - require.Equal(t, false, IsIndexPrefixCovered(tbl, i0, model.NewCIStr("c_1"), model.NewCIStr("c_2"))) - require.Equal(t, false, IsIndexPrefixCovered(tbl, i0, model.NewCIStr("c_0"), model.NewCIStr("c_2"))) + require.Equal(t, true, IsIndexPrefixCovered(tbl, i0, ast.NewCIStr("c_0"))) + require.Equal(t, true, IsIndexPrefixCovered(tbl, i0, ast.NewCIStr("c_0"), ast.NewCIStr("c_1"), ast.NewCIStr("c_2"))) + require.Equal(t, false, IsIndexPrefixCovered(tbl, i0, ast.NewCIStr("c_1"))) + require.Equal(t, false, IsIndexPrefixCovered(tbl, i0, ast.NewCIStr("c_2"))) + require.Equal(t, false, IsIndexPrefixCovered(tbl, i0, ast.NewCIStr("c_1"), ast.NewCIStr("c_2"))) + require.Equal(t, false, IsIndexPrefixCovered(tbl, i0, ast.NewCIStr("c_0"), ast.NewCIStr("c_2"))) - require.Equal(t, true, IsIndexPrefixCovered(tbl, i1, model.NewCIStr("c_4"))) - require.Equal(t, true, IsIndexPrefixCovered(tbl, i1, model.NewCIStr("c_4"), model.NewCIStr("c_2"))) - require.Equal(t, false, IsIndexPrefixCovered(tbl, i0, model.NewCIStr("c_2"))) + require.Equal(t, true, IsIndexPrefixCovered(tbl, i1, ast.NewCIStr("c_4"))) + require.Equal(t, true, IsIndexPrefixCovered(tbl, i1, ast.NewCIStr("c_4"), ast.NewCIStr("c_2"))) + require.Equal(t, false, IsIndexPrefixCovered(tbl, i0, ast.NewCIStr("c_2"))) } diff --git a/pkg/meta/model/job.go b/pkg/meta/model/job.go index 84ea0bc55bda4..1b0919fe2c01b 100644 --- a/pkg/meta/model/job.go +++ b/pkg/meta/model/job.go @@ -22,7 +22,7 @@ import ( "time" "github.com/pingcap/errors" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/util/intest" @@ -896,23 +896,23 @@ type MultiSchemaInfo struct { // SkipVersion is used to control whether generating a new schema version for a sub-job. SkipVersion bool `json:"-"` - AddColumns []model.CIStr `json:"-"` - DropColumns []model.CIStr `json:"-"` - ModifyColumns []model.CIStr `json:"-"` - AddIndexes []model.CIStr `json:"-"` - DropIndexes []model.CIStr `json:"-"` - AlterIndexes []model.CIStr `json:"-"` + AddColumns []ast.CIStr `json:"-"` + DropColumns []ast.CIStr `json:"-"` + ModifyColumns []ast.CIStr `json:"-"` + AddIndexes []ast.CIStr `json:"-"` + DropIndexes []ast.CIStr `json:"-"` + AlterIndexes []ast.CIStr `json:"-"` AddForeignKeys []AddForeignKeyInfo `json:"-"` - RelativeColumns []model.CIStr `json:"-"` - PositionColumns []model.CIStr `json:"-"` + RelativeColumns []ast.CIStr `json:"-"` + PositionColumns []ast.CIStr `json:"-"` } // AddForeignKeyInfo contains foreign key information. type AddForeignKeyInfo struct { - Name model.CIStr - Cols []model.CIStr + Name ast.CIStr + Cols []ast.CIStr } // NewMultiSchemaInfo new a MultiSchemaInfo. diff --git a/pkg/meta/model/job_args.go b/pkg/meta/model/job_args.go index 5aff6e7d851bd..0104198ad19c1 100644 --- a/pkg/meta/model/job_args.go +++ b/pkg/meta/model/job_args.go @@ -19,7 +19,6 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/parser/ast" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/util/intest" pdhttp "github.com/tikv/pd/client/http" @@ -53,7 +52,7 @@ type RecoverSchemaInfo struct { LoadTablesOnExecute bool DropJobID int64 SnapshotTS uint64 - OldSchemaName pmodel.CIStr + OldSchemaName ast.CIStr } // getOrDecodeArgsV1 get the args v1 from job, if the job.Args is nil, decode job.RawArgs @@ -577,14 +576,14 @@ func GetAlterTablePartitionArgs(job *Job) (*AlterTablePartitionArgs, error) { // It's also used for rename tables. type RenameTableArgs struct { // for Args - OldSchemaID int64 `json:"old_schema_id,omitempty"` - OldSchemaName pmodel.CIStr `json:"old_schema_name,omitempty"` - NewTableName pmodel.CIStr `json:"new_table_name,omitempty"` + OldSchemaID int64 `json:"old_schema_id,omitempty"` + OldSchemaName ast.CIStr `json:"old_schema_name,omitempty"` + NewTableName ast.CIStr `json:"new_table_name,omitempty"` // for rename tables - OldTableName pmodel.CIStr `json:"old_table_name,omitempty"` - NewSchemaID int64 `json:"new_schema_id,omitempty"` - TableID int64 `json:"table_id,omitempty"` + OldTableName ast.CIStr `json:"old_table_name,omitempty"` + NewSchemaID int64 `json:"new_schema_id,omitempty"` + TableID int64 `json:"table_id,omitempty"` // runtime info OldSchemaIDForSchemaDiff int64 `json:"-"` @@ -703,8 +702,8 @@ func GetModifyTableCharsetAndCollateArgs(job *Job) (*ModifyTableCharsetAndCollat // AlterIndexVisibilityArgs is the arguments for ActionAlterIndexVisibility ddl. type AlterIndexVisibilityArgs struct { - IndexName pmodel.CIStr `json:"index_name,omitempty"` - Invisible bool `json:"invisible,omitempty"` + IndexName ast.CIStr `json:"index_name,omitempty"` + Invisible bool `json:"invisible,omitempty"` } func (a *AlterIndexVisibilityArgs) getArgsV1(*Job) []any { @@ -741,7 +740,7 @@ func GetAddForeignKeyArgs(job *Job) (*AddForeignKeyArgs, error) { // DropForeignKeyArgs is the arguments for DropForeignKey ddl. type DropForeignKeyArgs struct { - FkName pmodel.CIStr `json:"fk_name,omitempty"` + FkName ast.CIStr `json:"fk_name,omitempty"` } func (a *DropForeignKeyArgs) getArgsV1(*Job) []any { @@ -815,10 +814,10 @@ type RenameTablesArgs struct { func (a *RenameTablesArgs) getArgsV1(*Job) []any { n := len(a.RenameTableInfos) oldSchemaIDs := make([]int64, n) - oldSchemaNames := make([]pmodel.CIStr, n) - oldTableNames := make([]pmodel.CIStr, n) + oldSchemaNames := make([]ast.CIStr, n) + oldTableNames := make([]ast.CIStr, n) newSchemaIDs := make([]int64, n) - newTableNames := make([]pmodel.CIStr, n) + newTableNames := make([]ast.CIStr, n) tableIDs := make([]int64, n) for i, info := range a.RenameTableInfos { @@ -837,10 +836,10 @@ func (a *RenameTablesArgs) getArgsV1(*Job) []any { func (a *RenameTablesArgs) decodeV1(job *Job) error { var ( oldSchemaIDs []int64 - oldSchemaNames []pmodel.CIStr - oldTableNames []pmodel.CIStr + oldSchemaNames []ast.CIStr + oldTableNames []ast.CIStr newSchemaIDs []int64 - newTableNames []pmodel.CIStr + newTableNames []ast.CIStr tableIDs []int64 ) if err := job.decodeArgs( @@ -859,10 +858,10 @@ func (a *RenameTablesArgs) decodeV1(job *Job) error { // GetRenameTablesArgsFromV1 get v2 args from v1 func GetRenameTablesArgsFromV1( oldSchemaIDs []int64, - oldSchemaNames []pmodel.CIStr, - oldTableNames []pmodel.CIStr, + oldSchemaNames []ast.CIStr, + oldTableNames []ast.CIStr, newSchemaIDs []int64, - newTableNames []pmodel.CIStr, + newTableNames []ast.CIStr, tableIDs []int64, ) []*RenameTableArgs { infos := make([]*RenameTableArgs, 0, len(oldSchemaIDs)) @@ -962,8 +961,8 @@ func GetAlterTTLInfoArgs(job *Job) (*AlterTTLInfoArgs, error) { // CheckConstraintArgs is the arguments for both AlterCheckConstraint and DropCheckConstraint job. type CheckConstraintArgs struct { - ConstraintName pmodel.CIStr `json:"constraint_name,omitempty"` - Enforced bool `json:"enforced,omitempty"` + ConstraintName ast.CIStr `json:"constraint_name,omitempty"` + Enforced bool `json:"enforced,omitempty"` } func (a *CheckConstraintArgs) getArgsV1(*Job) []any { @@ -1165,9 +1164,9 @@ func GetRecoverArgs(job *Job) (*RecoverArgs, error) { // PlacementPolicyArgs is the argument for create/alter/drop placement policy type PlacementPolicyArgs struct { - Policy *PolicyInfo `json:"policy,omitempty"` - ReplaceOnExist bool `json:"replace_on_exist,omitempty"` - PolicyName pmodel.CIStr `json:"policy_name,omitempty"` + Policy *PolicyInfo `json:"policy,omitempty"` + ReplaceOnExist bool `json:"replace_on_exist,omitempty"` + PolicyName ast.CIStr `json:"policy_name,omitempty"` // it's set for alter/drop policy in v2 PolicyID int64 `json:"policy_id"` @@ -1313,7 +1312,7 @@ type IndexArg struct { // Global is never used, we only use Global in IndexOption. Can be deprecated later. Global bool `json:"-"` Unique bool `json:"unique,omitempty"` - IndexName pmodel.CIStr `json:"index_name,omitempty"` + IndexName ast.CIStr `json:"index_name,omitempty"` IndexPartSpecifications []*ast.IndexPartSpecification `json:"index_part_specifications"` IndexOption *ast.IndexOption `json:"index_option,omitempty"` HiddenCols []*ColumnInfo `json:"hidden_cols,omitempty"` @@ -1367,7 +1366,7 @@ func (a *ModifyIndexArgs) getArgsV1(job *Job) []any { if len(a.IndexArgs) == 1 { return []any{a.IndexArgs[0].IndexName, a.IndexArgs[0].IfExist} } - indexNames := make([]pmodel.CIStr, len(a.IndexArgs)) + indexNames := make([]ast.CIStr, len(a.IndexArgs)) ifExists := make([]bool, len(a.IndexArgs)) for i, idxArg := range a.IndexArgs { indexNames[i] = idxArg.IndexName @@ -1397,7 +1396,7 @@ func (a *ModifyIndexArgs) getArgsV1(job *Job) []any { // Add index n := len(a.IndexArgs) unique := make([]bool, n) - indexName := make([]pmodel.CIStr, n) + indexName := make([]ast.CIStr, n) indexPartSpecification := make([][]*ast.IndexPartSpecification, n) indexOption := make([]*ast.IndexOption, n) hiddenCols := make([][]*ColumnInfo, n) @@ -1438,7 +1437,7 @@ func (a *ModifyIndexArgs) decodeV1(job *Job) error { } func (a *ModifyIndexArgs) decodeRenameIndexV1(job *Job) error { - var from, to pmodel.CIStr + var from, to ast.CIStr if err := job.decodeArgs(&from, &to); err != nil { return errors.Trace(err) } @@ -1450,7 +1449,7 @@ func (a *ModifyIndexArgs) decodeRenameIndexV1(job *Job) error { } func (a *ModifyIndexArgs) decodeDropIndexV1(job *Job) error { - indexNames := make([]pmodel.CIStr, 1) + indexNames := make([]ast.CIStr, 1) ifExists := make([]bool, 1) if err := job.decodeArgs(&indexNames[0], &ifExists[0]); err != nil { if err = job.decodeArgs(&indexNames, &ifExists); err != nil { @@ -1470,7 +1469,7 @@ func (a *ModifyIndexArgs) decodeDropIndexV1(job *Job) error { func (a *ModifyIndexArgs) decodeAddIndexV1(job *Job) error { uniques := make([]bool, 1) - indexNames := make([]pmodel.CIStr, 1) + indexNames := make([]ast.CIStr, 1) indexPartSpecifications := make([][]*ast.IndexPartSpecification, 1) indexOptions := make([]*ast.IndexOption, 1) hiddenCols := make([][]*ColumnInfo, 1) @@ -1513,7 +1512,7 @@ func (a *ModifyIndexArgs) decodeAddPrimaryKeyV1(job *Job) error { func (a *ModifyIndexArgs) decodeAddVectorIndexV1(job *Job) error { var ( - indexName pmodel.CIStr + indexName ast.CIStr indexPartSpecification *ast.IndexPartSpecification indexOption *ast.IndexOption funcExpr string @@ -1557,7 +1556,7 @@ func (a *ModifyIndexArgs) getFinishedArgsV1(job *Job) []any { // 1. For drop index, arguments are [CIStr, bool, int64, []int64, bool]. // 3. For rollback add index, arguments are [[]CIStr, []bool, []int64]. if a.OpType == OpRollbackAddIndex { - indexNames := make([]pmodel.CIStr, len(a.IndexArgs)) + indexNames := make([]ast.CIStr, len(a.IndexArgs)) ifExists := make([]bool, len(a.IndexArgs)) for i, idxArg := range a.IndexArgs { indexNames[i] = idxArg.IndexName @@ -1571,7 +1570,7 @@ func (a *ModifyIndexArgs) getFinishedArgsV1(job *Job) []any { } // GetRenameIndexes get name of renamed index. -func (a *ModifyIndexArgs) GetRenameIndexes() (from, to pmodel.CIStr) { +func (a *ModifyIndexArgs) GetRenameIndexes() (from, to ast.CIStr) { from, to = a.IndexArgs[0].IndexName, a.IndexArgs[1].IndexName return } @@ -1607,7 +1606,7 @@ func GetFinishedModifyIndexArgs(job *Job) (*ModifyIndexArgs, error) { } if job.IsRollingback() || job.Type == ActionDropIndex || job.Type == ActionDropPrimaryKey { - indexNames := make([]pmodel.CIStr, 1) + indexNames := make([]ast.CIStr, 1) ifExists := make([]bool, 1) indexIDs := make([]int64, 1) var partitionIDs []int64 @@ -1670,7 +1669,7 @@ func GetFinishedModifyIndexArgs(job *Job) (*ModifyIndexArgs, error) { // ModifyColumnArgs is the argument for modify column. type ModifyColumnArgs struct { Column *ColumnInfo `json:"column,omitempty"` - OldColumnName pmodel.CIStr `json:"old_column_name,omitempty"` + OldColumnName ast.CIStr `json:"old_column_name,omitempty"` Position *ast.ColumnPosition `json:"position,omitempty"` ModifyColumnType byte `json:"modify_column_type,omitempty"` NewShardBits uint64 `json:"new_shard_bits,omitempty"` diff --git a/pkg/meta/model/job_args_test.go b/pkg/meta/model/job_args_test.go index df2e6cd9c29d6..2a49c0b1d78e8 100644 --- a/pkg/meta/model/job_args_test.go +++ b/pkg/meta/model/job_args_test.go @@ -19,7 +19,6 @@ import ( "testing" "github.com/pingcap/tidb/pkg/parser/ast" - "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/stretchr/testify/require" pdhttp "github.com/tikv/pd/client/http" @@ -207,8 +206,8 @@ func TestBatchCreateTableArgs(t *testing.T) { func TestDropTableArgs(t *testing.T) { inArgs := &DropTableArgs{ Identifiers: []ast.Ident{ - {Schema: model.NewCIStr("db"), Name: model.NewCIStr("tbl")}, - {Schema: model.NewCIStr("db2"), Name: model.NewCIStr("tbl2")}, + {Schema: ast.NewCIStr("db"), Name: ast.NewCIStr("tbl")}, + {Schema: ast.NewCIStr("db2"), Name: ast.NewCIStr("tbl2")}, }, FKCheck: true, } @@ -289,9 +288,9 @@ func TestTruncateTableArgs(t *testing.T) { func TestTablePartitionArgs(t *testing.T) { inArgs := &TablePartitionArgs{ PartNames: []string{"a", "b"}, - PartInfo: &PartitionInfo{Type: model.PartitionTypeRange, Definitions: []PartitionDefinition{ - {ID: 1, Name: model.NewCIStr("a"), LessThan: []string{"1"}}, - {ID: 2, Name: model.NewCIStr("b"), LessThan: []string{"2"}}, + PartInfo: &PartitionInfo{Type: ast.PartitionTypeRange, Definitions: []PartitionDefinition{ + {ID: 1, Name: ast.NewCIStr("a"), LessThan: []string{"1"}}, + {ID: 2, Name: ast.NewCIStr("b"), LessThan: []string{"2"}}, }}, } for _, tp := range []ActionType{ @@ -341,9 +340,9 @@ func TestTablePartitionArgs(t *testing.T) { FillRollbackArgsForAddPartition(j, &TablePartitionArgs{ PartNames: partNames, - PartInfo: &PartitionInfo{Type: model.PartitionTypeRange, Definitions: []PartitionDefinition{ - {ID: 1, Name: model.NewCIStr("aaaa"), LessThan: []string{"1"}}, - {ID: 2, Name: model.NewCIStr("bbb"), LessThan: []string{"2"}}, + PartInfo: &PartitionInfo{Type: ast.PartitionTypeRange, Definitions: []PartitionDefinition{ + {ID: 1, Name: ast.NewCIStr("aaaa"), LessThan: []string{"1"}}, + {ID: 2, Name: ast.NewCIStr("bbb"), LessThan: []string{"2"}}, }}, }) require.Len(t, j.args, 1) @@ -449,8 +448,8 @@ func TestAlterTablePartitionArgs(t *testing.T) { func TestRenameTableArgs(t *testing.T) { inArgs := &RenameTableArgs{ OldSchemaID: 9527, - OldSchemaName: model.NewCIStr("old_schema_name"), - NewTableName: model.NewCIStr("new_table_name"), + OldSchemaName: ast.NewCIStr("old_schema_name"), + NewTableName: ast.NewCIStr("new_table_name"), } jobvers := []JobVersion{JobVersion1, JobVersion2} @@ -468,11 +467,11 @@ func TestRenameTableArgs(t *testing.T) { func TestGetRenameTablesArgs(t *testing.T) { inArgs := &RenameTablesArgs{ RenameTableInfos: []*RenameTableArgs{ - {OldSchemaID: 1, OldSchemaName: model.CIStr{O: "db1", L: "db1"}, - NewTableName: model.CIStr{O: "tb3", L: "tb3"}, OldTableName: model.CIStr{O: "tb1", L: "tb1"}, + {OldSchemaID: 1, OldSchemaName: ast.CIStr{O: "db1", L: "db1"}, + NewTableName: ast.CIStr{O: "tb3", L: "tb3"}, OldTableName: ast.CIStr{O: "tb1", L: "tb1"}, NewSchemaID: 3, TableID: 100}, - {OldSchemaID: 2, OldSchemaName: model.CIStr{O: "db2", L: "db2"}, - NewTableName: model.CIStr{O: "tb2", L: "tb2"}, OldTableName: model.CIStr{O: "tb4", L: "tb4"}, + {OldSchemaID: 2, OldSchemaName: ast.CIStr{O: "db2", L: "db2"}, + NewTableName: ast.CIStr{O: "tb2", L: "tb2"}, OldTableName: ast.CIStr{O: "tb4", L: "tb4"}, NewSchemaID: 3, TableID: 101}, }, } @@ -489,7 +488,7 @@ func TestGetRenameTablesArgs(t *testing.T) { func TestResourceGroupArgs(t *testing.T) { inArgs := &ResourceGroupArgs{ - RGInfo: &ResourceGroupInfo{ID: 100, Name: model.NewCIStr("rg_name")}, + RGInfo: &ResourceGroupInfo{ID: 100, Name: ast.NewCIStr("rg_name")}, } for _, tp := range []ActionType{ActionCreateResourceGroup, ActionAlterResourceGroup, ActionDropResourceGroup} { for _, v := range []JobVersion{JobVersion1, JobVersion2} { @@ -509,8 +508,8 @@ func TestResourceGroupArgs(t *testing.T) { func TestGetAlterSequenceArgs(t *testing.T) { inArgs := &AlterSequenceArgs{ Ident: ast.Ident{ - Schema: model.NewCIStr("test_db"), - Name: model.NewCIStr("test_t"), + Schema: ast.NewCIStr("test_db"), + Name: ast.NewCIStr("test_t"), }, SeqOptions: []*ast.SequenceOption{ { @@ -563,7 +562,7 @@ func TestGetModifyTableCommentArgs(t *testing.T) { func TestGetAlterIndexVisibilityArgs(t *testing.T) { inArgs := &AlterIndexVisibilityArgs{ - IndexName: model.NewCIStr("index-name"), + IndexName: ast.NewCIStr("index-name"), Invisible: true, } @@ -580,7 +579,7 @@ func TestGetAddForeignKeyArgs(t *testing.T) { inArgs := &AddForeignKeyArgs{ FkInfo: &FKInfo{ ID: 7527, - Name: model.NewCIStr("fk-name"), + Name: ast.NewCIStr("fk-name"), }, FkCheck: true, } @@ -623,7 +622,7 @@ func TestGetShardRowIDArgs(t *testing.T) { func TestGetDropForeignKeyArgs(t *testing.T) { inArgs := &DropForeignKeyArgs{ - FkName: model.NewCIStr("fk-name"), + FkName: ast.NewCIStr("fk-name"), } for _, v := range []JobVersion{JobVersion1, JobVersion2} { @@ -640,7 +639,7 @@ func TestGetAlterTTLInfoArgs(t *testing.T) { ttlCronJobSchedule := "ttl-schedule" inArgs := &AlterTTLInfoArgs{ TTLInfo: &TTLInfo{ - ColumnName: model.NewCIStr("column_name"), + ColumnName: ast.NewCIStr("column_name"), IntervalExprStr: "1", IntervalTimeUnit: 10010, }, @@ -659,8 +658,8 @@ func TestGetAlterTTLInfoArgs(t *testing.T) { func TestAddCheckConstraintArgs(t *testing.T) { Constraint := &ConstraintInfo{ - Name: model.NewCIStr("t3_c1"), - Table: model.NewCIStr("t3"), + Name: ast.NewCIStr("t3_c1"), + Table: ast.NewCIStr("t3"), ExprString: "id<10", State: StateDeleteOnly, } @@ -681,7 +680,7 @@ func TestAddCheckConstraintArgs(t *testing.T) { func TestCheckConstraintArgs(t *testing.T) { inArgs := &CheckConstraintArgs{ - ConstraintName: model.NewCIStr("c1"), + ConstraintName: ast.NewCIStr("c1"), Enforced: true, } for _, v := range []JobVersion{JobVersion1, JobVersion2} { @@ -698,7 +697,7 @@ func TestGetAlterTablePlacementArgs(t *testing.T) { inArgs := &AlterTablePlacementArgs{ PlacementPolicyRef: &PolicyRefInfo{ ID: 7527, - Name: model.NewCIStr("placement-policy"), + Name: ast.NewCIStr("placement-policy"), }, } for _, v := range []JobVersion{JobVersion1, JobVersion2} { @@ -754,8 +753,8 @@ func TestGetUpdateTiFlashReplicaStatusArgs(t *testing.T) { } func TestLockTableArgs(t *testing.T) { inArgs := &LockTablesArgs{ - LockTables: []TableLockTpInfo{{1, 1, model.TableLockNone}}, - UnlockTables: []TableLockTpInfo{{2, 2, model.TableLockNone}}, + LockTables: []TableLockTpInfo{{1, 1, ast.TableLockNone}}, + UnlockTables: []TableLockTpInfo{{2, 2, ast.TableLockNone}}, IndexOfLock: 13, IndexOfUnlock: 24, } @@ -775,7 +774,7 @@ func TestLockTableArgs(t *testing.T) { } func TestRepairTableArgs(t *testing.T) { - inArgs := &RepairTableArgs{&TableInfo{ID: 1, Name: model.NewCIStr("t")}} + inArgs := &RepairTableArgs{&TableInfo{ID: 1, Name: ast.NewCIStr("t")}} for _, v := range []JobVersion{JobVersion1, JobVersion2} { j2 := &Job{} require.NoError(t, j2.Decode(getJobBytes(t, inArgs, v, ActionRepairTable))) @@ -792,7 +791,7 @@ func TestRecoverArgs(t *testing.T) { DropJobID: 2, TableInfo: &TableInfo{ ID: 100, - Name: model.NewCIStr("table"), + Name: ast.NewCIStr("table"), }, OldSchemaName: "old", OldTableName: "table", @@ -820,8 +819,8 @@ func TestRecoverArgs(t *testing.T) { func TestPlacementPolicyArgs(t *testing.T) { inArgs := &PlacementPolicyArgs{ - Policy: &PolicyInfo{ID: 1, Name: model.NewCIStr("policy"), State: StateDeleteOnly}, - PolicyName: model.NewCIStr("policy_name"), + Policy: &PolicyInfo{ID: 1, Name: ast.NewCIStr("policy"), State: StateDeleteOnly}, + PolicyName: ast.NewCIStr("policy_name"), PolicyID: 123, ReplaceOnExist: false, } @@ -850,7 +849,7 @@ func TestGetSetDefaultValueArgs(t *testing.T) { inArgs := &SetDefaultValueArgs{ Col: &ColumnInfo{ ID: 7527, - Name: model.NewCIStr("col_name"), + Name: ast.NewCIStr("col_name"), }, } for _, v := range []JobVersion{JobVersion1, JobVersion2} { @@ -891,7 +890,7 @@ func TestFlashbackClusterArgs(t *testing.T) { func TestDropColumnArgs(t *testing.T) { inArgs := &TableColumnArgs{ Col: &ColumnInfo{ - Name: model.NewCIStr("col_name"), + Name: ast.NewCIStr("col_name"), }, IgnoreExistenceErr: true, IndexIDs: []int64{1, 2, 3}, @@ -912,7 +911,7 @@ func TestAddColumnArgs(t *testing.T) { inArgs := &TableColumnArgs{ Col: &ColumnInfo{ ID: 7527, - Name: model.NewCIStr("col_name"), + Name: ast.NewCIStr("col_name"), }, Pos: &ast.ColumnPosition{ Tp: ast.ColumnPositionFirst, @@ -922,7 +921,7 @@ func TestAddColumnArgs(t *testing.T) { } dropArgs := &TableColumnArgs{ Col: &ColumnInfo{ - Name: model.NewCIStr("drop_column"), + Name: ast.NewCIStr("drop_column"), }, Pos: &ast.ColumnPosition{}, } @@ -969,7 +968,7 @@ func TestAddIndexArgs(t *testing.T) { IndexArgs: []*IndexArg{{ Global: false, Unique: true, - IndexName: model.NewCIStr("idx1"), + IndexName: ast.NewCIStr("idx1"), IndexPartSpecifications: []*ast.IndexPartSpecification{{Length: 2}}, IndexOption: &ast.IndexOption{}, HiddenCols: []*ColumnInfo{{}, {}}, @@ -1074,7 +1073,7 @@ func TestDropIndexArguements(t *testing.T) { inArgs := &ModifyIndexArgs{ IndexArgs: []*IndexArg{ { - IndexName: model.NewCIStr("i2"), + IndexName: ast.NewCIStr("i2"), IfExist: true, IsVector: true, IndexID: 1, @@ -1089,8 +1088,8 @@ func TestDropIndexArguements(t *testing.T) { func TestGetRenameIndexArgs(t *testing.T) { inArgs := &ModifyIndexArgs{ IndexArgs: []*IndexArg{ - {IndexName: model.NewCIStr("old")}, - {IndexName: model.NewCIStr("new")}, + {IndexName: ast.NewCIStr("old")}, + {IndexName: ast.NewCIStr("new")}, }, } for _, v := range []JobVersion{JobVersion1, JobVersion2} { @@ -1105,12 +1104,12 @@ func TestGetRenameIndexArgs(t *testing.T) { func TestModifyColumnsArgs(t *testing.T) { inArgs := &ModifyColumnArgs{ - Column: &ColumnInfo{ID: 111, Name: model.NewCIStr("col1")}, - OldColumnName: model.NewCIStr("aa"), + Column: &ColumnInfo{ID: 111, Name: ast.NewCIStr("col1")}, + OldColumnName: ast.NewCIStr("aa"), Position: &ast.ColumnPosition{Tp: ast.ColumnPositionFirst}, ModifyColumnType: 1, NewShardBits: 123, - ChangingColumn: &ColumnInfo{ID: 222, Name: model.NewCIStr("col2")}, + ChangingColumn: &ColumnInfo{ID: 222, Name: ast.NewCIStr("col2")}, RedundantIdxs: []int64{1, 2}, IndexIDs: []int64{3, 4}, PartitionIDs: []int64{5, 6}, diff --git a/pkg/meta/model/job_test.go b/pkg/meta/model/job_test.go index a0eed61aba9d3..33f8a96c485b6 100644 --- a/pkg/meta/model/job_test.go +++ b/pkg/meta/model/job_test.go @@ -21,7 +21,7 @@ import ( "time" "unsafe" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/stretchr/testify/require" ) @@ -63,9 +63,9 @@ func TestJobCodec(t *testing.T) { Location: &TimeZoneLocation{Name: tzName, Offset: tzOffset}, }, } - job.FillArgs(&RenameTableArgs{OldSchemaID: 2, NewTableName: model.NewCIStr("table1")}) - job.BinlogInfo.AddDBInfo(123, &DBInfo{ID: 1, Name: model.NewCIStr("test_history_db")}) - job.BinlogInfo.AddTableInfo(123, &TableInfo{ID: 1, Name: model.NewCIStr("test_history_tbl")}) + job.FillArgs(&RenameTableArgs{OldSchemaID: 2, NewTableName: ast.NewCIStr("table1")}) + job.BinlogInfo.AddDBInfo(123, &DBInfo{ID: 1, Name: ast.NewCIStr("test_history_db")}) + job.BinlogInfo.AddTableInfo(123, &TableInfo{ID: 1, Name: ast.NewCIStr("test_history_tbl")}) require.Equal(t, false, job.IsCancelled()) b, err := job.Encode(false) diff --git a/pkg/meta/model/placement.go b/pkg/meta/model/placement.go index 5ef1ee42667ae..c6458777ba668 100644 --- a/pkg/meta/model/placement.go +++ b/pkg/meta/model/placement.go @@ -19,13 +19,13 @@ import ( "strings" "time" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" ) // PolicyRefInfo is the struct to refer the placement policy. type PolicyRefInfo struct { - ID int64 `json:"id"` - Name model.CIStr `json:"name"` + ID int64 `json:"id"` + Name ast.CIStr `json:"name"` } // PlacementSettings is the settings of the placement @@ -131,7 +131,7 @@ func writeSettingItemToBuilder(sb *strings.Builder, item string, separatorFns .. type PolicyInfo struct { *PlacementSettings ID int64 `json:"id"` - Name model.CIStr `json:"name"` + Name ast.CIStr `json:"name"` State SchemaState `json:"state"` } diff --git a/pkg/meta/model/placement_test.go b/pkg/meta/model/placement_test.go index fe5908a2f9efc..5e2b91bed5170 100644 --- a/pkg/meta/model/placement_test.go +++ b/pkg/meta/model/placement_test.go @@ -17,7 +17,7 @@ package model import ( "testing" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/stretchr/testify/require" ) @@ -76,12 +76,12 @@ func TestPlacementPolicyClone(t *testing.T) { } clonedPolicy := policy.Clone() clonedPolicy.ID = 100 - clonedPolicy.Name = model.NewCIStr("p2") + clonedPolicy.Name = ast.NewCIStr("p2") clonedPolicy.State = StateDeleteOnly clonedPolicy.PlacementSettings.Followers = 10 require.Equal(t, int64(0), policy.ID) - require.Equal(t, model.NewCIStr(""), policy.Name) + require.Equal(t, ast.NewCIStr(""), policy.Name) require.Equal(t, StateNone, policy.State) require.Equal(t, PlacementSettings{}, *(policy.PlacementSettings)) } diff --git a/pkg/meta/model/resource_group.go b/pkg/meta/model/resource_group.go index fede8cf3fb7ca..70fc303af6d3c 100644 --- a/pkg/meta/model/resource_group.go +++ b/pkg/meta/model/resource_group.go @@ -19,18 +19,18 @@ import ( "strings" "time" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" ) // ResourceGroupRunawaySettings is the runaway settings of the resource group type ResourceGroupRunawaySettings struct { - ExecElapsedTimeMs uint64 `json:"exec_elapsed_time_ms"` - ProcessedKeys int64 `json:"processed_keys"` - RequestUnit int64 `json:"request_unit"` - Action model.RunawayActionType `json:"action"` - SwitchGroupName string `json:"switch_group_name"` - WatchType model.RunawayWatchType `json:"watch_type"` - WatchDurationMs int64 `json:"watch_duration_ms"` + ExecElapsedTimeMs uint64 `json:"exec_elapsed_time_ms"` + ProcessedKeys int64 `json:"processed_keys"` + RequestUnit int64 `json:"request_unit"` + Action ast.RunawayActionType `json:"action"` + SwitchGroupName string `json:"switch_group_name"` + WatchType ast.RunawayWatchType `json:"watch_type"` + WatchDurationMs int64 `json:"watch_duration_ms"` } // ResourceGroupBackgroundSettings is the background settings of the resource group. @@ -55,7 +55,7 @@ type ResourceGroupSettings struct { func NewResourceGroupSettings() *ResourceGroupSettings { return &ResourceGroupSettings{ RURate: 0, - Priority: model.MediumPriorityValue, + Priority: ast.MediumPriorityValue, CPULimiter: "", IOReadBandwidth: "", IOWriteBandwidth: "", @@ -72,7 +72,7 @@ func (p *ResourceGroupSettings) String() string { if p.RURate != 0 { writeSettingIntegerToBuilder(sb, "RU_PER_SEC", p.RURate, separatorFn) } - writeSettingItemToBuilder(sb, "PRIORITY="+model.PriorityValueToName(p.Priority), separatorFn) + writeSettingItemToBuilder(sb, "PRIORITY="+ast.PriorityValueToName(p.Priority), separatorFn) if len(p.CPULimiter) > 0 { writeSettingStringToBuilder(sb, "CPU", p.CPULimiter, separatorFn) } @@ -108,12 +108,12 @@ func (p *ResourceGroupSettings) String() string { fmt.Fprintf(sb, "RU=%d", p.Runaway.RequestUnit) } // action settings - if p.Runaway.Action == model.RunawayActionSwitchGroup { + if p.Runaway.Action == ast.RunawayActionSwitchGroup { writeSettingItemToBuilder(sb, fmt.Sprintf("ACTION=%s(%s)", p.Runaway.Action.String(), p.Runaway.SwitchGroupName)) } else { writeSettingItemToBuilder(sb, "ACTION="+p.Runaway.Action.String()) } - if p.Runaway.WatchType != model.WatchNone { + if p.Runaway.WatchType != ast.WatchNone { writeSettingItemToBuilder(sb, "WATCH="+p.Runaway.WatchType.String()) if p.Runaway.WatchDurationMs > 0 { writeSettingDurationToBuilder(sb, "DURATION", time.Duration(p.Runaway.WatchDurationMs)*time.Millisecond) @@ -160,7 +160,7 @@ func (p *ResourceGroupSettings) Clone() *ResourceGroupSettings { type ResourceGroupInfo struct { *ResourceGroupSettings ID int64 `json:"id"` - Name model.CIStr `json:"name"` + Name ast.CIStr `json:"name"` State SchemaState `json:"state"` } diff --git a/pkg/meta/model/table.go b/pkg/meta/model/table.go index d2d7fc21d8e9b..0a34903ec65b2 100644 --- a/pkg/meta/model/table.go +++ b/pkg/meta/model/table.go @@ -23,9 +23,9 @@ import ( "unsafe" "github.com/pingcap/failpoint" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/auth" "github.com/pingcap/tidb/pkg/parser/duration" - "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/cascades/base" ) @@ -83,20 +83,20 @@ const ( ) // ExtraHandleName is the name of ExtraHandle Column. -var ExtraHandleName = model.NewCIStr("_tidb_rowid") +var ExtraHandleName = ast.NewCIStr("_tidb_rowid") // ExtraPhysTblIDName is the name of ExtraPhysTblID Column. -var ExtraPhysTblIDName = model.NewCIStr("_tidb_tid") +var ExtraPhysTblIDName = ast.NewCIStr("_tidb_tid") // Deprecated: Use ExtraPhysTblIDName instead. // var ExtraPartitionIdName = NewCIStr("_tidb_pid") //nolint:revive // TableInfo provides meta data describing a DB table. type TableInfo struct { - ID int64 `json:"id"` - Name model.CIStr `json:"name"` - Charset string `json:"charset"` - Collate string `json:"collate"` + ID int64 `json:"id"` + Name ast.CIStr `json:"name"` + Charset string `json:"charset"` + Collate string `json:"collate"` // Columns are listed in the order in which they appear in the schema. Columns []*ColumnInfo `json:"cols"` Indices []*IndexInfo `json:"index_info"` @@ -267,13 +267,13 @@ func (t *TableInfo) Clone() *TableInfo { } // GetPkName will return the pk name if pk exists. -func (t *TableInfo) GetPkName() model.CIStr { +func (t *TableInfo) GetPkName() ast.CIStr { for _, colInfo := range t.Columns { if mysql.HasPriKeyFlag(colInfo.GetFlag()) { return colInfo.Name } } - return model.CIStr{} + return ast.CIStr{} } // GetPkColInfo gets the ColumnInfo of pk if exists. @@ -558,8 +558,8 @@ func FindFKInfoByName(fks []*FKInfo, name string) *FKInfo { // TableNameInfo provides meta data describing a table name info. type TableNameInfo struct { - ID int64 `json:"id"` - Name model.CIStr `json:"name"` + ID int64 `json:"id"` + Name ast.CIStr `json:"name"` } // TableCacheStatusType is the type of the table cache status @@ -610,7 +610,7 @@ func (t TempTableType) String() string { // TableLockInfo provides meta data describing a table lock. type TableLockInfo struct { - Tp model.TableLockType + Tp ast.TableLockType // Use array because there may be multiple sessions holding the same read lock. Sessions []SessionInfo State TableLockState @@ -633,7 +633,7 @@ func (s SessionInfo) String() string { type TableLockTpInfo struct { SchemaID int64 TableID int64 - Tp model.TableLockType + Tp ast.TableLockType } // TableLockState is the state for table lock. @@ -680,12 +680,12 @@ func (tr *TiFlashReplicaInfo) IsPartitionAvailable(pid int64) bool { // ViewInfo provides meta data describing a DB view. type ViewInfo struct { - Algorithm model.ViewAlgorithm `json:"view_algorithm"` - Definer *auth.UserIdentity `json:"view_definer"` - Security model.ViewSecurity `json:"view_security"` - SelectStmt string `json:"view_select"` - CheckOption model.ViewCheckOption `json:"view_checkoption"` - Cols []model.CIStr `json:"view_cols"` + Algorithm ast.ViewAlgorithm `json:"view_algorithm"` + Definer *auth.UserIdentity `json:"view_definer"` + Security ast.ViewSecurity `json:"view_security"` + SelectStmt string `json:"view_select"` + CheckOption ast.ViewCheckOption `json:"view_checkoption"` + Cols []ast.CIStr `json:"view_cols"` } // Some constants for sequence. @@ -733,9 +733,9 @@ type UpdateIndexInfo struct { // PartitionInfo provides table partition info. type PartitionInfo struct { - Type model.PartitionType `json:"type"` - Expr string `json:"expr"` - Columns []model.CIStr `json:"columns"` + Type ast.PartitionType `json:"type"` + Expr string `json:"expr"` + Columns []ast.CIStr `json:"columns"` // User may already create table with partition but table partition is not // yet supported back then. When Enable is true, write/read need use tid @@ -770,9 +770,9 @@ type PartitionInfo struct { NewTableID int64 `json:"new_table_id,omitempty"` // Set during ALTER TABLE ... PARTITION BY ... // First as the new partition scheme, then in StateDeleteReorg as the old - DDLType model.PartitionType `json:"ddl_type,omitempty"` - DDLExpr string `json:"ddl_expr,omitempty"` - DDLColumns []model.CIStr `json:"ddl_columns,omitempty"` + DDLType ast.PartitionType `json:"ddl_type,omitempty"` + DDLExpr string `json:"ddl_expr,omitempty"` + DDLColumns []ast.CIStr `json:"ddl_columns,omitempty"` // For ActionAlterTablePartitioning, UPDATE INDEXES DDLUpdateIndexes []UpdateIndexInfo `json:"ddl_update_indexes,omitempty"` // Simplified way to handle Global Index changes, instead of calculating @@ -785,7 +785,7 @@ type PartitionInfo struct { // Clone clones itself. func (pi *PartitionInfo) Clone() *PartitionInfo { newPi := *pi - newPi.Columns = make([]model.CIStr, len(pi.Columns)) + newPi.Columns = make([]ast.CIStr, len(pi.Columns)) copy(newPi.Columns, pi.Columns) newPi.Definitions = make([]PartitionDefinition, len(pi.Definitions)) @@ -870,7 +870,7 @@ func (pi *PartitionInfo) GCPartitionStates() { func (pi *PartitionInfo) ClearReorgIntermediateInfo() { pi.DDLAction = ActionNone pi.DDLState = StateNone - pi.DDLType = model.PartitionTypeNone + pi.DDLType = ast.PartitionTypeNone pi.DDLExpr = "" pi.DDLColumns = nil pi.NewTableID = 0 @@ -903,7 +903,7 @@ func (pi *PartitionInfo) GetPartitionIDByName(partitionDefinitionName string) in // GetDefaultListPartition return the index of Definitions // that contains the LIST Default partition otherwise it returns -1 func (pi *PartitionInfo) GetDefaultListPartition() int { - if pi.Type != model.PartitionTypeList { + if pi.Type != ast.PartitionTypeList { return -1 } defs := pi.Definitions @@ -982,7 +982,7 @@ func (pi *PartitionInfo) GetOverlappingDroppingPartitionIdx(idx int) int { } if pi.CanHaveOverlappingDroppingPartition() { switch pi.Type { - case model.PartitionTypeRange: + case ast.PartitionTypeRange: for i := idx; i < len(pi.Definitions); i++ { if pi.IsDropping(i) { continue @@ -991,7 +991,7 @@ func (pi *PartitionInfo) GetOverlappingDroppingPartitionIdx(idx int) int { } // Last partition is also dropped! return -1 - case model.PartitionTypeList: + case ast.PartitionTypeList: if pi.IsDropping(idx) { defaultIdx := pi.GetDefaultListPartition() if defaultIdx == idx { @@ -1095,7 +1095,7 @@ type PartitionState struct { // PartitionDefinition defines a single partition. type PartitionDefinition struct { ID int64 `json:"id"` - Name model.CIStr `json:"name"` + Name ast.CIStr `json:"name"` LessThan []string `json:"less_than"` InValues [][]string `json:"in_values"` PlacementPolicyRef *PolicyRefInfo `json:"policy_ref_info"` @@ -1136,37 +1136,37 @@ func (ci *PartitionDefinition) MemoryUsage() (sum int64) { // ConstraintInfo provides meta data describing check-expression constraint. type ConstraintInfo struct { - ID int64 `json:"id"` - Name model.CIStr `json:"constraint_name"` - Table model.CIStr `json:"tbl_name"` // Table name. - ConstraintCols []model.CIStr `json:"constraint_cols"` // Depended column names. - Enforced bool `json:"enforced"` - InColumn bool `json:"in_column"` // Indicate whether the constraint is column type check. - ExprString string `json:"expr_string"` - State SchemaState `json:"state"` + ID int64 `json:"id"` + Name ast.CIStr `json:"constraint_name"` + Table ast.CIStr `json:"tbl_name"` // Table name. + ConstraintCols []ast.CIStr `json:"constraint_cols"` // Depended column names. + Enforced bool `json:"enforced"` + InColumn bool `json:"in_column"` // Indicate whether the constraint is column type check. + ExprString string `json:"expr_string"` + State SchemaState `json:"state"` } // Clone clones ConstraintInfo. func (ci *ConstraintInfo) Clone() *ConstraintInfo { nci := *ci - nci.ConstraintCols = make([]model.CIStr, len(ci.ConstraintCols)) + nci.ConstraintCols = make([]ast.CIStr, len(ci.ConstraintCols)) copy(nci.ConstraintCols, ci.ConstraintCols) return &nci } // FKInfo provides meta data describing a foreign key constraint. type FKInfo struct { - ID int64 `json:"id"` - Name model.CIStr `json:"fk_name"` - RefSchema model.CIStr `json:"ref_schema"` - RefTable model.CIStr `json:"ref_table"` - RefCols []model.CIStr `json:"ref_cols"` - Cols []model.CIStr `json:"cols"` - OnDelete int `json:"on_delete"` - OnUpdate int `json:"on_update"` - State SchemaState `json:"state"` - Version int `json:"version"` + ID int64 `json:"id"` + Name ast.CIStr `json:"fk_name"` + RefSchema ast.CIStr `json:"ref_schema"` + RefTable ast.CIStr `json:"ref_table"` + RefCols []ast.CIStr `json:"ref_cols"` + Cols []ast.CIStr `json:"cols"` + OnDelete int `json:"on_delete"` + OnUpdate int `json:"on_update"` + State SchemaState `json:"state"` + Version int `json:"version"` } // String returns the string representation of FKInfo. @@ -1195,11 +1195,11 @@ func (fk *FKInfo) String(db, tb string) string { buf.WriteString("`" + col.O + "`") } buf.WriteString(")") - if onDelete := model.ReferOptionType(fk.OnDelete); onDelete != model.ReferOptionNoOption { + if onDelete := ast.ReferOptionType(fk.OnDelete); onDelete != ast.ReferOptionNoOption { buf.WriteString(" ON DELETE ") buf.WriteString(onDelete.String()) } - if onUpdate := model.ReferOptionType(fk.OnUpdate); onUpdate != model.ReferOptionNoOption { + if onUpdate := ast.ReferOptionType(fk.OnUpdate); onUpdate != ast.ReferOptionNoOption { buf.WriteString(" ON UPDATE ") buf.WriteString(onUpdate.String()) } @@ -1210,8 +1210,8 @@ func (fk *FKInfo) String(db, tb string) string { func (fk *FKInfo) Clone() *FKInfo { nfk := *fk - nfk.RefCols = make([]model.CIStr, len(fk.RefCols)) - nfk.Cols = make([]model.CIStr, len(fk.Cols)) + nfk.RefCols = make([]ast.CIStr, len(fk.RefCols)) + nfk.Cols = make([]ast.CIStr, len(fk.Cols)) copy(nfk.RefCols, fk.RefCols) copy(nfk.Cols, fk.Cols) @@ -1229,10 +1229,10 @@ const ( // ReferredFKInfo provides the cited foreign key in the child table. type ReferredFKInfo struct { - Cols []model.CIStr `json:"cols"` - ChildSchema model.CIStr `json:"child_schema"` - ChildTable model.CIStr `json:"child_table"` - ChildFKName model.CIStr `json:"child_fk_name"` + Cols []ast.CIStr `json:"cols"` + ChildSchema ast.CIStr `json:"child_schema"` + ChildTable ast.CIStr `json:"child_table"` + ChildFKName ast.CIStr `json:"child_fk_name"` } // TableItemID is composed by table ID and column/index ID @@ -1262,22 +1262,22 @@ func (s StatsLoadItem) Key() string { // StatsOptions is the struct to store the stats options. type StatsOptions struct { *StatsWindowSettings - AutoRecalc bool `json:"auto_recalc"` - ColumnChoice model.ColumnChoice `json:"column_choice"` - ColumnList []model.CIStr `json:"column_list"` - SampleNum uint64 `json:"sample_num"` - SampleRate float64 `json:"sample_rate"` - Buckets uint64 `json:"buckets"` - TopN uint64 `json:"topn"` - Concurrency uint `json:"concurrency"` + AutoRecalc bool `json:"auto_recalc"` + ColumnChoice ast.ColumnChoice `json:"column_choice"` + ColumnList []ast.CIStr `json:"column_list"` + SampleNum uint64 `json:"sample_num"` + SampleRate float64 `json:"sample_rate"` + Buckets uint64 `json:"buckets"` + TopN uint64 `json:"topn"` + Concurrency uint `json:"concurrency"` } // NewStatsOptions creates a new StatsOptions. func NewStatsOptions() *StatsOptions { return &StatsOptions{ AutoRecalc: true, - ColumnChoice: model.DefaultChoice, - ColumnList: []model.CIStr{}, + ColumnChoice: ast.DefaultChoice, + ColumnList: []ast.CIStr{}, SampleNum: uint64(0), SampleRate: 0.0, Buckets: uint64(0), @@ -1330,8 +1330,8 @@ const OldDefaultTTLJobInterval = "1h" // TTLInfo records the TTL config type TTLInfo struct { - ColumnName model.CIStr `json:"column"` - IntervalExprStr string `json:"interval_expr"` + ColumnName ast.CIStr `json:"column"` + IntervalExprStr string `json:"interval_expr"` // `IntervalTimeUnit` is actually ast.TimeUnitType. Use `int` to avoid cycle dependency IntervalTimeUnit int `json:"interval_time_unit"` Enable bool `json:"enable"` diff --git a/pkg/meta/model/table_test.go b/pkg/meta/model/table_test.go index d51c9a915d167..b22647a62b10a 100644 --- a/pkg/meta/model/table_test.go +++ b/pkg/meta/model/table_test.go @@ -19,9 +19,9 @@ import ( "testing" "time" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/charset" "github.com/pingcap/tidb/pkg/parser/duration" - "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/types" "github.com/stretchr/testify/require" @@ -62,7 +62,7 @@ func TestMoveColumnInfo(t *testing.T) { tbl := &TableInfo{ ID: 1, - Name: model.NewCIStr("t"), + Name: ast.NewCIStr("t"), Columns: []*ColumnInfo{c0, c1, c2, c3, c4}, Indices: []*IndexInfo{i0, i1, i2, i3, i4}, } @@ -89,7 +89,7 @@ func TestMoveColumnInfo(t *testing.T) { func TestModelBasic(t *testing.T) { column := &ColumnInfo{ ID: 1, - Name: model.NewCIStr("c"), + Name: ast.NewCIStr("c"), Offset: 0, DefaultValue: 0, FieldType: *types.NewFieldType(0), @@ -98,11 +98,11 @@ func TestModelBasic(t *testing.T) { column.AddFlag(mysql.PriKeyFlag) index := &IndexInfo{ - Name: model.NewCIStr("key"), - Table: model.NewCIStr("t"), + Name: ast.NewCIStr("key"), + Table: ast.NewCIStr("t"), Columns: []*IndexColumn{ { - Name: model.NewCIStr("c"), + Name: ast.NewCIStr("c"), Offset: 0, Length: 10, }}, @@ -111,8 +111,8 @@ func TestModelBasic(t *testing.T) { } fk := &FKInfo{ - RefCols: []model.CIStr{model.NewCIStr("a")}, - Cols: []model.CIStr{model.NewCIStr("a")}, + RefCols: []ast.CIStr{ast.NewCIStr("a")}, + Cols: []ast.CIStr{ast.NewCIStr("a")}, } seq := &SequenceInfo{ @@ -123,7 +123,7 @@ func TestModelBasic(t *testing.T) { table := &TableInfo{ ID: 1, - Name: model.NewCIStr("t"), + Name: ast.NewCIStr("t"), Charset: "utf8", Collate: "utf8_bin", Columns: []*ColumnInfo{column}, @@ -134,13 +134,13 @@ func TestModelBasic(t *testing.T) { table2 := &TableInfo{ ID: 2, - Name: model.NewCIStr("s"), + Name: ast.NewCIStr("s"), Sequence: seq, } dbInfo := &DBInfo{ ID: 1, - Name: model.NewCIStr("test"), + Name: ast.NewCIStr("test"), Charset: "utf8", Collate: "utf8_bin", } @@ -150,15 +150,15 @@ func TestModelBasic(t *testing.T) { require.Equal(t, dbInfo, n) pkName := table.GetPkName() - require.Equal(t, model.NewCIStr("c"), pkName) + require.Equal(t, ast.NewCIStr("c"), pkName) newColumn := table.GetPkColInfo() require.Equal(t, true, newColumn.Hidden) require.Equal(t, column, newColumn) inIdx := table.ColumnIsInIndex(column) require.Equal(t, true, inIdx) - tp := model.IndexTypeBtree + tp := ast.IndexTypeBtree require.Equal(t, "BTREE", tp.String()) - tp = model.IndexTypeHash + tp = ast.IndexTypeHash require.Equal(t, "HASH", tp.String()) tp = 1e5 require.Equal(t, "", tp.String()) @@ -171,11 +171,11 @@ func TestModelBasic(t *testing.T) { // Corner cases column.ToggleFlag(mysql.PriKeyFlag) pkName = table.GetPkName() - require.Equal(t, model.NewCIStr(""), pkName) + require.Equal(t, ast.NewCIStr(""), pkName) newColumn = table.GetPkColInfo() require.Nil(t, newColumn) anCol := &ColumnInfo{ - Name: model.NewCIStr("d"), + Name: ast.NewCIStr("d"), } exIdx := table.ColumnIsInIndex(anCol) require.Equal(t, false, exIdx) @@ -193,14 +193,14 @@ func TestModelBasic(t *testing.T) { func TestTTLInfoClone(t *testing.T) { ttlInfo := &TTLInfo{ - ColumnName: model.NewCIStr("test"), + ColumnName: ast.NewCIStr("test"), IntervalExprStr: "test_expr", IntervalTimeUnit: 5, Enable: true, } clonedTTLInfo := ttlInfo.Clone() - clonedTTLInfo.ColumnName = model.NewCIStr("test_2") + clonedTTLInfo.ColumnName = ast.NewCIStr("test_2") clonedTTLInfo.IntervalExprStr = "test_expr_2" clonedTTLInfo.IntervalTimeUnit = 9 clonedTTLInfo.Enable = false @@ -226,12 +226,12 @@ func TestTTLJobInterval(t *testing.T) { func TestClearReorgIntermediateInfo(t *testing.T) { ptInfo := &PartitionInfo{} - ptInfo.DDLType = model.PartitionTypeHash + ptInfo.DDLType = ast.PartitionTypeHash ptInfo.DDLExpr = "Test DDL Expr" ptInfo.NewTableID = 1111 ptInfo.ClearReorgIntermediateInfo() - require.Equal(t, model.PartitionTypeNone, ptInfo.DDLType) + require.Equal(t, ast.PartitionTypeNone, ptInfo.DDLType) require.Equal(t, "", ptInfo.DDLExpr) require.Equal(t, true, ptInfo.DDLColumns == nil) require.Equal(t, int64(0), ptInfo.NewTableID) diff --git a/pkg/parser/BUILD.bazel b/pkg/parser/BUILD.bazel index c1229045532ff..b81fab9b917c6 100644 --- a/pkg/parser/BUILD.bazel +++ b/pkg/parser/BUILD.bazel @@ -25,7 +25,6 @@ go_library( "//pkg/parser/auth", "//pkg/parser/charset", "//pkg/parser/duration", - "//pkg/parser/model", "//pkg/parser/mysql", "//pkg/parser/opcode", "//pkg/parser/terror", @@ -56,7 +55,6 @@ go_test( "//pkg/parser/ast", "//pkg/parser/charset", "//pkg/parser/format", - "//pkg/parser/model", "//pkg/parser/mysql", "//pkg/parser/opcode", "//pkg/parser/terror", diff --git a/pkg/parser/ast/BUILD.bazel b/pkg/parser/ast/BUILD.bazel index 2e763a8abf8c0..e0291f44d6ce7 100644 --- a/pkg/parser/ast/BUILD.bazel +++ b/pkg/parser/ast/BUILD.bazel @@ -11,6 +11,7 @@ go_library( "flag.go", "functions.go", "misc.go", + "model.go", "procedure.go", "stats.go", "util.go", @@ -21,7 +22,6 @@ go_library( "//pkg/parser/auth", "//pkg/parser/charset", "//pkg/parser/format", - "//pkg/parser/model", "//pkg/parser/mysql", "//pkg/parser/opcode", "//pkg/parser/terror", @@ -44,6 +44,7 @@ go_test( "format_test.go", "functions_test.go", "misc_test.go", + "model_test.go", "procedure_test.go", "util_test.go", ], diff --git a/pkg/parser/ast/ddl.go b/pkg/parser/ast/ddl.go index 753126dbca540..1feaec655bdca 100644 --- a/pkg/parser/ast/ddl.go +++ b/pkg/parser/ast/ddl.go @@ -20,7 +20,6 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/parser/auth" "github.com/pingcap/tidb/pkg/parser/format" - "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/parser/tidb" @@ -140,7 +139,7 @@ type CreateDatabaseStmt struct { ddlNode IfNotExists bool - Name model.CIStr + Name CIStr Options []*DatabaseOption } @@ -176,7 +175,7 @@ func (n *CreateDatabaseStmt) Accept(v Visitor) (Node, bool) { type AlterDatabaseStmt struct { ddlNode - Name model.CIStr + Name CIStr AlterDefaultDatabase bool Options []*DatabaseOption } @@ -238,7 +237,7 @@ type DropDatabaseStmt struct { ddlNode IfExists bool - Name model.CIStr + Name CIStr } // Restore implements Node interface. @@ -265,7 +264,7 @@ func (n *DropDatabaseStmt) Accept(v Visitor) (Node, bool) { type FlashBackDatabaseStmt struct { ddlNode - DBName model.CIStr + DBName CIStr NewName string } @@ -405,13 +404,13 @@ func (n *ReferenceDef) Restore(ctx *format.RestoreCtx) error { ctx.WriteKeyWord("SIMPLE") } } - if n.OnDelete.ReferOpt != model.ReferOptionNoOption { + if n.OnDelete.ReferOpt != ReferOptionNoOption { ctx.WritePlain(" ") if err := n.OnDelete.Restore(ctx); err != nil { return errors.Annotate(err, "An error occurred while splicing OnDelete") } } - if n.OnUpdate.ReferOpt != model.ReferOptionNoOption { + if n.OnUpdate.ReferOpt != ReferOptionNoOption { ctx.WritePlain(" ") if err := n.OnUpdate.Restore(ctx); err != nil { return errors.Annotate(err, "An error occurred while splicing OnUpdate") @@ -457,12 +456,12 @@ func (n *ReferenceDef) Accept(v Visitor) (Node, bool) { // OnDeleteOpt is used for optional on delete clause. type OnDeleteOpt struct { node - ReferOpt model.ReferOptionType + ReferOpt ReferOptionType } // Restore implements Node interface. func (n *OnDeleteOpt) Restore(ctx *format.RestoreCtx) error { - if n.ReferOpt != model.ReferOptionNoOption { + if n.ReferOpt != ReferOptionNoOption { ctx.WriteKeyWord("ON DELETE ") ctx.WriteKeyWord(n.ReferOpt.String()) } @@ -482,12 +481,12 @@ func (n *OnDeleteOpt) Accept(v Visitor) (Node, bool) { // OnUpdateOpt is used for optional on update clause. type OnUpdateOpt struct { node - ReferOpt model.ReferOptionType + ReferOpt ReferOptionType } // Restore implements Node interface. func (n *OnUpdateOpt) Restore(ctx *format.RestoreCtx) error { - if n.ReferOpt != model.ReferOptionNoOption { + if n.ReferOpt != ReferOptionNoOption { ctx.WriteKeyWord("ON UPDATE ") ctx.WriteKeyWord(n.ReferOpt.String()) } @@ -555,7 +554,7 @@ type ColumnOption struct { Enforced bool // Name is only used for Check Constraint name. ConstraintName string - PrimaryKeyTp model.PrimaryKeyType + PrimaryKeyTp PrimaryKeyType } // Restore implements Node interface. @@ -729,11 +728,11 @@ type IndexOption struct { node KeyBlockSize uint64 - Tp model.IndexType + Tp IndexType Comment string - ParserName model.CIStr + ParserName CIStr Visibility IndexVisibility - PrimaryKeyTp model.PrimaryKeyType + PrimaryKeyTp PrimaryKeyType Global bool SplitOpt *SplitOption `json:"-"` // SplitOption contains expr nodes, which cannot marshal for DDL job arguments. } @@ -741,9 +740,9 @@ type IndexOption struct { // IsEmpty is true if only default options are given // and it should not be added to the output func (n *IndexOption) IsEmpty() bool { - if n.PrimaryKeyTp != model.PrimaryKeyTypeDefault || + if n.PrimaryKeyTp != PrimaryKeyTypeDefault || n.KeyBlockSize > 0 || - n.Tp != model.IndexTypeInvalid || + n.Tp != IndexTypeInvalid || len(n.ParserName.O) > 0 || n.Comment != "" || n.Global || @@ -757,7 +756,7 @@ func (n *IndexOption) IsEmpty() bool { // Restore implements Node interface. func (n *IndexOption) Restore(ctx *format.RestoreCtx) error { hasPrevOption := false - if n.PrimaryKeyTp != model.PrimaryKeyTypeDefault { + if n.PrimaryKeyTp != PrimaryKeyTypeDefault { _ = ctx.WriteWithSpecialComments(tidb.FeatureIDClusteredIndex, func() error { ctx.WriteKeyWord(n.PrimaryKeyTp.String()) return nil @@ -773,7 +772,7 @@ func (n *IndexOption) Restore(ctx *format.RestoreCtx) error { hasPrevOption = true } - if n.Tp != model.IndexTypeInvalid { + if n.Tp != IndexTypeInvalid { if hasPrevOption { ctx.WritePlain(" ") } @@ -1353,7 +1352,7 @@ type DropPlacementPolicyStmt struct { ddlNode IfExists bool - PolicyName model.CIStr + PolicyName CIStr } // Restore implements Restore interface. @@ -1383,7 +1382,7 @@ type DropResourceGroupStmt struct { ddlNode IfExists bool - ResourceGroupName model.CIStr + ResourceGroupName CIStr } // Restore implements Restore interface. @@ -1573,13 +1572,13 @@ type CreateViewStmt struct { OrReplace bool ViewName *TableName - Cols []model.CIStr + Cols []CIStr Select StmtNode - SchemaCols []model.CIStr - Algorithm model.ViewAlgorithm + SchemaCols []CIStr + Algorithm ViewAlgorithm Definer *auth.UserIdentity - Security model.ViewSecurity - CheckOption model.ViewCheckOption + Security ViewSecurity + CheckOption ViewCheckOption } // Restore implements Node interface. @@ -1631,7 +1630,7 @@ func (n *CreateViewStmt) Restore(ctx *format.RestoreCtx) error { return errors.Annotate(err, "An error occurred while create CreateViewStmt.Select") } - if n.CheckOption != model.CheckOptionCascaded { + if n.CheckOption != CheckOptionCascaded { ctx.WriteKeyWord(" WITH ") ctx.WriteKeyWord(n.CheckOption.String()) ctx.WriteKeyWord(" CHECK OPTION") @@ -1665,7 +1664,7 @@ type CreatePlacementPolicyStmt struct { OrReplace bool IfNotExists bool - PolicyName model.CIStr + PolicyName CIStr PlacementOptions []*PlacementOption } @@ -1708,7 +1707,7 @@ type CreateResourceGroupStmt struct { ddlNode IfNotExists bool - ResourceGroupName model.CIStr + ResourceGroupName CIStr ResourceGroupOptionList []*ResourceGroupOption } @@ -2023,7 +2022,7 @@ type LockTablesStmt struct { // TableLock contains the table name and lock type. type TableLock struct { Table *TableName - Type model.TableLockType + Type TableLockType } // Accept implements Node Accept interface. @@ -2287,7 +2286,7 @@ func (n *ResourceGroupOption) Restore(ctx *format.RestoreCtx) error { case ResourcePriority: ctx.WriteKeyWord("PRIORITY ") ctx.WritePlain("= ") - ctx.WriteKeyWord(model.PriorityValueToName(n.UintValue)) + ctx.WriteKeyWord(PriorityValueToName(n.UintValue)) case ResourceUnitCPU: ctx.WriteKeyWord("CPU ") ctx.WritePlain("= ") @@ -2346,7 +2345,7 @@ func (n *ResourceGroupOption) Restore(ctx *format.RestoreCtx) error { // ResourceGroupRunawayOption is used for parsing resource group runaway rule option. type ResourceGroupRunawayOption struct { - Tp model.RunawayOptionType + Tp RunawayOptionType RuleOption *ResourceGroupRunawayRuleOption ActionOption *ResourceGroupRunawayActionOption WatchOption *ResourceGroupRunawayWatchOption @@ -2354,11 +2353,11 @@ type ResourceGroupRunawayOption struct { func (n *ResourceGroupRunawayOption) Restore(ctx *format.RestoreCtx) error { switch n.Tp { - case model.RunawayRule: + case RunawayRule: n.RuleOption.restore(ctx) - case model.RunawayAction: + case RunawayAction: n.ActionOption.Restore(ctx) - case model.RunawayWatch: + case RunawayWatch: n.WatchOption.restore(ctx) default: return errors.Errorf("invalid ResourceGroupRunawayOption: %d", n.Tp) @@ -2403,8 +2402,8 @@ func (n *ResourceGroupRunawayRuleOption) restore(ctx *format.RestoreCtx) error { // ResourceGroupRunawayActionOption is used for parsing the resource group runaway action. type ResourceGroupRunawayActionOption struct { node - Type model.RunawayActionType - SwitchGroupName model.CIStr + Type RunawayActionType + SwitchGroupName CIStr } // Restore implements Node interface. @@ -2412,9 +2411,9 @@ func (n *ResourceGroupRunawayActionOption) Restore(ctx *format.RestoreCtx) error ctx.WriteKeyWord("ACTION ") ctx.WritePlain("= ") switch n.Type { - case model.RunawayActionNone, model.RunawayActionDryRun, model.RunawayActionCooldown, model.RunawayActionKill: + case RunawayActionNone, RunawayActionDryRun, RunawayActionCooldown, RunawayActionKill: ctx.WriteKeyWord(n.Type.String()) - case model.RunawayActionSwitchGroup: + case RunawayActionSwitchGroup: switchGroup := n.SwitchGroupName.String() if len(switchGroup) == 0 { return errors.New("SWITCH_GROUP runaway watch action requires a non-empty group name") @@ -2438,7 +2437,7 @@ func (n *ResourceGroupRunawayActionOption) Accept(v Visitor) (Node, bool) { // ResourceGroupRunawayWatchOption is used for parsing the resource group runaway watch. type ResourceGroupRunawayWatchOption struct { - Type model.RunawayWatchType + Type RunawayWatchType Duration string } @@ -3200,7 +3199,7 @@ type AlterTableSpec struct { Tp AlterTableType Name string - IndexName model.CIStr + IndexName CIStr Constraint *Constraint Options []*TableOption OrderByList []*AlterOrderItem @@ -3213,10 +3212,10 @@ type AlterTableSpec struct { LockType LockType Algorithm AlgorithmType Comment string - FromKey model.CIStr - ToKey model.CIStr + FromKey CIStr + ToKey CIStr Partition *PartitionOptions - PartitionNames []model.CIStr + PartitionNames []CIStr PartDefinitions []*PartitionDefinition WithValidation bool Num uint64 @@ -4023,7 +4022,7 @@ var ( ) type SubPartitionDefinition struct { - Name model.CIStr + Name CIStr Options []*TableOption } @@ -4045,7 +4044,7 @@ type PartitionDefinitionClause interface { // Validate checks if the clause is consistent with the given options. // `pt` can be 0 and `columns` can be -1 to skip checking the clause against // the partition type or number of columns in the expression list. - Validate(pt model.PartitionType, columns int) error + Validate(pt PartitionType, columns int) error } type PartitionDefinitionClauseNone struct{} @@ -4058,14 +4057,14 @@ func (*PartitionDefinitionClauseNone) acceptInPlace(_ Visitor) bool { return true } -func (*PartitionDefinitionClauseNone) Validate(pt model.PartitionType, _ int) error { +func (*PartitionDefinitionClauseNone) Validate(pt PartitionType, _ int) error { switch pt { case 0: - case model.PartitionTypeRange: + case PartitionTypeRange: return ErrPartitionRequiresValues.GenWithStackByArgs("RANGE", "LESS THAN") - case model.PartitionTypeList: + case PartitionTypeList: return ErrPartitionRequiresValues.GenWithStackByArgs("LIST", "IN") - case model.PartitionTypeSystemTime: + case PartitionTypeSystemTime: return ErrSystemVersioningWrongPartitions } return nil @@ -4101,9 +4100,9 @@ func (n *PartitionDefinitionClauseLessThan) acceptInPlace(v Visitor) bool { return true } -func (n *PartitionDefinitionClauseLessThan) Validate(pt model.PartitionType, columns int) error { +func (n *PartitionDefinitionClauseLessThan) Validate(pt PartitionType, columns int) error { switch pt { - case model.PartitionTypeRange, 0: + case PartitionTypeRange, 0: default: return ErrPartitionWrongValues.GenWithStackByArgs("RANGE", "LESS THAN") } @@ -4174,9 +4173,9 @@ func (n *PartitionDefinitionClauseIn) acceptInPlace(v Visitor) bool { return true } -func (n *PartitionDefinitionClauseIn) Validate(pt model.PartitionType, columns int) error { +func (n *PartitionDefinitionClauseIn) Validate(pt PartitionType, columns int) error { switch pt { - case model.PartitionTypeList, 0: + case PartitionTypeList, 0: default: return ErrPartitionWrongValues.GenWithStackByArgs("LIST", "IN") } @@ -4238,9 +4237,9 @@ func (*PartitionDefinitionClauseHistory) acceptInPlace(_ Visitor) bool { return true } -func (*PartitionDefinitionClauseHistory) Validate(pt model.PartitionType, _ int) error { +func (*PartitionDefinitionClauseHistory) Validate(pt PartitionType, _ int) error { switch pt { - case 0, model.PartitionTypeSystemTime: + case 0, PartitionTypeSystemTime: default: return ErrWrongPartitionTypeExpectedSystemTime } @@ -4250,7 +4249,7 @@ func (*PartitionDefinitionClauseHistory) Validate(pt model.PartitionType, _ int) // PartitionDefinition defines a single partition. type PartitionDefinition struct { - Name model.CIStr + Name CIStr Clause PartitionDefinitionClause Options []*TableOption Sub []*SubPartitionDefinition @@ -4326,7 +4325,7 @@ type PartitionMethod struct { // partition definitions node // Tp is the type of the partition function - Tp model.PartitionType + Tp PartitionType // Linear is a modifier to the HASH and KEY type for choosing a different // algorithm Linear bool @@ -4366,7 +4365,7 @@ func (n *PartitionMethod) Restore(ctx *format.RestoreCtx) error { } switch { - case n.Tp == model.PartitionTypeSystemTime: + case n.Tp == PartitionTypeSystemTime: if n.Expr != nil && n.Unit != TimeUnitInvalid { ctx.WriteKeyWord(" INTERVAL ") if err := n.Expr.Restore(ctx); err != nil { @@ -4388,7 +4387,7 @@ func (n *PartitionMethod) Restore(ctx *format.RestoreCtx) error { ctx.WritePlain(")") default: - if n.Tp == model.PartitionTypeRange || n.Tp == model.PartitionTypeList { + if n.Tp == PartitionTypeRange || n.Tp == PartitionTypeList { ctx.WriteKeyWord(" COLUMNS") } ctx.WritePlain(" (") @@ -4490,15 +4489,15 @@ func (n *PartitionOptions) Validate() error { } switch n.Tp { - case model.PartitionTypeHash, model.PartitionTypeKey: + case PartitionTypeHash, PartitionTypeKey: if n.Num == 0 { n.Num = 1 } - case model.PartitionTypeRange, model.PartitionTypeList: + case PartitionTypeRange, PartitionTypeList: if n.Interval == nil && len(n.Definitions) == 0 { return ErrPartitionsMustBeDefined.GenWithStackByArgs(n.Tp) } - case model.PartitionTypeSystemTime: + case PartitionTypeSystemTime: if len(n.Definitions) < 2 { return ErrSystemVersioningWrongPartitions } @@ -4641,7 +4640,7 @@ type FlashBackToTimestampStmt struct { FlashbackTS ExprNode FlashbackTSO uint64 Tables []*TableName - DBName model.CIStr + DBName CIStr } // Restore implements Node interface @@ -4799,7 +4798,7 @@ func (n *StatsOptionsSpec) Accept(v Visitor) (Node, bool) { type AlterPlacementPolicyStmt struct { ddlNode - PolicyName model.CIStr + PolicyName CIStr IfExists bool PlacementOptions []*PlacementOption } @@ -4848,7 +4847,7 @@ func CheckRunawayAppend(ops []*ResourceGroupRunawayOption, newOp *ResourceGroupR for _, op := range ops { if op.Tp == newOp.Tp { // support multiple runaway rules. - if op.Tp == model.RunawayRule { + if op.Tp == RunawayRule { continue } return false @@ -4870,7 +4869,7 @@ func CheckBackgroundAppend(ops []*ResourceGroupBackgroundOption, newOp *Resource type AlterResourceGroupStmt struct { ddlNode - ResourceGroupName model.CIStr + ResourceGroupName CIStr IfExists bool ResourceGroupOptionList []*ResourceGroupOption } diff --git a/pkg/parser/ast/dml.go b/pkg/parser/ast/dml.go index f9fc601de03d2..0e81bdded5c0c 100644 --- a/pkg/parser/ast/dml.go +++ b/pkg/parser/ast/dml.go @@ -19,7 +19,6 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/parser/auth" "github.com/pingcap/tidb/pkg/parser/format" - "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" ) @@ -274,11 +273,11 @@ func (n *Join) Accept(v Visitor) (Node, bool) { type TableName struct { node - Schema model.CIStr - Name model.CIStr + Schema CIStr + Name CIStr IndexHints []*IndexHint - PartitionNames []model.CIStr + PartitionNames []CIStr TableSample *TableSample // AS OF is used to see the data as it was at a specific point in time. AsOf *AsOfClause @@ -380,7 +379,7 @@ const ( // IndexHint represents a hint for optimizer to use/ignore/force for join/order by/group by. type IndexHint struct { - IndexNames []model.CIStr + IndexNames []CIStr HintType IndexHintType HintScope IndexHintScope } @@ -532,7 +531,7 @@ type TableSource struct { Source ResultSetNode // AsName is the alias name of the table source. - AsName model.CIStr + AsName CIStr } func (*TableSource) resultSet() {} @@ -659,8 +658,8 @@ func (n SelectLockType) String() string { type WildCardField struct { node - Table model.CIStr - Schema model.CIStr + Table CIStr + Schema CIStr } // Restore implements Node interface. @@ -700,7 +699,7 @@ type SelectField struct { // Expr is not nil, WildCard will be nil. Expr ExprNode // AsName is alias name for Expr. - AsName model.CIStr + AsName CIStr // Auxiliary stands for if this field is auxiliary. // When we add a Field into SelectField list which is used for having/orderby clause but the field is not in select clause, // we should set its Auxiliary to true. Then the TrimExec will trim the field. @@ -1084,9 +1083,9 @@ func (s *SelectStmtKind) String() string { type CommonTableExpression struct { node - Name model.CIStr + Name CIStr Query *SubqueryExpr - ColNameList []model.CIStr + ColNameList []CIStr IsRecursive bool // Record how many consumers the current cte has @@ -2303,7 +2302,7 @@ type InsertStmt struct { Select ResultSetNode // TableHints represents the table level Optimizer Hint for join type. TableHints []*TableOptimizerHint - PartitionNames []model.CIStr + PartitionNames []CIStr } // Restore implements Node interface. @@ -3076,9 +3075,9 @@ type ShowStmt struct { Table *TableName // Used for showing columns. // Procedure's naming method is consistent with the table name Procedure *TableName - Partition model.CIStr // Used for showing partition. + Partition CIStr // Used for showing partition. Column *ColumnName // Used for `desc table column`. - IndexName model.CIStr + IndexName CIStr ResourceGroupName string // used for showing resource group Flag int // Some flag parsed from sql, such as FULL. Full bool @@ -3520,10 +3519,10 @@ func (n *ShowStmt) NeedLimitRSRow() bool { type WindowSpec struct { node - Name model.CIStr + Name CIStr // Ref is the reference window of this specification. For example, in `w2 as (w1 order by a)`, // the definition of `w2` references `w1`. - Ref model.CIStr + Ref CIStr PartitionBy *PartitionByClause OrderBy *OrderByClause @@ -3831,8 +3830,8 @@ type SplitRegionStmt struct { dmlNode Table *TableName - IndexName model.CIStr - PartitionNames []model.CIStr + IndexName CIStr + PartitionNames []CIStr SplitSyntaxOpt *SplitSyntaxOption diff --git a/pkg/parser/ast/expressions.go b/pkg/parser/ast/expressions.go index 7c341ef396924..a0a8449aa8153 100644 --- a/pkg/parser/ast/expressions.go +++ b/pkg/parser/ast/expressions.go @@ -22,7 +22,6 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/parser/format" - "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/opcode" ) @@ -505,9 +504,9 @@ func (n *TableNameExpr) Accept(v Visitor) (Node, bool) { // ColumnName represents column name. type ColumnName struct { node - Schema model.CIStr - Table model.CIStr - Name model.CIStr + Schema CIStr + Table CIStr + Name CIStr } // Restore implements Node interface. diff --git a/pkg/parser/ast/functions.go b/pkg/parser/ast/functions.go index 21fc5d7ee3ecb..7b803db6f94e9 100644 --- a/pkg/parser/ast/functions.go +++ b/pkg/parser/ast/functions.go @@ -21,7 +21,6 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/parser/format" - "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/types" ) @@ -386,9 +385,9 @@ const ( type FuncCallExpr struct { funcNode Tp FuncCallExprType - Schema model.CIStr + Schema CIStr // FnName is the function name. - FnName model.CIStr + FnName CIStr // Args is the function args. Args []ExprNode } diff --git a/pkg/parser/ast/misc.go b/pkg/parser/ast/misc.go index 349f7c3680f30..88aeac748f000 100644 --- a/pkg/parser/ast/misc.go +++ b/pkg/parser/ast/misc.go @@ -24,7 +24,6 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb/pkg/parser/auth" "github.com/pingcap/tidb/pkg/parser/format" - "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" ) @@ -540,7 +539,7 @@ type CompactTableStmt struct { stmtNode Table *TableName - PartitionNames []model.CIStr + PartitionNames []CIStr ReplicaKind CompactReplicaKind } @@ -1954,7 +1953,7 @@ func (n *AlterInstanceStmt) Accept(v Visitor) (Node, bool) { // AlterRangeStmt modifies range configuration. type AlterRangeStmt struct { stmtNode - RangeName model.CIStr + RangeName CIStr PlacementOption *PlacementOption } @@ -3833,8 +3832,8 @@ func (n *ImportIntoActionStmt) Restore(ctx *format.RestoreCtx) error { // Ident is the table identifier composed of schema name and table name. type Ident struct { - Schema model.CIStr - Name model.CIStr + Schema CIStr + Name CIStr } // String implements fmt.Stringer interface. @@ -3865,7 +3864,7 @@ type TableOptimizerHint struct { // HintName is the name or alias of the table(s) which the hint will affect. // Table hints has no schema info // It allows only table name or alias (if table has an alias) - HintName model.CIStr + HintName CIStr // HintData is the payload of the hint. The actual type of this field // is defined differently as according `HintName`. Define as following: // @@ -3883,9 +3882,9 @@ type TableOptimizerHint struct { // - NTH_PLAN => int64 HintData interface{} // QBName is the default effective query block of this hint. - QBName model.CIStr + QBName CIStr Tables []HintTable - Indexes []model.CIStr + Indexes []CIStr } // HintTimeRange is the payload of `TIME_RANGE` hint @@ -3902,10 +3901,10 @@ type HintSetVar struct { // HintTable is table in the hint. It may have query block info. type HintTable struct { - DBName model.CIStr - TableName model.CIStr - QBName model.CIStr - PartitionList []model.CIStr + DBName CIStr + TableName CIStr + QBName CIStr + PartitionList []CIStr } func (ht *HintTable) Restore(ctx *format.RestoreCtx) { @@ -3999,11 +3998,11 @@ func (n *TableOptimizerHint) Restore(ctx *format.RestoreCtx) error { ctx.WritePlain("FALSE") } case "query_type": - ctx.WriteKeyWord(n.HintData.(model.CIStr).String()) + ctx.WriteKeyWord(n.HintData.(CIStr).String()) case "memory_quota": ctx.WritePlainf("%d MB", n.HintData.(int64)/1024/1024) case "read_from_storage": - ctx.WriteKeyWord(n.HintData.(model.CIStr).String()) + ctx.WriteKeyWord(n.HintData.(CIStr).String()) for i, table := range n.Tables { if i == 0 { ctx.WritePlain("[") @@ -4062,7 +4061,7 @@ var NewBitLiteral func(string) (interface{}, error) // SetResourceGroupStmt is a statement to set the resource group name for current session. type SetResourceGroupStmt struct { stmtNode - Name model.CIStr + Name CIStr } func (n *SetResourceGroupStmt) Restore(ctx *format.RestoreCtx) error { @@ -4333,7 +4332,7 @@ func CheckQueryWatchAppend(ops []*QueryWatchOption, newOp *QueryWatchOption) boo // QueryWatchResourceGroupOption is used for parsing the query watch resource group name. type QueryWatchResourceGroupOption struct { - GroupNameStr model.CIStr + GroupNameStr CIStr GroupNameExpr ExprNode } @@ -4352,7 +4351,7 @@ func (n *QueryWatchResourceGroupOption) restore(ctx *format.RestoreCtx) error { // QueryWatchTextOption is used for parsing the query watch text option. type QueryWatchTextOption struct { node - Type model.RunawayWatchType + Type RunawayWatchType PatternExpr ExprNode TypeSpecified bool } @@ -4365,9 +4364,9 @@ func (n *QueryWatchTextOption) Restore(ctx *format.RestoreCtx) error { ctx.WriteKeyWord(" TO ") } else { switch n.Type { - case model.WatchSimilar: + case WatchSimilar: ctx.WriteKeyWord("SQL DIGEST ") - case model.WatchPlan: + case WatchPlan: ctx.WriteKeyWord("PLAN DIGEST ") } } diff --git a/pkg/parser/model/model.go b/pkg/parser/ast/model.go similarity index 99% rename from pkg/parser/model/model.go rename to pkg/parser/ast/model.go index c4eb8058dad3b..bdd25f4365463 100644 --- a/pkg/parser/model/model.go +++ b/pkg/parser/ast/model.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package model +package ast import ( "encoding/json" diff --git a/pkg/parser/model/model_test.go b/pkg/parser/ast/model_test.go similarity index 98% rename from pkg/parser/model/model_test.go rename to pkg/parser/ast/model_test.go index 15e8112c65e36..67aa931f4feb7 100644 --- a/pkg/parser/model/model_test.go +++ b/pkg/parser/ast/model_test.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package model +package ast import ( "encoding/json" diff --git a/pkg/parser/ast/stats.go b/pkg/parser/ast/stats.go index 2c0a8c6beec48..ac5bb008c0904 100644 --- a/pkg/parser/ast/stats.go +++ b/pkg/parser/ast/stats.go @@ -16,7 +16,6 @@ package ast import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/parser/format" - "github.com/pingcap/tidb/pkg/parser/model" ) var ( @@ -30,8 +29,8 @@ type AnalyzeTableStmt struct { stmtNode TableNames []*TableName - PartitionNames []model.CIStr - IndexNames []model.CIStr + PartitionNames []CIStr + IndexNames []CIStr AnalyzeOpts []AnalyzeOpt // IndexFlag is true when we only analyze indices for a table. @@ -41,8 +40,8 @@ type AnalyzeTableStmt struct { // HistogramOperation is set in "ANALYZE TABLE ... UPDATE/DROP HISTOGRAM ..." statement. HistogramOperation HistogramOperationType // ColumnNames indicate the columns whose statistics need to be collected. - ColumnNames []model.CIStr - ColumnChoice model.ColumnChoice + ColumnNames []CIStr + ColumnChoice ColumnChoice } // AnalyzeOptType is the type for analyze options. @@ -139,11 +138,11 @@ func (n *AnalyzeTableStmt) Restore(ctx *format.RestoreCtx) error { } } switch n.ColumnChoice { - case model.AllColumns: + case AllColumns: ctx.WriteKeyWord(" ALL COLUMNS") - case model.PredicateColumns: + case PredicateColumns: ctx.WriteKeyWord(" PREDICATE COLUMNS") - case model.ColumnList: + case ColumnList: ctx.WriteKeyWord(" COLUMNS ") for i, columnName := range n.ColumnNames { if i != 0 { @@ -199,7 +198,7 @@ type DropStatsStmt struct { stmtNode Tables []*TableName - PartitionNames []model.CIStr + PartitionNames []CIStr IsGlobalStats bool } diff --git a/pkg/parser/hintparser.go b/pkg/parser/hintparser.go index 0912b6a7f55c5..2dbeaa57fe67f 100644 --- a/pkg/parser/hintparser.go +++ b/pkg/parser/hintparser.go @@ -22,7 +22,6 @@ import ( "strconv" "github.com/pingcap/tidb/pkg/parser/ast" - "github.com/pingcap/tidb/pkg/parser/model" ) type yyhintSymType struct { @@ -33,7 +32,7 @@ type yyhintSymType struct { hint *ast.TableOptimizerHint hints []*ast.TableOptimizerHint table ast.HintTable - modelIdents []model.CIStr + modelIdents []ast.CIStr } type yyhintXError struct { @@ -1279,7 +1278,7 @@ yynewstate: case 9: { h := yyS[yypt-1].hint - h.HintName = model.NewCIStr(yyS[yypt-3].ident) + h.HintName = ast.NewCIStr(yyS[yypt-3].ident) parser.yyVAL.hint = h } case 10: @@ -1290,7 +1289,7 @@ yynewstate: case 11: { h := yyS[yypt-1].hint - h.HintName = model.NewCIStr(yyS[yypt-3].ident) + h.HintName = ast.NewCIStr(yyS[yypt-3].ident) parser.yyVAL.hint = h } case 12: @@ -1301,23 +1300,23 @@ yynewstate: case 13: { parser.yyVAL.hint = &ast.TableOptimizerHint{ - HintName: model.NewCIStr(yyS[yypt-4].ident), - QBName: model.NewCIStr(yyS[yypt-2].ident), + HintName: ast.NewCIStr(yyS[yypt-4].ident), + QBName: ast.NewCIStr(yyS[yypt-2].ident), HintData: yyS[yypt-1].number, } } case 14: { parser.yyVAL.hint = &ast.TableOptimizerHint{ - HintName: model.NewCIStr(yyS[yypt-4].ident), - QBName: model.NewCIStr(yyS[yypt-2].ident), + HintName: ast.NewCIStr(yyS[yypt-4].ident), + QBName: ast.NewCIStr(yyS[yypt-2].ident), HintData: int64(yyS[yypt-1].number), } } case 15: { parser.yyVAL.hint = &ast.TableOptimizerHint{ - HintName: model.NewCIStr(yyS[yypt-5].ident), + HintName: ast.NewCIStr(yyS[yypt-5].ident), HintData: ast.HintSetVar{ VarName: yyS[yypt-3].ident, Value: yyS[yypt-1].ident, @@ -1327,22 +1326,22 @@ yynewstate: case 16: { parser.yyVAL.hint = &ast.TableOptimizerHint{ - HintName: model.NewCIStr(yyS[yypt-3].ident), + HintName: ast.NewCIStr(yyS[yypt-3].ident), HintData: yyS[yypt-1].ident, } } case 17: { parser.yyVAL.hint = &ast.TableOptimizerHint{ - HintName: model.NewCIStr(yyS[yypt-3].ident), - QBName: model.NewCIStr(yyS[yypt-1].ident), + HintName: ast.NewCIStr(yyS[yypt-3].ident), + QBName: ast.NewCIStr(yyS[yypt-1].ident), } } case 18: { parser.yyVAL.hint = &ast.TableOptimizerHint{ - HintName: model.NewCIStr(yyS[yypt-5].ident), - QBName: model.NewCIStr(yyS[yypt-3].ident), + HintName: ast.NewCIStr(yyS[yypt-5].ident), + QBName: ast.NewCIStr(yyS[yypt-3].ident), Tables: yyS[yypt-1].hint.Tables, } } @@ -1351,9 +1350,9 @@ yynewstate: maxValue := uint64(math.MaxInt64) / yyS[yypt-1].number if yyS[yypt-2].number <= maxValue { parser.yyVAL.hint = &ast.TableOptimizerHint{ - HintName: model.NewCIStr(yyS[yypt-5].ident), + HintName: ast.NewCIStr(yyS[yypt-5].ident), HintData: int64(yyS[yypt-2].number * yyS[yypt-1].number), - QBName: model.NewCIStr(yyS[yypt-3].ident), + QBName: ast.NewCIStr(yyS[yypt-3].ident), } } else { yylex.AppendError(ErrWarnMemoryQuotaOverflow.GenWithStackByArgs(math.MaxInt64)) @@ -1364,7 +1363,7 @@ yynewstate: case 20: { parser.yyVAL.hint = &ast.TableOptimizerHint{ - HintName: model.NewCIStr(yyS[yypt-5].ident), + HintName: ast.NewCIStr(yyS[yypt-5].ident), HintData: ast.HintTimeRange{ From: yyS[yypt-3].ident, To: yyS[yypt-1].ident, @@ -1374,23 +1373,23 @@ yynewstate: case 21: { h := yyS[yypt-1].hint - h.HintName = model.NewCIStr(yyS[yypt-4].ident) - h.QBName = model.NewCIStr(yyS[yypt-2].ident) + h.HintName = ast.NewCIStr(yyS[yypt-4].ident) + h.QBName = ast.NewCIStr(yyS[yypt-2].ident) parser.yyVAL.hint = h } case 22: { parser.yyVAL.hint = &ast.TableOptimizerHint{ - HintName: model.NewCIStr(yyS[yypt-3].ident), - QBName: model.NewCIStr(yyS[yypt-1].ident), + HintName: ast.NewCIStr(yyS[yypt-3].ident), + QBName: ast.NewCIStr(yyS[yypt-1].ident), } } case 23: { parser.yyVAL.hint = &ast.TableOptimizerHint{ - HintName: model.NewCIStr(yyS[yypt-4].ident), - QBName: model.NewCIStr(yyS[yypt-2].ident), - HintData: model.NewCIStr(yyS[yypt-1].ident), + HintName: ast.NewCIStr(yyS[yypt-4].ident), + QBName: ast.NewCIStr(yyS[yypt-2].ident), + HintData: ast.NewCIStr(yyS[yypt-1].ident), } } case 24: @@ -1416,8 +1415,8 @@ yynewstate: case 28: { hs := yyS[yypt-1].hints - name := model.NewCIStr(yyS[yypt-4].ident) - qb := model.NewCIStr(yyS[yypt-2].ident) + name := ast.NewCIStr(yyS[yypt-4].ident) + qb := ast.NewCIStr(yyS[yypt-2].ident) for _, h := range hs { h.HintName = name h.QBName = qb @@ -1435,7 +1434,7 @@ yynewstate: case 31: { h := yyS[yypt-1].hint - h.HintData = model.NewCIStr(yyS[yypt-3].ident) + h.HintData = ast.NewCIStr(yyS[yypt-3].ident) parser.yyVAL.hint = h } case 32: @@ -1452,23 +1451,23 @@ yynewstate: } case 38: { - parser.yyVAL.modelIdents = []model.CIStr{model.NewCIStr(yyS[yypt-0].ident)} + parser.yyVAL.modelIdents = []ast.CIStr{ast.NewCIStr(yyS[yypt-0].ident)} } case 39: { - parser.yyVAL.modelIdents = append(yyS[yypt-2].modelIdents, model.NewCIStr(yyS[yypt-0].ident)) + parser.yyVAL.modelIdents = append(yyS[yypt-2].modelIdents, ast.NewCIStr(yyS[yypt-0].ident)) } case 41: { parser.yyVAL.hint = &ast.TableOptimizerHint{ - QBName: model.NewCIStr(yyS[yypt-0].ident), + QBName: ast.NewCIStr(yyS[yypt-0].ident), } } case 42: { parser.yyVAL.hint = &ast.TableOptimizerHint{ Tables: []ast.HintTable{yyS[yypt-0].table}, - QBName: model.NewCIStr(yyS[yypt-1].ident), + QBName: ast.NewCIStr(yyS[yypt-1].ident), } } case 43: @@ -1480,17 +1479,17 @@ yynewstate: case 44: { parser.yyVAL.table = ast.HintTable{ - TableName: model.NewCIStr(yyS[yypt-2].ident), - QBName: model.NewCIStr(yyS[yypt-1].ident), + TableName: ast.NewCIStr(yyS[yypt-2].ident), + QBName: ast.NewCIStr(yyS[yypt-1].ident), PartitionList: yyS[yypt-0].modelIdents, } } case 45: { parser.yyVAL.table = ast.HintTable{ - DBName: model.NewCIStr(yyS[yypt-4].ident), - TableName: model.NewCIStr(yyS[yypt-2].ident), - QBName: model.NewCIStr(yyS[yypt-1].ident), + DBName: ast.NewCIStr(yyS[yypt-4].ident), + TableName: ast.NewCIStr(yyS[yypt-2].ident), + QBName: ast.NewCIStr(yyS[yypt-1].ident), PartitionList: yyS[yypt-0].modelIdents, } } @@ -1509,21 +1508,21 @@ yynewstate: case 48: { parser.yyVAL.table = ast.HintTable{ - TableName: model.NewCIStr(yyS[yypt-1].ident), - QBName: model.NewCIStr(yyS[yypt-0].ident), + TableName: ast.NewCIStr(yyS[yypt-1].ident), + QBName: ast.NewCIStr(yyS[yypt-0].ident), } } case 49: { parser.yyVAL.table = ast.HintTable{ - QBName: model.NewCIStr(yyS[yypt-0].ident), + QBName: ast.NewCIStr(yyS[yypt-0].ident), } } case 50: { h := yyS[yypt-0].hint h.Tables = []ast.HintTable{yyS[yypt-2].table} - h.QBName = model.NewCIStr(yyS[yypt-3].ident) + h.QBName = ast.NewCIStr(yyS[yypt-3].ident) parser.yyVAL.hint = h } case 51: @@ -1533,13 +1532,13 @@ yynewstate: case 53: { parser.yyVAL.hint = &ast.TableOptimizerHint{ - Indexes: []model.CIStr{model.NewCIStr(yyS[yypt-0].ident)}, + Indexes: []ast.CIStr{ast.NewCIStr(yyS[yypt-0].ident)}, } } case 54: { h := yyS[yypt-2].hint - h.Indexes = append(h.Indexes, model.NewCIStr(yyS[yypt-0].ident)) + h.Indexes = append(h.Indexes, ast.NewCIStr(yyS[yypt-0].ident)) parser.yyVAL.hint = h } case 61: diff --git a/pkg/parser/hintparser.y b/pkg/parser/hintparser.y index cb4fcdbd9145a..dc57f4f440e0f 100644 --- a/pkg/parser/hintparser.y +++ b/pkg/parser/hintparser.y @@ -19,7 +19,6 @@ import ( "strconv" "github.com/pingcap/tidb/pkg/parser/ast" - "github.com/pingcap/tidb/pkg/parser/model" ) %} @@ -31,7 +30,7 @@ import ( hint *ast.TableOptimizerHint hints []*ast.TableOptimizerHint table ast.HintTable - modelIdents []model.CIStr + modelIdents []ast.CIStr } %token @@ -240,7 +239,7 @@ TableOptimizerHintOpt: | SupportedTableLevelOptimizerHintName '(' HintTableListOpt ')' { h := $3 - h.HintName = model.NewCIStr($1) + h.HintName = ast.NewCIStr($1) $$ = h } | UnsupportedIndexLevelOptimizerHintName '(' HintIndexList ')' @@ -251,7 +250,7 @@ TableOptimizerHintOpt: | SupportedIndexLevelOptimizerHintName '(' HintIndexList ')' { h := $3 - h.HintName = model.NewCIStr($1) + h.HintName = ast.NewCIStr($1) $$ = h } | SubqueryOptimizerHintName '(' QueryBlockOpt SubqueryStrategiesOpt ')' @@ -262,23 +261,23 @@ TableOptimizerHintOpt: | "MAX_EXECUTION_TIME" '(' QueryBlockOpt hintIntLit ')' { $$ = &ast.TableOptimizerHint{ - HintName: model.NewCIStr($1), - QBName: model.NewCIStr($3), + HintName: ast.NewCIStr($1), + QBName: ast.NewCIStr($3), HintData: $4, } } | "NTH_PLAN" '(' QueryBlockOpt hintIntLit ')' { $$ = &ast.TableOptimizerHint{ - HintName: model.NewCIStr($1), - QBName: model.NewCIStr($3), + HintName: ast.NewCIStr($1), + QBName: ast.NewCIStr($3), HintData: int64($4), } } | "SET_VAR" '(' Identifier '=' Value ')' { $$ = &ast.TableOptimizerHint{ - HintName: model.NewCIStr($1), + HintName: ast.NewCIStr($1), HintData: ast.HintSetVar{ VarName: $3, Value: $5, @@ -288,22 +287,22 @@ TableOptimizerHintOpt: | "RESOURCE_GROUP" '(' Identifier ')' { $$ = &ast.TableOptimizerHint{ - HintName: model.NewCIStr($1), + HintName: ast.NewCIStr($1), HintData: $3, } } | "QB_NAME" '(' Identifier ')' { $$ = &ast.TableOptimizerHint{ - HintName: model.NewCIStr($1), - QBName: model.NewCIStr($3), + HintName: ast.NewCIStr($1), + QBName: ast.NewCIStr($3), } } | "QB_NAME" '(' Identifier ',' ViewNameList ')' { $$ = &ast.TableOptimizerHint{ - HintName: model.NewCIStr($1), - QBName: model.NewCIStr($3), + HintName: ast.NewCIStr($1), + QBName: ast.NewCIStr($3), Tables: $5.Tables, } } @@ -312,9 +311,9 @@ TableOptimizerHintOpt: maxValue := uint64(math.MaxInt64) / $5 if $4 <= maxValue { $$ = &ast.TableOptimizerHint{ - HintName: model.NewCIStr($1), + HintName: ast.NewCIStr($1), HintData: int64($4 * $5), - QBName: model.NewCIStr($3), + QBName: ast.NewCIStr($3), } } else { yylex.AppendError(ErrWarnMemoryQuotaOverflow.GenWithStackByArgs(math.MaxInt64)) @@ -325,7 +324,7 @@ TableOptimizerHintOpt: | "TIME_RANGE" '(' hintStringLit CommaOpt hintStringLit ')' { $$ = &ast.TableOptimizerHint{ - HintName: model.NewCIStr($1), + HintName: ast.NewCIStr($1), HintData: ast.HintTimeRange{ From: $3, To: $5, @@ -335,23 +334,23 @@ TableOptimizerHintOpt: | BooleanHintName '(' QueryBlockOpt HintTrueOrFalse ')' { h := $4 - h.HintName = model.NewCIStr($1) - h.QBName = model.NewCIStr($3) + h.HintName = ast.NewCIStr($1) + h.QBName = ast.NewCIStr($3) $$ = h } | NullaryHintName '(' QueryBlockOpt ')' { $$ = &ast.TableOptimizerHint{ - HintName: model.NewCIStr($1), - QBName: model.NewCIStr($3), + HintName: ast.NewCIStr($1), + QBName: ast.NewCIStr($3), } } | "QUERY_TYPE" '(' QueryBlockOpt HintQueryType ')' { $$ = &ast.TableOptimizerHint{ - HintName: model.NewCIStr($1), - QBName: model.NewCIStr($3), - HintData: model.NewCIStr($4), + HintName: ast.NewCIStr($1), + QBName: ast.NewCIStr($3), + HintData: ast.NewCIStr($4), } } | hintIdentifier '(' QueryBlockOpt hintIntLit ')' @@ -380,8 +379,8 @@ StorageOptimizerHintOpt: "READ_FROM_STORAGE" '(' QueryBlockOpt HintStorageTypeAndTableList ')' { hs := $4 - name := model.NewCIStr($1) - qb := model.NewCIStr($3) + name := ast.NewCIStr($1) + qb := ast.NewCIStr($3) for _, h := range hs { h.HintName = name h.QBName = qb @@ -403,7 +402,7 @@ HintStorageTypeAndTable: HintStorageType '[' HintTableList ']' { h := $3 - h.HintData = model.NewCIStr($1) + h.HintData = ast.NewCIStr($1) $$ = h } @@ -433,11 +432,11 @@ PartitionListOpt: PartitionList: Identifier { - $$ = []model.CIStr{model.NewCIStr($1)} + $$ = []ast.CIStr{ast.NewCIStr($1)} } | PartitionList CommaOpt Identifier { - $$ = append($1, model.NewCIStr($3)) + $$ = append($1, ast.NewCIStr($3)) } /** @@ -452,7 +451,7 @@ HintTableListOpt: | QueryBlockOpt { $$ = &ast.TableOptimizerHint{ - QBName: model.NewCIStr($1), + QBName: ast.NewCIStr($1), } } @@ -461,7 +460,7 @@ HintTableList: { $$ = &ast.TableOptimizerHint{ Tables: []ast.HintTable{$2}, - QBName: model.NewCIStr($1), + QBName: ast.NewCIStr($1), } } | HintTableList ',' HintTable @@ -475,17 +474,17 @@ HintTable: Identifier QueryBlockOpt PartitionListOpt { $$ = ast.HintTable{ - TableName: model.NewCIStr($1), - QBName: model.NewCIStr($2), + TableName: ast.NewCIStr($1), + QBName: ast.NewCIStr($2), PartitionList: $3, } } | Identifier '.' Identifier QueryBlockOpt PartitionListOpt { $$ = ast.HintTable{ - DBName: model.NewCIStr($1), - TableName: model.NewCIStr($3), - QBName: model.NewCIStr($4), + DBName: ast.NewCIStr($1), + TableName: ast.NewCIStr($3), + QBName: ast.NewCIStr($4), PartitionList: $5, } } @@ -508,14 +507,14 @@ ViewName: Identifier QueryBlockOpt { $$ = ast.HintTable{ - TableName: model.NewCIStr($1), - QBName: model.NewCIStr($2), + TableName: ast.NewCIStr($1), + QBName: ast.NewCIStr($2), } } | QueryBlockOpt { $$ = ast.HintTable{ - QBName: model.NewCIStr($1), + QBName: ast.NewCIStr($1), } } @@ -530,7 +529,7 @@ HintIndexList: { h := $4 h.Tables = []ast.HintTable{$2} - h.QBName = model.NewCIStr($1) + h.QBName = ast.NewCIStr($1) $$ = h } @@ -545,13 +544,13 @@ IndexNameList: Identifier { $$ = &ast.TableOptimizerHint{ - Indexes: []model.CIStr{model.NewCIStr($1)}, + Indexes: []ast.CIStr{ast.NewCIStr($1)}, } } | IndexNameList ',' Identifier { h := $1 - h.Indexes = append(h.Indexes, model.NewCIStr($3)) + h.Indexes = append(h.Indexes, ast.NewCIStr($3)) $$ = h } diff --git a/pkg/parser/hintparser_test.go b/pkg/parser/hintparser_test.go index 862f81ff6e49f..ef73a0fdd0f55 100644 --- a/pkg/parser/hintparser_test.go +++ b/pkg/parser/hintparser_test.go @@ -18,7 +18,6 @@ import ( "github.com/pingcap/tidb/pkg/parser" "github.com/pingcap/tidb/pkg/parser/ast" - "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/stretchr/testify/require" ) @@ -38,11 +37,11 @@ func TestParseHint(t *testing.T) { input: "MEMORY_QUOTA(8 MB) MEMORY_QUOTA(6 GB)", output: []*ast.TableOptimizerHint{ { - HintName: model.NewCIStr("MEMORY_QUOTA"), + HintName: ast.NewCIStr("MEMORY_QUOTA"), HintData: int64(8 * 1024 * 1024), }, { - HintName: model.NewCIStr("MEMORY_QUOTA"), + HintName: ast.NewCIStr("MEMORY_QUOTA"), HintData: int64(6 * 1024 * 1024 * 1024), }, }, @@ -52,32 +51,32 @@ func TestParseHint(t *testing.T) { mode: mysql.ModeANSIQuotes, output: []*ast.TableOptimizerHint{ { - HintName: model.NewCIStr("QB_NAME"), - QBName: model.NewCIStr("qb1"), + HintName: ast.NewCIStr("QB_NAME"), + QBName: ast.NewCIStr("qb1"), }, { - HintName: model.NewCIStr("QB_NAME"), - QBName: model.NewCIStr("qb2"), + HintName: ast.NewCIStr("QB_NAME"), + QBName: ast.NewCIStr("qb2"), }, { - HintName: model.NewCIStr("QB_NAME"), - QBName: model.NewCIStr("TRUE"), + HintName: ast.NewCIStr("QB_NAME"), + QBName: ast.NewCIStr("TRUE"), }, { - HintName: model.NewCIStr("QB_NAME"), - QBName: model.NewCIStr("ANSI quoted"), + HintName: ast.NewCIStr("QB_NAME"), + QBName: ast.NewCIStr("ANSI quoted"), }, { - HintName: model.NewCIStr("QB_NAME"), - QBName: model.NewCIStr("_utf8"), + HintName: ast.NewCIStr("QB_NAME"), + QBName: ast.NewCIStr("_utf8"), }, { - HintName: model.NewCIStr("QB_NAME"), - QBName: model.NewCIStr("0b10"), + HintName: ast.NewCIStr("QB_NAME"), + QBName: ast.NewCIStr("0b10"), }, { - HintName: model.NewCIStr("QB_NAME"), - QBName: model.NewCIStr("0x1a"), + HintName: ast.NewCIStr("QB_NAME"), + QBName: ast.NewCIStr("0x1a"), }, }, }, @@ -122,23 +121,23 @@ func TestParseHint(t *testing.T) { input: "HASH_JOIN() TIDB_HJ(@qb1) INL_JOIN(x, `y y`.z) MERGE_JOIN(w@`First QB`)", output: []*ast.TableOptimizerHint{ { - HintName: model.NewCIStr("HASH_JOIN"), + HintName: ast.NewCIStr("HASH_JOIN"), }, { - HintName: model.NewCIStr("TIDB_HJ"), - QBName: model.NewCIStr("qb1"), + HintName: ast.NewCIStr("TIDB_HJ"), + QBName: ast.NewCIStr("qb1"), }, { - HintName: model.NewCIStr("INL_JOIN"), + HintName: ast.NewCIStr("INL_JOIN"), Tables: []ast.HintTable{ - {TableName: model.NewCIStr("x")}, - {DBName: model.NewCIStr("y y"), TableName: model.NewCIStr("z")}, + {TableName: ast.NewCIStr("x")}, + {DBName: ast.NewCIStr("y y"), TableName: ast.NewCIStr("z")}, }, }, { - HintName: model.NewCIStr("MERGE_JOIN"), + HintName: ast.NewCIStr("MERGE_JOIN"), Tables: []ast.HintTable{ - {TableName: model.NewCIStr("w"), QBName: model.NewCIStr("First QB")}, + {TableName: ast.NewCIStr("w"), QBName: ast.NewCIStr("First QB")}, }, }, }, @@ -147,24 +146,24 @@ func TestParseHint(t *testing.T) { input: "USE_INDEX_MERGE(@qb1 tbl1 x, y, z) IGNORE_INDEX(tbl2@qb2) USE_INDEX(tbl3 PRIMARY) FORCE_INDEX(tbl4@qb3 c1)", output: []*ast.TableOptimizerHint{ { - HintName: model.NewCIStr("USE_INDEX_MERGE"), - Tables: []ast.HintTable{{TableName: model.NewCIStr("tbl1")}}, - QBName: model.NewCIStr("qb1"), - Indexes: []model.CIStr{model.NewCIStr("x"), model.NewCIStr("y"), model.NewCIStr("z")}, + HintName: ast.NewCIStr("USE_INDEX_MERGE"), + Tables: []ast.HintTable{{TableName: ast.NewCIStr("tbl1")}}, + QBName: ast.NewCIStr("qb1"), + Indexes: []ast.CIStr{ast.NewCIStr("x"), ast.NewCIStr("y"), ast.NewCIStr("z")}, }, { - HintName: model.NewCIStr("IGNORE_INDEX"), - Tables: []ast.HintTable{{TableName: model.NewCIStr("tbl2"), QBName: model.NewCIStr("qb2")}}, + HintName: ast.NewCIStr("IGNORE_INDEX"), + Tables: []ast.HintTable{{TableName: ast.NewCIStr("tbl2"), QBName: ast.NewCIStr("qb2")}}, }, { - HintName: model.NewCIStr("USE_INDEX"), - Tables: []ast.HintTable{{TableName: model.NewCIStr("tbl3")}}, - Indexes: []model.CIStr{model.NewCIStr("PRIMARY")}, + HintName: ast.NewCIStr("USE_INDEX"), + Tables: []ast.HintTable{{TableName: ast.NewCIStr("tbl3")}}, + Indexes: []ast.CIStr{ast.NewCIStr("PRIMARY")}, }, { - HintName: model.NewCIStr("FORCE_INDEX"), - Tables: []ast.HintTable{{TableName: model.NewCIStr("tbl4"), QBName: model.NewCIStr("qb3")}}, - Indexes: []model.CIStr{model.NewCIStr("c1")}, + HintName: ast.NewCIStr("FORCE_INDEX"), + Tables: []ast.HintTable{{TableName: ast.NewCIStr("tbl4"), QBName: ast.NewCIStr("qb3")}}, + Indexes: []ast.CIStr{ast.NewCIStr("c1")}, }, }, }, @@ -172,23 +171,23 @@ func TestParseHint(t *testing.T) { input: "USE_INDEX(@qb1 tbl1 partition(p0) x) USE_INDEX_MERGE(@qb2 tbl2@qb2 partition(p0, p1) x, y, z)", output: []*ast.TableOptimizerHint{ { - HintName: model.NewCIStr("USE_INDEX"), + HintName: ast.NewCIStr("USE_INDEX"), Tables: []ast.HintTable{{ - TableName: model.NewCIStr("tbl1"), - PartitionList: []model.CIStr{model.NewCIStr("p0")}, + TableName: ast.NewCIStr("tbl1"), + PartitionList: []ast.CIStr{ast.NewCIStr("p0")}, }}, - QBName: model.NewCIStr("qb1"), - Indexes: []model.CIStr{model.NewCIStr("x")}, + QBName: ast.NewCIStr("qb1"), + Indexes: []ast.CIStr{ast.NewCIStr("x")}, }, { - HintName: model.NewCIStr("USE_INDEX_MERGE"), + HintName: ast.NewCIStr("USE_INDEX_MERGE"), Tables: []ast.HintTable{{ - TableName: model.NewCIStr("tbl2"), - QBName: model.NewCIStr("qb2"), - PartitionList: []model.CIStr{model.NewCIStr("p0"), model.NewCIStr("p1")}, + TableName: ast.NewCIStr("tbl2"), + QBName: ast.NewCIStr("qb2"), + PartitionList: []ast.CIStr{ast.NewCIStr("p0"), ast.NewCIStr("p1")}, }}, - QBName: model.NewCIStr("qb2"), - Indexes: []model.CIStr{model.NewCIStr("x"), model.NewCIStr("y"), model.NewCIStr("z")}, + QBName: ast.NewCIStr("qb2"), + Indexes: []ast.CIStr{ast.NewCIStr("x"), ast.NewCIStr("y"), ast.NewCIStr("z")}, }, }, }, @@ -196,35 +195,35 @@ func TestParseHint(t *testing.T) { input: `SET_VAR(sbs = 16M) SET_VAR(fkc=OFF) SET_VAR(os="mcb=off") set_var(abc=1) set_var(os2='mcb2=off')`, output: []*ast.TableOptimizerHint{ { - HintName: model.NewCIStr("SET_VAR"), + HintName: ast.NewCIStr("SET_VAR"), HintData: ast.HintSetVar{ VarName: "sbs", Value: "16M", }, }, { - HintName: model.NewCIStr("SET_VAR"), + HintName: ast.NewCIStr("SET_VAR"), HintData: ast.HintSetVar{ VarName: "fkc", Value: "OFF", }, }, { - HintName: model.NewCIStr("SET_VAR"), + HintName: ast.NewCIStr("SET_VAR"), HintData: ast.HintSetVar{ VarName: "os", Value: "mcb=off", }, }, { - HintName: model.NewCIStr("set_var"), + HintName: ast.NewCIStr("set_var"), HintData: ast.HintSetVar{ VarName: "abc", Value: "1", }, }, { - HintName: model.NewCIStr("set_var"), + HintName: ast.NewCIStr("set_var"), HintData: ast.HintSetVar{ VarName: "os2", Value: "mcb2=off", @@ -236,30 +235,30 @@ func TestParseHint(t *testing.T) { input: "USE_TOJA(TRUE) IGNORE_PLAN_CACHE() USE_CASCADES(TRUE) QUERY_TYPE(@qb1 OLAP) QUERY_TYPE(OLTP) NO_INDEX_MERGE() RESOURCE_GROUP(rg1)", output: []*ast.TableOptimizerHint{ { - HintName: model.NewCIStr("USE_TOJA"), + HintName: ast.NewCIStr("USE_TOJA"), HintData: true, }, { - HintName: model.NewCIStr("IGNORE_PLAN_CACHE"), + HintName: ast.NewCIStr("IGNORE_PLAN_CACHE"), }, { - HintName: model.NewCIStr("USE_CASCADES"), + HintName: ast.NewCIStr("USE_CASCADES"), HintData: true, }, { - HintName: model.NewCIStr("QUERY_TYPE"), - QBName: model.NewCIStr("qb1"), - HintData: model.NewCIStr("OLAP"), + HintName: ast.NewCIStr("QUERY_TYPE"), + QBName: ast.NewCIStr("qb1"), + HintData: ast.NewCIStr("OLAP"), }, { - HintName: model.NewCIStr("QUERY_TYPE"), - HintData: model.NewCIStr("OLTP"), + HintName: ast.NewCIStr("QUERY_TYPE"), + HintData: ast.NewCIStr("OLTP"), }, { - HintName: model.NewCIStr("NO_INDEX_MERGE"), + HintName: ast.NewCIStr("NO_INDEX_MERGE"), }, { - HintName: model.NewCIStr("RESOURCE_GROUP"), + HintName: ast.NewCIStr("RESOURCE_GROUP"), HintData: "rg1", }, }, @@ -268,34 +267,34 @@ func TestParseHint(t *testing.T) { input: "READ_FROM_STORAGE(@foo TIKV[a, b], TIFLASH[c, d]) HASH_AGG() SEMI_JOIN_REWRITE() READ_FROM_STORAGE(TIKV[e])", output: []*ast.TableOptimizerHint{ { - HintName: model.NewCIStr("READ_FROM_STORAGE"), - HintData: model.NewCIStr("TIKV"), - QBName: model.NewCIStr("foo"), + HintName: ast.NewCIStr("READ_FROM_STORAGE"), + HintData: ast.NewCIStr("TIKV"), + QBName: ast.NewCIStr("foo"), Tables: []ast.HintTable{ - {TableName: model.NewCIStr("a")}, - {TableName: model.NewCIStr("b")}, + {TableName: ast.NewCIStr("a")}, + {TableName: ast.NewCIStr("b")}, }, }, { - HintName: model.NewCIStr("READ_FROM_STORAGE"), - HintData: model.NewCIStr("TIFLASH"), - QBName: model.NewCIStr("foo"), + HintName: ast.NewCIStr("READ_FROM_STORAGE"), + HintData: ast.NewCIStr("TIFLASH"), + QBName: ast.NewCIStr("foo"), Tables: []ast.HintTable{ - {TableName: model.NewCIStr("c")}, - {TableName: model.NewCIStr("d")}, + {TableName: ast.NewCIStr("c")}, + {TableName: ast.NewCIStr("d")}, }, }, { - HintName: model.NewCIStr("HASH_AGG"), + HintName: ast.NewCIStr("HASH_AGG"), }, { - HintName: model.NewCIStr("SEMI_JOIN_REWRITE"), + HintName: ast.NewCIStr("SEMI_JOIN_REWRITE"), }, { - HintName: model.NewCIStr("READ_FROM_STORAGE"), - HintData: model.NewCIStr("TIKV"), + HintName: ast.NewCIStr("READ_FROM_STORAGE"), + HintData: ast.NewCIStr("TIKV"), Tables: []ast.HintTable{ - {TableName: model.NewCIStr("e")}, + {TableName: ast.NewCIStr("e")}, }, }, }, @@ -338,7 +337,7 @@ func TestParseHint(t *testing.T) { input: "TIME_RANGE('2020-02-20 12:12:12','2020-02-20 13:12:12')", output: []*ast.TableOptimizerHint{ { - HintName: model.NewCIStr("TIME_RANGE"), + HintName: ast.NewCIStr("TIME_RANGE"), HintData: ast.HintTimeRange{ From: "2020-02-20 12:12:12", To: "2020-02-20 13:12:12", diff --git a/pkg/parser/model/BUILD.bazel b/pkg/parser/model/BUILD.bazel deleted file mode 100644 index af539c1b09da7..0000000000000 --- a/pkg/parser/model/BUILD.bazel +++ /dev/null @@ -1,21 +0,0 @@ -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") - -go_library( - name = "model", - srcs = ["model.go"], - importpath = "github.com/pingcap/tidb/pkg/parser/model", - visibility = ["//visibility:public"], - deps = [ - "//pkg/parser/types", - "@com_github_pingcap_errors//:errors", - ], -) - -go_test( - name = "model_test", - timeout = "short", - srcs = ["model_test.go"], - embed = [":model"], - flaky = True, - deps = ["@com_github_stretchr_testify//require"], -) diff --git a/pkg/parser/parser.go b/pkg/parser/parser.go index 222889d8a4879..ac87233b4a95d 100644 --- a/pkg/parser/parser.go +++ b/pkg/parser/parser.go @@ -36,7 +36,6 @@ import ( "github.com/pingcap/tidb/pkg/parser/auth" "github.com/pingcap/tidb/pkg/parser/charset" "github.com/pingcap/tidb/pkg/parser/duration" - "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/opcode" "github.com/pingcap/tidb/pkg/parser/types" @@ -13433,14 +13432,14 @@ yynewstate: } case 3: { - parser.yyVAL.statement = &ast.AnalyzeTableStmt{TableNames: []*ast.TableName{yyS[yypt-4].item.(*ast.TableName)}, PartitionNames: yyS[yypt-1].item.([]model.CIStr), AnalyzeOpts: yyS[yypt-0].item.([]ast.AnalyzeOpt)} + parser.yyVAL.statement = &ast.AnalyzeTableStmt{TableNames: []*ast.TableName{yyS[yypt-4].item.(*ast.TableName)}, PartitionNames: yyS[yypt-1].item.([]ast.CIStr), AnalyzeOpts: yyS[yypt-0].item.([]ast.AnalyzeOpt)} } case 4: { parser.yyVAL.statement = &ast.AnalyzeTableStmt{ TableNames: []*ast.TableName{yyS[yypt-6].item.(*ast.TableName)}, - PartitionNames: yyS[yypt-3].item.([]model.CIStr), - IndexNames: yyS[yypt-1].item.([]model.CIStr), + PartitionNames: yyS[yypt-3].item.([]ast.CIStr), + IndexNames: yyS[yypt-1].item.([]ast.CIStr), IndexFlag: true, AnalyzeOpts: yyS[yypt-0].item.([]ast.AnalyzeOpt), } @@ -13463,7 +13462,7 @@ yynewstate: { parser.yyVAL.statement = &ast.CompactTableStmt{ Table: yyS[yypt-3].item.(*ast.TableName), - PartitionNames: yyS[yypt-0].item.([]model.CIStr), + PartitionNames: yyS[yypt-0].item.([]ast.CIStr), ReplicaKind: ast.CompactReplicaKindAll, } } @@ -13471,7 +13470,7 @@ yynewstate: { parser.yyVAL.statement = &ast.CompactTableStmt{ Table: yyS[yypt-5].item.(*ast.TableName), - PartitionNames: yyS[yypt-2].item.([]model.CIStr), + PartitionNames: yyS[yypt-2].item.([]ast.CIStr), ReplicaKind: ast.CompactReplicaKindTiFlash, } } @@ -13529,33 +13528,33 @@ yynewstate: } case 18: { - parser.yyVAL.item = model.WatchExact + parser.yyVAL.item = ast.WatchExact } case 19: { - parser.yyVAL.item = model.WatchSimilar + parser.yyVAL.item = ast.WatchSimilar } case 20: { - parser.yyVAL.item = model.WatchPlan + parser.yyVAL.item = ast.WatchPlan } case 21: { - parser.yyVAL.item = &ast.ResourceGroupRunawayActionOption{Type: model.RunawayActionDryRun} + parser.yyVAL.item = &ast.ResourceGroupRunawayActionOption{Type: ast.RunawayActionDryRun} } case 22: { - parser.yyVAL.item = &ast.ResourceGroupRunawayActionOption{Type: model.RunawayActionCooldown} + parser.yyVAL.item = &ast.ResourceGroupRunawayActionOption{Type: ast.RunawayActionCooldown} } case 23: { - parser.yyVAL.item = &ast.ResourceGroupRunawayActionOption{Type: model.RunawayActionKill} + parser.yyVAL.item = &ast.ResourceGroupRunawayActionOption{Type: ast.RunawayActionKill} } case 24: { parser.yyVAL.item = &ast.ResourceGroupRunawayActionOption{ - Type: model.RunawayActionSwitchGroup, - SwitchGroupName: model.NewCIStr(yyS[yypt-1].ident), + Type: ast.RunawayActionSwitchGroup, + SwitchGroupName: ast.NewCIStr(yyS[yypt-1].ident), } } case 25: @@ -13566,28 +13565,28 @@ yynewstate: return 1 } parser.yyVAL.item = &ast.ResourceGroupRunawayOption{ - Tp: model.RunawayRule, + Tp: ast.RunawayRule, RuleOption: &ast.ResourceGroupRunawayRuleOption{Tp: ast.RunawayRuleExecElapsed, ExecElapsed: yyS[yypt-0].ident}, } } case 26: { parser.yyVAL.item = &ast.ResourceGroupRunawayOption{ - Tp: model.RunawayRule, + Tp: ast.RunawayRule, RuleOption: &ast.ResourceGroupRunawayRuleOption{Tp: ast.RunawayRuleProcessedKeys, ProcessedKeys: yyS[yypt-0].item.(int64)}, } } case 27: { parser.yyVAL.item = &ast.ResourceGroupRunawayOption{ - Tp: model.RunawayRule, + Tp: ast.RunawayRule, RuleOption: &ast.ResourceGroupRunawayRuleOption{Tp: ast.RunawayRuleRequestUnit, RequestUnit: yyS[yypt-0].item.(int64)}, } } case 28: { parser.yyVAL.item = &ast.ResourceGroupRunawayOption{ - Tp: model.RunawayAction, + Tp: ast.RunawayAction, ActionOption: yyS[yypt-0].item.(*ast.ResourceGroupRunawayActionOption), } } @@ -13605,9 +13604,9 @@ yynewstate: } } parser.yyVAL.item = &ast.ResourceGroupRunawayOption{ - Tp: model.RunawayWatch, + Tp: ast.RunawayWatch, WatchOption: &ast.ResourceGroupRunawayWatchOption{ - Type: yyS[yypt-1].item.(model.RunawayWatchType), + Type: yyS[yypt-1].item.(ast.RunawayWatchType), Duration: dur, }, } @@ -13842,7 +13841,7 @@ yynewstate: { parser.yyVAL.item = &ast.AlterTableSpec{ Tp: ast.AlterTablePartitionAttributes, - PartitionNames: []model.CIStr{model.NewCIStr(yyS[yypt-1].ident)}, + PartitionNames: []ast.CIStr{ast.NewCIStr(yyS[yypt-1].ident)}, AttributesSpec: yyS[yypt-0].item.(*ast.AttributesSpec), } } @@ -13850,7 +13849,7 @@ yynewstate: { parser.yyVAL.item = &ast.AlterTableSpec{ Tp: ast.AlterTablePartitionOptions, - PartitionNames: []model.CIStr{model.NewCIStr(yyS[yypt-1].ident)}, + PartitionNames: []ast.CIStr{ast.NewCIStr(yyS[yypt-1].ident)}, Options: yyS[yypt-0].item.([]*ast.TableOption), } } @@ -14053,7 +14052,7 @@ yynewstate: if yyS[yypt-0].item == nil { ret.OnAllPartitions = true } else { - ret.PartitionNames = yyS[yypt-0].item.([]model.CIStr) + ret.PartitionNames = yyS[yypt-0].item.([]ast.CIStr) } parser.yyVAL.item = ret } @@ -14087,7 +14086,7 @@ yynewstate: parser.yyVAL.item = &ast.AlterTableSpec{ IfExists: yyS[yypt-1].item.(bool), Tp: ast.AlterTableDropPartition, - PartitionNames: yyS[yypt-0].item.([]model.CIStr), + PartitionNames: yyS[yypt-0].item.([]ast.CIStr), } } case 102: @@ -14117,7 +14116,7 @@ yynewstate: { parser.yyVAL.item = &ast.AlterTableSpec{ Tp: ast.AlterTableExchangePartition, - PartitionNames: []model.CIStr{model.NewCIStr(yyS[yypt-4].ident)}, + PartitionNames: []ast.CIStr{ast.NewCIStr(yyS[yypt-4].ident)}, NewTable: yyS[yypt-1].item.(*ast.TableName), WithValidation: yyS[yypt-0].item.(bool), } @@ -14130,7 +14129,7 @@ yynewstate: if yyS[yypt-0].item == nil { ret.OnAllPartitions = true } else { - ret.PartitionNames = yyS[yypt-0].item.([]model.CIStr) + ret.PartitionNames = yyS[yypt-0].item.([]ast.CIStr) } parser.yyVAL.item = ret } @@ -14143,7 +14142,7 @@ yynewstate: if yyS[yypt-0].item == nil { ret.OnAllPartitions = true } else { - ret.PartitionNames = yyS[yypt-0].item.([]model.CIStr) + ret.PartitionNames = yyS[yypt-0].item.([]ast.CIStr) } parser.yyVAL.item = ret } @@ -14156,7 +14155,7 @@ yynewstate: if yyS[yypt-0].item == nil { ret.OnAllPartitions = true } else { - ret.PartitionNames = yyS[yypt-0].item.([]model.CIStr) + ret.PartitionNames = yyS[yypt-0].item.([]ast.CIStr) } parser.yyVAL.item = ret } @@ -14168,7 +14167,7 @@ yynewstate: if yyS[yypt-1].item == nil { ret.OnAllPartitions = true } else { - ret.PartitionNames = yyS[yypt-1].item.([]model.CIStr) + ret.PartitionNames = yyS[yypt-1].item.([]ast.CIStr) } parser.yyVAL.item = ret yylex.AppendError(yylex.Errorf("The IMPORT PARTITION TABLESPACE clause is parsed but ignored by all storage engines.")) @@ -14182,7 +14181,7 @@ yynewstate: if yyS[yypt-1].item == nil { ret.OnAllPartitions = true } else { - ret.PartitionNames = yyS[yypt-1].item.([]model.CIStr) + ret.PartitionNames = yyS[yypt-1].item.([]ast.CIStr) } parser.yyVAL.item = ret yylex.AppendError(yylex.Errorf("The DISCARD PARTITION TABLESPACE clause is parsed but ignored by all storage engines.")) @@ -14215,7 +14214,7 @@ yynewstate: if yyS[yypt-0].item == nil { ret.OnAllPartitions = true } else { - ret.PartitionNames = yyS[yypt-0].item.([]model.CIStr) + ret.PartitionNames = yyS[yypt-0].item.([]ast.CIStr) } parser.yyVAL.item = ret } @@ -14308,8 +14307,8 @@ yynewstate: } case 123: { - oldColName := &ast.ColumnName{Name: model.NewCIStr(yyS[yypt-2].ident)} - newColName := &ast.ColumnName{Name: model.NewCIStr(yyS[yypt-0].ident)} + oldColName := &ast.ColumnName{Name: ast.NewCIStr(yyS[yypt-2].ident)} + newColName := &ast.ColumnName{Name: ast.NewCIStr(yyS[yypt-0].ident)} parser.yyVAL.item = &ast.AlterTableSpec{ Tp: ast.AlterTableRenameColumn, OldColumnName: oldColName, @@ -14341,8 +14340,8 @@ yynewstate: { parser.yyVAL.item = &ast.AlterTableSpec{ Tp: ast.AlterTableRenameIndex, - FromKey: model.NewCIStr(yyS[yypt-2].ident), - ToKey: model.NewCIStr(yyS[yypt-0].ident), + FromKey: ast.NewCIStr(yyS[yypt-2].ident), + ToKey: ast.NewCIStr(yyS[yypt-0].ident), } } case 128: @@ -14432,7 +14431,7 @@ yynewstate: { parser.yyVAL.item = &ast.AlterTableSpec{ Tp: ast.AlterTableIndexInvisible, - IndexName: model.NewCIStr(yyS[yypt-1].ident), + IndexName: ast.NewCIStr(yyS[yypt-1].ident), Visibility: yyS[yypt-0].item.(ast.IndexVisibility), } } @@ -14460,7 +14459,7 @@ yynewstate: { ret := &ast.AlterTableSpec{ Tp: ast.AlterTableReorganizePartition, - PartitionNames: yyS[yypt-4].item.([]model.CIStr), + PartitionNames: yyS[yypt-4].item.([]ast.CIStr), PartDefinitions: yyS[yypt-1].item.([]*ast.PartitionDefinition), } parser.yyVAL.item = ret @@ -14483,11 +14482,11 @@ yynewstate: } case 149: { - parser.yyVAL.item = model.PrimaryKeyTypeClustered + parser.yyVAL.item = ast.PrimaryKeyTypeClustered } case 150: { - parser.yyVAL.item = model.PrimaryKeyTypeNonClustered + parser.yyVAL.item = ast.PrimaryKeyTypeNonClustered } case 151: { @@ -14578,11 +14577,11 @@ yynewstate: } case 176: { - parser.yyVAL.item = []model.CIStr{model.NewCIStr(yyS[yypt-0].ident)} + parser.yyVAL.item = []ast.CIStr{ast.NewCIStr(yyS[yypt-0].ident)} } case 177: { - parser.yyVAL.item = append(yyS[yypt-2].item.([]model.CIStr), model.NewCIStr(yyS[yypt-0].ident)) + parser.yyVAL.item = append(yyS[yypt-2].item.([]ast.CIStr), ast.NewCIStr(yyS[yypt-0].ident)) } case 178: { @@ -14675,7 +14674,7 @@ yynewstate: case 195: { parser.yyVAL.statement = &ast.FlashBackToTimestampStmt{ - DBName: model.NewCIStr(yyS[yypt-2].ident), + DBName: ast.NewCIStr(yyS[yypt-2].ident), FlashbackTS: ast.NewValueExpr(yyS[yypt-0].ident, "", ""), FlashbackTSO: 0, } @@ -14707,7 +14706,7 @@ yynewstate: { if tsoValue, ok := yyS[yypt-0].item.(uint64); ok && tsoValue > 0 { parser.yyVAL.statement = &ast.FlashBackToTimestampStmt{ - DBName: model.NewCIStr(yyS[yypt-2].ident), + DBName: ast.NewCIStr(yyS[yypt-2].ident), FlashbackTSO: tsoValue, } } else { @@ -14733,7 +14732,7 @@ yynewstate: case 202: { parser.yyVAL.statement = &ast.FlashBackDatabaseStmt{ - DBName: model.NewCIStr(yyS[yypt-1].ident), + DBName: ast.NewCIStr(yyS[yypt-1].ident), NewName: yyS[yypt-0].ident, } } @@ -14742,7 +14741,7 @@ yynewstate: parser.yyVAL.statement = &ast.SplitRegionStmt{ SplitSyntaxOpt: yyS[yypt-4].item.(*ast.SplitSyntaxOption), Table: yyS[yypt-2].item.(*ast.TableName), - PartitionNames: yyS[yypt-1].item.([]model.CIStr), + PartitionNames: yyS[yypt-1].item.([]ast.CIStr), SplitOpt: yyS[yypt-0].item.(*ast.SplitOption), } } @@ -14751,8 +14750,8 @@ yynewstate: parser.yyVAL.statement = &ast.SplitRegionStmt{ SplitSyntaxOpt: yyS[yypt-6].item.(*ast.SplitSyntaxOption), Table: yyS[yypt-4].item.(*ast.TableName), - PartitionNames: yyS[yypt-3].item.([]model.CIStr), - IndexName: model.NewCIStr(yyS[yypt-1].ident), + PartitionNames: yyS[yypt-3].item.([]ast.CIStr), + IndexName: ast.NewCIStr(yyS[yypt-1].ident), SplitOpt: yyS[yypt-0].item.(*ast.SplitOption), } } @@ -14795,27 +14794,27 @@ yynewstate: } case 211: { - parser.yyVAL.statement = &ast.AnalyzeTableStmt{TableNames: yyS[yypt-2].item.([]*ast.TableName), NoWriteToBinLog: yyS[yypt-4].item.(bool), ColumnChoice: yyS[yypt-1].item.(model.ColumnChoice), AnalyzeOpts: yyS[yypt-0].item.([]ast.AnalyzeOpt)} + parser.yyVAL.statement = &ast.AnalyzeTableStmt{TableNames: yyS[yypt-2].item.([]*ast.TableName), NoWriteToBinLog: yyS[yypt-4].item.(bool), ColumnChoice: yyS[yypt-1].item.(ast.ColumnChoice), AnalyzeOpts: yyS[yypt-0].item.([]ast.AnalyzeOpt)} } case 212: { - parser.yyVAL.statement = &ast.AnalyzeTableStmt{TableNames: []*ast.TableName{yyS[yypt-3].item.(*ast.TableName)}, NoWriteToBinLog: yyS[yypt-5].item.(bool), IndexNames: yyS[yypt-1].item.([]model.CIStr), IndexFlag: true, AnalyzeOpts: yyS[yypt-0].item.([]ast.AnalyzeOpt)} + parser.yyVAL.statement = &ast.AnalyzeTableStmt{TableNames: []*ast.TableName{yyS[yypt-3].item.(*ast.TableName)}, NoWriteToBinLog: yyS[yypt-5].item.(bool), IndexNames: yyS[yypt-1].item.([]ast.CIStr), IndexFlag: true, AnalyzeOpts: yyS[yypt-0].item.([]ast.AnalyzeOpt)} } case 213: { - parser.yyVAL.statement = &ast.AnalyzeTableStmt{TableNames: []*ast.TableName{yyS[yypt-3].item.(*ast.TableName)}, NoWriteToBinLog: yyS[yypt-6].item.(bool), IndexNames: yyS[yypt-1].item.([]model.CIStr), IndexFlag: true, Incremental: true, AnalyzeOpts: yyS[yypt-0].item.([]ast.AnalyzeOpt)} + parser.yyVAL.statement = &ast.AnalyzeTableStmt{TableNames: []*ast.TableName{yyS[yypt-3].item.(*ast.TableName)}, NoWriteToBinLog: yyS[yypt-6].item.(bool), IndexNames: yyS[yypt-1].item.([]ast.CIStr), IndexFlag: true, Incremental: true, AnalyzeOpts: yyS[yypt-0].item.([]ast.AnalyzeOpt)} } case 214: { - parser.yyVAL.statement = &ast.AnalyzeTableStmt{TableNames: []*ast.TableName{yyS[yypt-4].item.(*ast.TableName)}, NoWriteToBinLog: yyS[yypt-6].item.(bool), PartitionNames: yyS[yypt-2].item.([]model.CIStr), ColumnChoice: yyS[yypt-1].item.(model.ColumnChoice), AnalyzeOpts: yyS[yypt-0].item.([]ast.AnalyzeOpt)} + parser.yyVAL.statement = &ast.AnalyzeTableStmt{TableNames: []*ast.TableName{yyS[yypt-4].item.(*ast.TableName)}, NoWriteToBinLog: yyS[yypt-6].item.(bool), PartitionNames: yyS[yypt-2].item.([]ast.CIStr), ColumnChoice: yyS[yypt-1].item.(ast.ColumnChoice), AnalyzeOpts: yyS[yypt-0].item.([]ast.AnalyzeOpt)} } case 215: { parser.yyVAL.statement = &ast.AnalyzeTableStmt{ TableNames: []*ast.TableName{yyS[yypt-5].item.(*ast.TableName)}, NoWriteToBinLog: yyS[yypt-7].item.(bool), - PartitionNames: yyS[yypt-3].item.([]model.CIStr), - IndexNames: yyS[yypt-1].item.([]model.CIStr), + PartitionNames: yyS[yypt-3].item.([]ast.CIStr), + IndexNames: yyS[yypt-1].item.([]ast.CIStr), IndexFlag: true, AnalyzeOpts: yyS[yypt-0].item.([]ast.AnalyzeOpt), } @@ -14825,8 +14824,8 @@ yynewstate: parser.yyVAL.statement = &ast.AnalyzeTableStmt{ TableNames: []*ast.TableName{yyS[yypt-5].item.(*ast.TableName)}, NoWriteToBinLog: yyS[yypt-8].item.(bool), - PartitionNames: yyS[yypt-3].item.([]model.CIStr), - IndexNames: yyS[yypt-1].item.([]model.CIStr), + PartitionNames: yyS[yypt-3].item.([]ast.CIStr), + IndexNames: yyS[yypt-1].item.([]ast.CIStr), IndexFlag: true, Incremental: true, AnalyzeOpts: yyS[yypt-0].item.([]ast.AnalyzeOpt), @@ -14837,7 +14836,7 @@ yynewstate: parser.yyVAL.statement = &ast.AnalyzeTableStmt{ TableNames: []*ast.TableName{yyS[yypt-5].item.(*ast.TableName)}, NoWriteToBinLog: yyS[yypt-7].item.(bool), - ColumnNames: yyS[yypt-1].item.([]model.CIStr), + ColumnNames: yyS[yypt-1].item.([]ast.CIStr), AnalyzeOpts: yyS[yypt-0].item.([]ast.AnalyzeOpt), HistogramOperation: ast.HistogramOperationUpdate, } @@ -14847,7 +14846,7 @@ yynewstate: parser.yyVAL.statement = &ast.AnalyzeTableStmt{ TableNames: []*ast.TableName{yyS[yypt-4].item.(*ast.TableName)}, NoWriteToBinLog: yyS[yypt-6].item.(bool), - ColumnNames: yyS[yypt-0].item.([]model.CIStr), + ColumnNames: yyS[yypt-0].item.([]ast.CIStr), HistogramOperation: ast.HistogramOperationDrop, } } @@ -14856,8 +14855,8 @@ yynewstate: parser.yyVAL.statement = &ast.AnalyzeTableStmt{ TableNames: []*ast.TableName{yyS[yypt-3].item.(*ast.TableName)}, NoWriteToBinLog: yyS[yypt-5].item.(bool), - ColumnNames: yyS[yypt-1].item.([]model.CIStr), - ColumnChoice: model.ColumnList, + ColumnNames: yyS[yypt-1].item.([]ast.CIStr), + ColumnChoice: ast.ColumnList, AnalyzeOpts: yyS[yypt-0].item.([]ast.AnalyzeOpt)} } case 220: @@ -14865,22 +14864,22 @@ yynewstate: parser.yyVAL.statement = &ast.AnalyzeTableStmt{ TableNames: []*ast.TableName{yyS[yypt-5].item.(*ast.TableName)}, NoWriteToBinLog: yyS[yypt-7].item.(bool), - PartitionNames: yyS[yypt-3].item.([]model.CIStr), - ColumnNames: yyS[yypt-1].item.([]model.CIStr), - ColumnChoice: model.ColumnList, + PartitionNames: yyS[yypt-3].item.([]ast.CIStr), + ColumnNames: yyS[yypt-1].item.([]ast.CIStr), + ColumnChoice: ast.ColumnList, AnalyzeOpts: yyS[yypt-0].item.([]ast.AnalyzeOpt)} } case 221: { - parser.yyVAL.item = model.DefaultChoice + parser.yyVAL.item = ast.DefaultChoice } case 222: { - parser.yyVAL.item = model.AllColumns + parser.yyVAL.item = ast.AllColumns } case 223: { - parser.yyVAL.item = model.PredicateColumns + parser.yyVAL.item = ast.PredicateColumns } case 224: { @@ -15010,15 +15009,15 @@ yynewstate: } case 249: { - parser.yyVAL.item = &ast.ColumnName{Name: model.NewCIStr(yyS[yypt-0].ident)} + parser.yyVAL.item = &ast.ColumnName{Name: ast.NewCIStr(yyS[yypt-0].ident)} } case 250: { - parser.yyVAL.item = &ast.ColumnName{Table: model.NewCIStr(yyS[yypt-2].ident), Name: model.NewCIStr(yyS[yypt-0].ident)} + parser.yyVAL.item = &ast.ColumnName{Table: ast.NewCIStr(yyS[yypt-2].ident), Name: ast.NewCIStr(yyS[yypt-0].ident)} } case 251: { - parser.yyVAL.item = &ast.ColumnName{Schema: model.NewCIStr(yyS[yypt-4].ident), Table: model.NewCIStr(yyS[yypt-2].ident), Name: model.NewCIStr(yyS[yypt-0].ident)} + parser.yyVAL.item = &ast.ColumnName{Schema: ast.NewCIStr(yyS[yypt-4].ident), Table: ast.NewCIStr(yyS[yypt-2].ident), Name: ast.NewCIStr(yyS[yypt-0].ident)} } case 252: { @@ -15034,7 +15033,7 @@ yynewstate: } case 256: { - parser.yyVAL.item = []model.CIStr{} + parser.yyVAL.item = []ast.CIStr{} } case 257: { @@ -15042,11 +15041,11 @@ yynewstate: } case 258: { - parser.yyVAL.item = []model.CIStr{model.NewCIStr(yyS[yypt-0].ident)} + parser.yyVAL.item = []ast.CIStr{ast.NewCIStr(yyS[yypt-0].ident)} } case 259: { - parser.yyVAL.item = append(yyS[yypt-2].item.([]model.CIStr), model.NewCIStr(yyS[yypt-0].ident)) + parser.yyVAL.item = append(yyS[yypt-2].item.([]ast.CIStr), ast.NewCIStr(yyS[yypt-0].ident)) } case 260: { @@ -15136,7 +15135,7 @@ yynewstate: // KEY is normally a synonym for INDEX. The key attribute PRIMARY KEY // can also be specified as just KEY when given in a column definition. // See http://dev.mysql.com/doc/refman/5.7/en/create-table.html - parser.yyVAL.item = &ast.ColumnOption{Tp: ast.ColumnOptionPrimaryKey, PrimaryKeyTp: yyS[yypt-1].item.(model.PrimaryKeyType), StrValue: yyS[yypt-0].ident} + parser.yyVAL.item = &ast.ColumnOption{Tp: ast.ColumnOptionPrimaryKey, PrimaryKeyTp: yyS[yypt-1].item.(ast.PrimaryKeyType), StrValue: yyS[yypt-0].ident} } case 285: { @@ -15306,7 +15305,7 @@ yynewstate: if c.Option == nil { c.Option = &ast.IndexOption{} } - c.Option.Tp = indexType.(model.IndexType) + c.Option.Tp = indexType.(ast.IndexType) } parser.yyVAL.item = c } @@ -15339,7 +15338,7 @@ yynewstate: if c.Option == nil { c.Option = &ast.IndexOption{} } - c.Option.Tp = indexType.(model.IndexType) + c.Option.Tp = indexType.(ast.IndexType) } parser.yyVAL.item = c } @@ -15359,7 +15358,7 @@ yynewstate: if c.Option == nil { c.Option = &ast.IndexOption{} } - c.Option.Tp = indexType.(model.IndexType) + c.Option.Tp = indexType.(ast.IndexType) } parser.yyVAL.item = c } @@ -15417,11 +15416,11 @@ yynewstate: } case 330: { - parser.yyVAL.item = &ast.OnDeleteOpt{ReferOpt: yyS[yypt-0].item.(model.ReferOptionType)} + parser.yyVAL.item = &ast.OnDeleteOpt{ReferOpt: yyS[yypt-0].item.(ast.ReferOptionType)} } case 331: { - parser.yyVAL.item = &ast.OnUpdateOpt{ReferOpt: yyS[yypt-0].item.(model.ReferOptionType)} + parser.yyVAL.item = &ast.OnUpdateOpt{ReferOpt: yyS[yypt-0].item.(ast.ReferOptionType)} } case 332: { @@ -15445,23 +15444,23 @@ yynewstate: } case 337: { - parser.yyVAL.item = model.ReferOptionRestrict + parser.yyVAL.item = ast.ReferOptionRestrict } case 338: { - parser.yyVAL.item = model.ReferOptionCascade + parser.yyVAL.item = ast.ReferOptionCascade } case 339: { - parser.yyVAL.item = model.ReferOptionSetNull + parser.yyVAL.item = ast.ReferOptionSetNull } case 340: { - parser.yyVAL.item = model.ReferOptionNoAction + parser.yyVAL.item = ast.ReferOptionNoAction } case 341: { - parser.yyVAL.item = model.ReferOptionSetDefault + parser.yyVAL.item = ast.ReferOptionSetDefault yylex.AppendError(yylex.Errorf("The SET DEFAULT clause is parsed but ignored by all storage engines.")) parser.lastErrorAsWarn() } @@ -15472,20 +15471,20 @@ yynewstate: case 347: { parser.yyVAL.expr = &ast.FuncCallExpr{ - FnName: model.NewCIStr(yyS[yypt-2].ident), + FnName: ast.NewCIStr(yyS[yypt-2].ident), } } case 348: { parser.yyVAL.expr = &ast.FuncCallExpr{ - FnName: model.NewCIStr(yyS[yypt-3].ident), + FnName: ast.NewCIStr(yyS[yypt-3].ident), Args: yyS[yypt-1].item.([]ast.ExprNode), } } case 349: { parser.yyVAL.expr = &ast.FuncCallExpr{ - FnName: model.NewCIStr(yyS[yypt-3].ident), + FnName: ast.NewCIStr(yyS[yypt-3].ident), Args: yyS[yypt-1].item.([]ast.ExprNode), } } @@ -15495,23 +15494,23 @@ yynewstate: } case 352: { - parser.yyVAL.expr = &ast.FuncCallExpr{FnName: model.NewCIStr("CURRENT_TIMESTAMP")} + parser.yyVAL.expr = &ast.FuncCallExpr{FnName: ast.NewCIStr("CURRENT_TIMESTAMP")} } case 353: { - parser.yyVAL.expr = &ast.FuncCallExpr{FnName: model.NewCIStr("CURRENT_TIMESTAMP")} + parser.yyVAL.expr = &ast.FuncCallExpr{FnName: ast.NewCIStr("CURRENT_TIMESTAMP")} } case 354: { - parser.yyVAL.expr = &ast.FuncCallExpr{FnName: model.NewCIStr("CURRENT_TIMESTAMP"), Args: []ast.ExprNode{ast.NewValueExpr(yyS[yypt-1].item, parser.charset, parser.collation)}} + parser.yyVAL.expr = &ast.FuncCallExpr{FnName: ast.NewCIStr("CURRENT_TIMESTAMP"), Args: []ast.ExprNode{ast.NewValueExpr(yyS[yypt-1].item, parser.charset, parser.collation)}} } case 355: { - parser.yyVAL.expr = &ast.FuncCallExpr{FnName: model.NewCIStr("CURRENT_DATE")} + parser.yyVAL.expr = &ast.FuncCallExpr{FnName: ast.NewCIStr("CURRENT_DATE")} } case 356: { - parser.yyVAL.expr = &ast.FuncCallExpr{FnName: model.NewCIStr("CURRENT_DATE")} + parser.yyVAL.expr = &ast.FuncCallExpr{FnName: ast.NewCIStr("CURRENT_DATE")} } case 357: { @@ -15523,7 +15522,7 @@ yynewstate: Name: yyS[yypt-0].item.(*ast.TableName), } parser.yyVAL.expr = &ast.FuncCallExpr{ - FnName: model.NewCIStr(ast.NextVal), + FnName: ast.NewCIStr(ast.NextVal), Args: []ast.ExprNode{objNameExpr}, } } @@ -15533,7 +15532,7 @@ yynewstate: Name: yyS[yypt-1].item.(*ast.TableName), } parser.yyVAL.expr = &ast.FuncCallExpr{ - FnName: model.NewCIStr(ast.NextVal), + FnName: ast.NewCIStr(ast.NextVal), Args: []ast.ExprNode{objNameExpr}, } } @@ -15588,15 +15587,15 @@ yynewstate: var indexOption *ast.IndexOption if yyS[yypt-1].item != nil { indexOption = yyS[yypt-1].item.(*ast.IndexOption) - if indexOption.Tp == model.IndexTypeInvalid { + if indexOption.Tp == ast.IndexTypeInvalid { if yyS[yypt-7].item != nil { - indexOption.Tp = yyS[yypt-7].item.(model.IndexType) + indexOption.Tp = yyS[yypt-7].item.(ast.IndexType) } } } else { indexOption = &ast.IndexOption{} if yyS[yypt-7].item != nil { - indexOption.Tp = yyS[yypt-7].item.(model.IndexType) + indexOption.Tp = yyS[yypt-7].item.(ast.IndexType) } } var indexLockAndAlgorithm *ast.IndexLockAndAlgorithm @@ -15609,10 +15608,10 @@ yynewstate: keyType := yyS[yypt-11].item.(ast.IndexKeyType) isVectorIndex := keyType == ast.IndexKeyTypeVector - if isVectorIndex && indexOption.Tp == model.IndexTypeInvalid { - indexOption.Tp = model.IndexTypeHNSW + if isVectorIndex && indexOption.Tp == ast.IndexTypeInvalid { + indexOption.Tp = ast.IndexTypeHNSW } - if (isVectorIndex && indexOption.Tp != model.IndexTypeHNSW) || (!isVectorIndex && indexOption.Tp == model.IndexTypeHNSW) { + if (isVectorIndex && indexOption.Tp != ast.IndexTypeHNSW) || (!isVectorIndex && indexOption.Tp == ast.IndexTypeHNSW) { yylex.AppendError(ErrSyntax) return 1 } @@ -15713,7 +15712,7 @@ yynewstate: case 400: { parser.yyVAL.statement = &ast.AlterDatabaseStmt{ - Name: model.NewCIStr(yyS[yypt-1].ident), + Name: ast.NewCIStr(yyS[yypt-1].ident), AlterDefaultDatabase: false, Options: yyS[yypt-0].item.([]*ast.DatabaseOption), } @@ -15721,7 +15720,7 @@ yynewstate: case 401: { parser.yyVAL.statement = &ast.AlterDatabaseStmt{ - Name: model.NewCIStr(""), + Name: ast.NewCIStr(""), AlterDefaultDatabase: true, Options: yyS[yypt-0].item.([]*ast.DatabaseOption), } @@ -15730,7 +15729,7 @@ yynewstate: { parser.yyVAL.statement = &ast.CreateDatabaseStmt{ IfNotExists: yyS[yypt-2].item.(bool), - Name: model.NewCIStr(yyS[yypt-1].ident), + Name: ast.NewCIStr(yyS[yypt-1].ident), Options: yyS[yypt-0].item.([]*ast.DatabaseOption), } } @@ -15898,7 +15897,7 @@ yynewstate: { keyAlgorithm, _ := yyS[yypt-3].item.(*ast.PartitionKeyAlgorithm) parser.yyVAL.item = &ast.PartitionMethod{ - Tp: model.PartitionTypeKey, + Tp: ast.PartitionTypeKey, Linear: len(yyS[yypt-5].ident) != 0, ColumnNames: yyS[yypt-1].item.([]*ast.ColumnName), KeyAlgorithm: keyAlgorithm, @@ -15907,7 +15906,7 @@ yynewstate: case 434: { parser.yyVAL.item = &ast.PartitionMethod{ - Tp: model.PartitionTypeHash, + Tp: ast.PartitionTypeHash, Linear: len(yyS[yypt-4].ident) != 0, Expr: yyS[yypt-1].expr.(ast.ExprNode), } @@ -15931,7 +15930,7 @@ yynewstate: { partitionInterval, _ := yyS[yypt-0].item.(*ast.PartitionInterval) parser.yyVAL.item = &ast.PartitionMethod{ - Tp: model.PartitionTypeRange, + Tp: ast.PartitionTypeRange, Expr: yyS[yypt-2].expr.(ast.ExprNode), Interval: partitionInterval, } @@ -15940,7 +15939,7 @@ yynewstate: { partitionInterval, _ := yyS[yypt-0].item.(*ast.PartitionInterval) parser.yyVAL.item = &ast.PartitionMethod{ - Tp: model.PartitionTypeRange, + Tp: ast.PartitionTypeRange, ColumnNames: yyS[yypt-2].item.([]*ast.ColumnName), Interval: partitionInterval, } @@ -15948,21 +15947,21 @@ yynewstate: case 440: { parser.yyVAL.item = &ast.PartitionMethod{ - Tp: model.PartitionTypeList, + Tp: ast.PartitionTypeList, Expr: yyS[yypt-1].expr.(ast.ExprNode), } } case 441: { parser.yyVAL.item = &ast.PartitionMethod{ - Tp: model.PartitionTypeList, + Tp: ast.PartitionTypeList, ColumnNames: yyS[yypt-1].item.([]*ast.ColumnName), } } case 442: { parser.yyVAL.item = &ast.PartitionMethod{ - Tp: model.PartitionTypeSystemTime, + Tp: ast.PartitionTypeSystemTime, Expr: yyS[yypt-1].expr.(ast.ExprNode), Unit: yyS[yypt-0].item.(ast.TimeUnitType), } @@ -15970,14 +15969,14 @@ yynewstate: case 443: { parser.yyVAL.item = &ast.PartitionMethod{ - Tp: model.PartitionTypeSystemTime, + Tp: ast.PartitionTypeSystemTime, Limit: yyS[yypt-0].item.(uint64), } } case 444: { parser.yyVAL.item = &ast.PartitionMethod{ - Tp: model.PartitionTypeSystemTime, + Tp: ast.PartitionTypeSystemTime, } } case 445: @@ -16094,7 +16093,7 @@ yynewstate: case 467: { parser.yyVAL.item = &ast.PartitionDefinition{ - Name: model.NewCIStr(yyS[yypt-3].ident), + Name: ast.NewCIStr(yyS[yypt-3].ident), Clause: yyS[yypt-2].item.(ast.PartitionDefinitionClause), Options: yyS[yypt-1].item.([]*ast.TableOption), Sub: yyS[yypt-0].item.([]*ast.SubPartitionDefinition), @@ -16120,7 +16119,7 @@ yynewstate: case 472: { parser.yyVAL.item = &ast.SubPartitionDefinition{ - Name: model.NewCIStr(yyS[yypt-1].ident), + Name: ast.NewCIStr(yyS[yypt-1].ident), Options: yyS[yypt-0].item.([]*ast.TableOption), } } @@ -16297,18 +16296,18 @@ yynewstate: OrReplace: yyS[yypt-9].item.(bool), ViewName: yyS[yypt-4].item.(*ast.TableName), Select: selStmt, - Algorithm: yyS[yypt-8].item.(model.ViewAlgorithm), + Algorithm: yyS[yypt-8].item.(ast.ViewAlgorithm), Definer: yyS[yypt-7].item.(*auth.UserIdentity), - Security: yyS[yypt-6].item.(model.ViewSecurity), + Security: yyS[yypt-6].item.(ast.ViewSecurity), } if yyS[yypt-3].item != nil { - x.Cols = yyS[yypt-3].item.([]model.CIStr) + x.Cols = yyS[yypt-3].item.([]ast.CIStr) } if yyS[yypt-0].item != nil { - x.CheckOption = yyS[yypt-0].item.(model.ViewCheckOption) + x.CheckOption = yyS[yypt-0].item.(ast.ViewCheckOption) endOffset = parser.startOffset(&yyS[yypt]) } else { - x.CheckOption = model.CheckOptionCascaded + x.CheckOption = ast.CheckOptionCascaded } selStmt.SetText(parser.lexer.client, strings.TrimSpace(parser.src[startOffset:endOffset])) parser.yyVAL.statement = x @@ -16323,19 +16322,19 @@ yynewstate: } case 512: { - parser.yyVAL.item = model.AlgorithmUndefined + parser.yyVAL.item = ast.AlgorithmUndefined } case 513: { - parser.yyVAL.item = model.AlgorithmUndefined + parser.yyVAL.item = ast.AlgorithmUndefined } case 514: { - parser.yyVAL.item = model.AlgorithmMerge + parser.yyVAL.item = ast.AlgorithmMerge } case 515: { - parser.yyVAL.item = model.AlgorithmTemptable + parser.yyVAL.item = ast.AlgorithmTemptable } case 516: { @@ -16347,15 +16346,15 @@ yynewstate: } case 518: { - parser.yyVAL.item = model.SecurityDefiner + parser.yyVAL.item = ast.SecurityDefiner } case 519: { - parser.yyVAL.item = model.SecurityDefiner + parser.yyVAL.item = ast.SecurityDefiner } case 520: { - parser.yyVAL.item = model.SecurityInvoker + parser.yyVAL.item = ast.SecurityInvoker } case 522: { @@ -16363,15 +16362,15 @@ yynewstate: } case 523: { - parser.yyVAL.item = yyS[yypt-1].item.([]model.CIStr) + parser.yyVAL.item = yyS[yypt-1].item.([]ast.CIStr) } case 524: { - parser.yyVAL.item = []model.CIStr{model.NewCIStr(yyS[yypt-0].ident)} + parser.yyVAL.item = []ast.CIStr{ast.NewCIStr(yyS[yypt-0].ident)} } case 525: { - parser.yyVAL.item = append(yyS[yypt-2].item.([]model.CIStr), model.NewCIStr(yyS[yypt-0].ident)) + parser.yyVAL.item = append(yyS[yypt-2].item.([]ast.CIStr), ast.NewCIStr(yyS[yypt-0].ident)) } case 526: { @@ -16379,11 +16378,11 @@ yynewstate: } case 527: { - parser.yyVAL.item = model.CheckOptionCascaded + parser.yyVAL.item = ast.CheckOptionCascaded } case 528: { - parser.yyVAL.item = model.CheckOptionLocal + parser.yyVAL.item = ast.CheckOptionLocal } case 529: { @@ -16396,8 +16395,8 @@ yynewstate: // Single Table tn := yyS[yypt-6].item.(*ast.TableName) tn.IndexHints = yyS[yypt-3].item.([]*ast.IndexHint) - tn.PartitionNames = yyS[yypt-5].item.([]model.CIStr) - join := &ast.Join{Left: &ast.TableSource{Source: tn, AsName: yyS[yypt-4].item.(model.CIStr)}, Right: nil} + tn.PartitionNames = yyS[yypt-5].item.([]ast.CIStr) + join := &ast.Join{Left: &ast.TableSource{Source: tn, AsName: yyS[yypt-4].item.(ast.CIStr)}, Right: nil} x := &ast.DeleteStmt{ TableRefs: &ast.TableRefsClause{TableRefs: join}, Priority: yyS[yypt-10].item.(mysql.PriorityEnum), @@ -16472,7 +16471,7 @@ yynewstate: } case 538: { - parser.yyVAL.statement = &ast.DropDatabaseStmt{IfExists: yyS[yypt-1].item.(bool), Name: model.NewCIStr(yyS[yypt-0].ident)} + parser.yyVAL.statement = &ast.DropDatabaseStmt{IfExists: yyS[yypt-1].item.(bool), Name: ast.NewCIStr(yyS[yypt-0].ident)} } case 539: { @@ -16549,7 +16548,7 @@ yynewstate: parser.lastErrorAsWarn() parser.yyVAL.statement = &ast.DropStatsStmt{ Tables: []*ast.TableName{yyS[yypt-2].item.(*ast.TableName)}, - PartitionNames: yyS[yypt-0].item.([]model.CIStr), + PartitionNames: yyS[yypt-0].item.([]ast.CIStr), } } case 553: @@ -17435,7 +17434,7 @@ yynewstate: } case 736: { - parser.yyVAL.expr = &ast.FuncCallExpr{FnName: model.NewCIStr(ast.JSONMemberOf), Args: []ast.ExprNode{yyS[yypt-4].expr, yyS[yypt-1].expr}} + parser.yyVAL.expr = &ast.FuncCallExpr{FnName: ast.NewCIStr(ast.JSONMemberOf), Args: []ast.ExprNode{yyS[yypt-4].expr, yyS[yypt-1].expr}} } case 740: { @@ -17451,19 +17450,19 @@ yynewstate: } case 743: { - wildCard := &ast.WildCardField{Table: model.NewCIStr(yyS[yypt-2].ident)} + wildCard := &ast.WildCardField{Table: ast.NewCIStr(yyS[yypt-2].ident)} parser.yyVAL.item = &ast.SelectField{WildCard: wildCard} } case 744: { - wildCard := &ast.WildCardField{Schema: model.NewCIStr(yyS[yypt-4].ident), Table: model.NewCIStr(yyS[yypt-2].ident)} + wildCard := &ast.WildCardField{Schema: ast.NewCIStr(yyS[yypt-4].ident), Table: ast.NewCIStr(yyS[yypt-2].ident)} parser.yyVAL.item = &ast.SelectField{WildCard: wildCard} } case 745: { expr := yyS[yypt-1].expr asName := yyS[yypt-0].ident - parser.yyVAL.item = &ast.SelectField{Expr: expr, AsName: model.NewCIStr(asName)} + parser.yyVAL.item = &ast.SelectField{Expr: expr, AsName: ast.NewCIStr(asName)} } case 746: { @@ -17588,7 +17587,7 @@ yynewstate: opt1.ParserName = opt2.ParserName } else if opt2.Visibility != ast.IndexVisibilityDefault { opt1.Visibility = opt2.Visibility - } else if opt2.PrimaryKeyTp != model.PrimaryKeyTypeDefault { + } else if opt2.PrimaryKeyTp != ast.PrimaryKeyTypeDefault { opt1.PrimaryKeyTp = opt2.PrimaryKeyTp } else if opt2.Global { opt1.Global = true @@ -17607,13 +17606,13 @@ yynewstate: case 773: { parser.yyVAL.item = &ast.IndexOption{ - Tp: yyS[yypt-0].item.(model.IndexType), + Tp: yyS[yypt-0].item.(ast.IndexType), } } case 774: { parser.yyVAL.item = &ast.IndexOption{ - ParserName: model.NewCIStr(yyS[yypt-0].ident), + ParserName: ast.NewCIStr(yyS[yypt-0].ident), } yylex.AppendError(yylex.Errorf("The WITH PARASER clause is parsed but ignored by all storage engines.")) parser.lastErrorAsWarn() @@ -17633,7 +17632,7 @@ yynewstate: case 777: { parser.yyVAL.item = &ast.IndexOption{ - PrimaryKeyTp: yyS[yypt-0].item.(model.PrimaryKeyType), + PrimaryKeyTp: yyS[yypt-0].item.(ast.PrimaryKeyType), } } case 778: @@ -17688,23 +17687,23 @@ yynewstate: } case 789: { - parser.yyVAL.item = model.IndexTypeBtree + parser.yyVAL.item = ast.IndexTypeBtree } case 790: { - parser.yyVAL.item = model.IndexTypeHash + parser.yyVAL.item = ast.IndexTypeHash } case 791: { - parser.yyVAL.item = model.IndexTypeRtree + parser.yyVAL.item = ast.IndexTypeRtree } case 792: { - parser.yyVAL.item = model.IndexTypeHypo + parser.yyVAL.item = ast.IndexTypeHypo } case 793: { - parser.yyVAL.item = model.IndexTypeHNSW + parser.yyVAL.item = ast.IndexTypeHNSW } case 794: { @@ -17724,7 +17723,7 @@ yynewstate: { parser.yyVAL.expr = &ast.FuncCallExpr{ Tp: ast.FuncCallExprTypeGeneric, - FnName: model.NewCIStr(yyS[yypt-0].ident), + FnName: ast.NewCIStr(yyS[yypt-0].ident), Args: []ast.ExprNode{}, } } @@ -17732,8 +17731,8 @@ yynewstate: { parser.yyVAL.expr = &ast.FuncCallExpr{ Tp: ast.FuncCallExprTypeGeneric, - Schema: model.NewCIStr(yyS[yypt-2].ident), - FnName: model.NewCIStr(yyS[yypt-0].ident), + Schema: ast.NewCIStr(yyS[yypt-2].ident), + FnName: ast.NewCIStr(yyS[yypt-0].ident), Args: []ast.ExprNode{}, } } @@ -17741,7 +17740,7 @@ yynewstate: { parser.yyVAL.expr = &ast.FuncCallExpr{ Tp: ast.FuncCallExprTypeGeneric, - FnName: model.NewCIStr(yyS[yypt-3].ident), + FnName: ast.NewCIStr(yyS[yypt-3].ident), Args: yyS[yypt-1].item.([]ast.ExprNode), } } @@ -17749,8 +17748,8 @@ yynewstate: { parser.yyVAL.expr = &ast.FuncCallExpr{ Tp: ast.FuncCallExprTypeGeneric, - Schema: model.NewCIStr(yyS[yypt-5].ident), - FnName: model.NewCIStr(yyS[yypt-3].ident), + Schema: ast.NewCIStr(yyS[yypt-5].ident), + FnName: ast.NewCIStr(yyS[yypt-3].ident), Args: yyS[yypt-1].item.([]ast.ExprNode), } } @@ -17768,7 +17767,7 @@ yynewstate: if yyS[yypt-7].item != nil { x.TableHints = yyS[yypt-7].item.([]*ast.TableOptimizerHint) } - x.PartitionNames = yyS[yypt-2].item.([]model.CIStr) + x.PartitionNames = yyS[yypt-2].item.([]ast.CIStr) parser.yyVAL.statement = x } case 1357: @@ -17897,7 +17896,7 @@ yynewstate: x.Priority = yyS[yypt-4].item.(mysql.PriorityEnum) ts := &ast.TableSource{Source: yyS[yypt-2].item.(*ast.TableName)} x.Table = &ast.TableRefsClause{TableRefs: &ast.Join{Left: ts}} - x.PartitionNames = yyS[yypt-1].item.([]model.CIStr) + x.PartitionNames = yyS[yypt-1].item.([]ast.CIStr) parser.yyVAL.statement = x } case 1384: @@ -18101,7 +18100,7 @@ yynewstate: case 1419: { parser.yyVAL.expr = &ast.FuncCallExpr{ - FnName: model.NewCIStr("DATE_ADD"), + FnName: ast.NewCIStr("DATE_ADD"), Args: []ast.ExprNode{ yyS[yypt-4].expr, yyS[yypt-1].expr, @@ -18112,7 +18111,7 @@ yynewstate: case 1420: { parser.yyVAL.expr = &ast.FuncCallExpr{ - FnName: model.NewCIStr("DATE_SUB"), + FnName: ast.NewCIStr("DATE_SUB"), Args: []ast.ExprNode{ yyS[yypt-4].expr, yyS[yypt-1].expr, @@ -18123,7 +18122,7 @@ yynewstate: case 1421: { parser.yyVAL.expr = &ast.FuncCallExpr{ - FnName: model.NewCIStr("DATE_ADD"), + FnName: ast.NewCIStr("DATE_ADD"), Args: []ast.ExprNode{ yyS[yypt-0].expr, yyS[yypt-3].expr, @@ -18158,22 +18157,22 @@ yynewstate: case 1429: { parser.yyVAL.expr = &ast.ColumnNameExpr{Name: &ast.ColumnName{ - Name: model.NewCIStr(yyS[yypt-0].ident), + Name: ast.NewCIStr(yyS[yypt-0].ident), }} } case 1430: { parser.yyVAL.expr = &ast.ColumnNameExpr{Name: &ast.ColumnName{ - Table: model.NewCIStr(yyS[yypt-2].ident), - Name: model.NewCIStr(yyS[yypt-0].ident), + Table: ast.NewCIStr(yyS[yypt-2].ident), + Name: ast.NewCIStr(yyS[yypt-0].ident), }} } case 1431: { parser.yyVAL.expr = &ast.ColumnNameExpr{Name: &ast.ColumnName{ - Schema: model.NewCIStr(yyS[yypt-4].ident), - Table: model.NewCIStr(yyS[yypt-2].ident), - Name: model.NewCIStr(yyS[yypt-0].ident), + Schema: ast.NewCIStr(yyS[yypt-4].ident), + Table: ast.NewCIStr(yyS[yypt-2].ident), + Name: ast.NewCIStr(yyS[yypt-0].ident), }} } case 1436: @@ -18202,7 +18201,7 @@ yynewstate: } case 1446: { - parser.yyVAL.expr = &ast.FuncCallExpr{FnName: model.NewCIStr(ast.Concat), Args: []ast.ExprNode{yyS[yypt-2].expr, yyS[yypt-0].expr}} + parser.yyVAL.expr = &ast.FuncCallExpr{FnName: ast.NewCIStr(ast.Concat), Args: []ast.ExprNode{yyS[yypt-2].expr, yyS[yypt-0].expr}} } case 1447: { @@ -18243,15 +18242,15 @@ yynewstate: case "d": tp.SetCharset("") tp.SetCollate("") - parser.yyVAL.expr = &ast.FuncCallExpr{FnName: model.NewCIStr(ast.DateLiteral), Args: []ast.ExprNode{yyS[yypt-1].expr}} + parser.yyVAL.expr = &ast.FuncCallExpr{FnName: ast.NewCIStr(ast.DateLiteral), Args: []ast.ExprNode{yyS[yypt-1].expr}} case "t": tp.SetCharset("") tp.SetCollate("") - parser.yyVAL.expr = &ast.FuncCallExpr{FnName: model.NewCIStr(ast.TimeLiteral), Args: []ast.ExprNode{yyS[yypt-1].expr}} + parser.yyVAL.expr = &ast.FuncCallExpr{FnName: ast.NewCIStr(ast.TimeLiteral), Args: []ast.ExprNode{yyS[yypt-1].expr}} case "ts": tp.SetCharset("") tp.SetCollate("") - parser.yyVAL.expr = &ast.FuncCallExpr{FnName: model.NewCIStr(ast.TimestampLiteral), Args: []ast.ExprNode{yyS[yypt-1].expr}} + parser.yyVAL.expr = &ast.FuncCallExpr{FnName: ast.NewCIStr(ast.TimestampLiteral), Args: []ast.ExprNode{yyS[yypt-1].expr}} default: parser.yyVAL.expr = yyS[yypt-1].expr } @@ -18331,7 +18330,7 @@ yynewstate: // See https://dev.mysql.com/doc/refman/5.7/en/cast-functions.html#function_convert charset1 := ast.NewValueExpr(yyS[yypt-1].ident, "", "") parser.yyVAL.expr = &ast.FuncCallExpr{ - FnName: model.NewCIStr(yyS[yypt-5].ident), + FnName: ast.NewCIStr(yyS[yypt-5].ident), Args: []ast.ExprNode{yyS[yypt-3].expr, charset1}, } } @@ -18346,13 +18345,13 @@ yynewstate: case 1461: { expr := ast.NewValueExpr(yyS[yypt-0].ident, parser.charset, parser.collation) - parser.yyVAL.expr = &ast.FuncCallExpr{FnName: model.NewCIStr(ast.JSONExtract), Args: []ast.ExprNode{yyS[yypt-2].expr, expr}} + parser.yyVAL.expr = &ast.FuncCallExpr{FnName: ast.NewCIStr(ast.JSONExtract), Args: []ast.ExprNode{yyS[yypt-2].expr, expr}} } case 1462: { expr := ast.NewValueExpr(yyS[yypt-0].ident, parser.charset, parser.collation) - extract := &ast.FuncCallExpr{FnName: model.NewCIStr(ast.JSONExtract), Args: []ast.ExprNode{yyS[yypt-2].expr, expr}} - parser.yyVAL.expr = &ast.FuncCallExpr{FnName: model.NewCIStr(ast.JSONUnquote), Args: []ast.ExprNode{extract}} + extract := &ast.FuncCallExpr{FnName: ast.NewCIStr(ast.JSONExtract), Args: []ast.ExprNode{yyS[yypt-2].expr, expr}} + parser.yyVAL.expr = &ast.FuncCallExpr{FnName: ast.NewCIStr(ast.JSONUnquote), Args: []ast.ExprNode{extract}} } case 1463: { @@ -18384,19 +18383,19 @@ yynewstate: } case 1519: { - parser.yyVAL.expr = &ast.FuncCallExpr{FnName: model.NewCIStr(yyS[yypt-3].ident), Args: yyS[yypt-1].item.([]ast.ExprNode)} + parser.yyVAL.expr = &ast.FuncCallExpr{FnName: ast.NewCIStr(yyS[yypt-3].ident), Args: yyS[yypt-1].item.([]ast.ExprNode)} } case 1520: { - parser.yyVAL.expr = &ast.FuncCallExpr{FnName: model.NewCIStr(yyS[yypt-3].ident), Args: yyS[yypt-1].item.([]ast.ExprNode)} + parser.yyVAL.expr = &ast.FuncCallExpr{FnName: ast.NewCIStr(yyS[yypt-3].ident), Args: yyS[yypt-1].item.([]ast.ExprNode)} } case 1521: { - parser.yyVAL.expr = &ast.FuncCallExpr{FnName: model.NewCIStr(yyS[yypt-1].ident)} + parser.yyVAL.expr = &ast.FuncCallExpr{FnName: ast.NewCIStr(yyS[yypt-1].ident)} } case 1522: { - parser.yyVAL.expr = &ast.FuncCallExpr{FnName: model.NewCIStr(yyS[yypt-2].ident)} + parser.yyVAL.expr = &ast.FuncCallExpr{FnName: ast.NewCIStr(yyS[yypt-2].ident)} } case 1523: { @@ -18404,14 +18403,14 @@ yynewstate: if yyS[yypt-0].item != nil { args = append(args, yyS[yypt-0].item.(ast.ExprNode)) } - parser.yyVAL.expr = &ast.FuncCallExpr{FnName: model.NewCIStr(yyS[yypt-1].ident), Args: args} + parser.yyVAL.expr = &ast.FuncCallExpr{FnName: ast.NewCIStr(yyS[yypt-1].ident), Args: args} } case 1524: { nilVal := ast.NewValueExpr(nil, parser.charset, parser.collation) args := yyS[yypt-1].item.([]ast.ExprNode) parser.yyVAL.expr = &ast.FuncCallExpr{ - FnName: model.NewCIStr(ast.CharFunc), + FnName: ast.NewCIStr(ast.CharFunc), Args: append(args, nilVal), } } @@ -18420,28 +18419,28 @@ yynewstate: charset1 := ast.NewValueExpr(yyS[yypt-1].ident, "", "") args := yyS[yypt-3].item.([]ast.ExprNode) parser.yyVAL.expr = &ast.FuncCallExpr{ - FnName: model.NewCIStr(ast.CharFunc), + FnName: ast.NewCIStr(ast.CharFunc), Args: append(args, charset1), } } case 1526: { expr := ast.NewValueExpr(yyS[yypt-0].ident, "", "") - parser.yyVAL.expr = &ast.FuncCallExpr{FnName: model.NewCIStr(ast.DateLiteral), Args: []ast.ExprNode{expr}} + parser.yyVAL.expr = &ast.FuncCallExpr{FnName: ast.NewCIStr(ast.DateLiteral), Args: []ast.ExprNode{expr}} } case 1527: { expr := ast.NewValueExpr(yyS[yypt-0].ident, "", "") - parser.yyVAL.expr = &ast.FuncCallExpr{FnName: model.NewCIStr(ast.TimeLiteral), Args: []ast.ExprNode{expr}} + parser.yyVAL.expr = &ast.FuncCallExpr{FnName: ast.NewCIStr(ast.TimeLiteral), Args: []ast.ExprNode{expr}} } case 1528: { expr := ast.NewValueExpr(yyS[yypt-0].ident, "", "") - parser.yyVAL.expr = &ast.FuncCallExpr{FnName: model.NewCIStr(ast.TimestampLiteral), Args: []ast.ExprNode{expr}} + parser.yyVAL.expr = &ast.FuncCallExpr{FnName: ast.NewCIStr(ast.TimestampLiteral), Args: []ast.ExprNode{expr}} } case 1529: { - parser.yyVAL.expr = &ast.FuncCallExpr{FnName: model.NewCIStr(ast.InsertFunc), Args: yyS[yypt-1].item.([]ast.ExprNode)} + parser.yyVAL.expr = &ast.FuncCallExpr{FnName: ast.NewCIStr(ast.InsertFunc), Args: yyS[yypt-1].item.([]ast.ExprNode)} } case 1530: { @@ -18449,20 +18448,20 @@ yynewstate: } case 1531: { - parser.yyVAL.expr = &ast.FuncCallExpr{FnName: model.NewCIStr(ast.PasswordFunc), Args: yyS[yypt-1].item.([]ast.ExprNode)} + parser.yyVAL.expr = &ast.FuncCallExpr{FnName: ast.NewCIStr(ast.PasswordFunc), Args: yyS[yypt-1].item.([]ast.ExprNode)} } case 1532: { - parser.yyVAL.expr = &ast.FuncCallExpr{FnName: model.NewCIStr(yyS[yypt-3].ident), Args: yyS[yypt-1].item.([]ast.ExprNode)} + parser.yyVAL.expr = &ast.FuncCallExpr{FnName: ast.NewCIStr(yyS[yypt-3].ident), Args: yyS[yypt-1].item.([]ast.ExprNode)} } case 1533: { - parser.yyVAL.expr = &ast.FuncCallExpr{FnName: model.NewCIStr(yyS[yypt-3].ident), Args: yyS[yypt-1].item.([]ast.ExprNode)} + parser.yyVAL.expr = &ast.FuncCallExpr{FnName: ast.NewCIStr(yyS[yypt-3].ident), Args: yyS[yypt-1].item.([]ast.ExprNode)} } case 1534: { parser.yyVAL.expr = &ast.FuncCallExpr{ - FnName: model.NewCIStr(yyS[yypt-5].ident), + FnName: ast.NewCIStr(yyS[yypt-5].ident), Args: []ast.ExprNode{ yyS[yypt-3].expr, yyS[yypt-1].expr, @@ -18473,7 +18472,7 @@ yynewstate: case 1535: { parser.yyVAL.expr = &ast.FuncCallExpr{ - FnName: model.NewCIStr(yyS[yypt-7].ident), + FnName: ast.NewCIStr(yyS[yypt-7].ident), Args: []ast.ExprNode{ yyS[yypt-5].expr, yyS[yypt-2].expr, @@ -18484,7 +18483,7 @@ yynewstate: case 1536: { parser.yyVAL.expr = &ast.FuncCallExpr{ - FnName: model.NewCIStr(yyS[yypt-7].ident), + FnName: ast.NewCIStr(yyS[yypt-7].ident), Args: []ast.ExprNode{ yyS[yypt-5].expr, yyS[yypt-2].expr, @@ -18496,14 +18495,14 @@ yynewstate: { timeUnit := &ast.TimeUnitExpr{Unit: yyS[yypt-3].item.(ast.TimeUnitType)} parser.yyVAL.expr = &ast.FuncCallExpr{ - FnName: model.NewCIStr(yyS[yypt-5].ident), + FnName: ast.NewCIStr(yyS[yypt-5].ident), Args: []ast.ExprNode{timeUnit, yyS[yypt-1].expr}, } } case 1538: { parser.yyVAL.expr = &ast.FuncCallExpr{ - FnName: model.NewCIStr(yyS[yypt-5].ident), + FnName: ast.NewCIStr(yyS[yypt-5].ident), Args: []ast.ExprNode{ &ast.GetFormatSelectorExpr{Selector: yyS[yypt-3].item.(ast.GetFormatSelectorType)}, yyS[yypt-1].expr, @@ -18512,61 +18511,61 @@ yynewstate: } case 1539: { - parser.yyVAL.expr = &ast.FuncCallExpr{FnName: model.NewCIStr(yyS[yypt-5].ident), Args: []ast.ExprNode{yyS[yypt-3].expr, yyS[yypt-1].expr}} + parser.yyVAL.expr = &ast.FuncCallExpr{FnName: ast.NewCIStr(yyS[yypt-5].ident), Args: []ast.ExprNode{yyS[yypt-3].expr, yyS[yypt-1].expr}} } case 1540: { parser.yyVAL.expr = &ast.FuncCallExpr{ - FnName: model.NewCIStr(yyS[yypt-5].ident), + FnName: ast.NewCIStr(yyS[yypt-5].ident), Args: []ast.ExprNode{yyS[yypt-3].expr, yyS[yypt-1].expr}, } } case 1541: { parser.yyVAL.expr = &ast.FuncCallExpr{ - FnName: model.NewCIStr(yyS[yypt-5].ident), + FnName: ast.NewCIStr(yyS[yypt-5].ident), Args: []ast.ExprNode{yyS[yypt-3].expr, yyS[yypt-1].expr}, } } case 1542: { parser.yyVAL.expr = &ast.FuncCallExpr{ - FnName: model.NewCIStr(yyS[yypt-7].ident), + FnName: ast.NewCIStr(yyS[yypt-7].ident), Args: []ast.ExprNode{yyS[yypt-5].expr, yyS[yypt-3].expr, yyS[yypt-1].expr}, } } case 1543: { parser.yyVAL.expr = &ast.FuncCallExpr{ - FnName: model.NewCIStr(yyS[yypt-7].ident), + FnName: ast.NewCIStr(yyS[yypt-7].ident), Args: []ast.ExprNode{yyS[yypt-5].expr, yyS[yypt-3].expr, yyS[yypt-1].expr}, } } case 1544: { parser.yyVAL.expr = &ast.FuncCallExpr{ - FnName: model.NewCIStr(yyS[yypt-7].ident), + FnName: ast.NewCIStr(yyS[yypt-7].ident), Args: []ast.ExprNode{&ast.TimeUnitExpr{Unit: yyS[yypt-5].item.(ast.TimeUnitType)}, yyS[yypt-3].expr, yyS[yypt-1].expr}, } } case 1545: { parser.yyVAL.expr = &ast.FuncCallExpr{ - FnName: model.NewCIStr(yyS[yypt-7].ident), + FnName: ast.NewCIStr(yyS[yypt-7].ident), Args: []ast.ExprNode{&ast.TimeUnitExpr{Unit: yyS[yypt-5].item.(ast.TimeUnitType)}, yyS[yypt-3].expr, yyS[yypt-1].expr}, } } case 1546: { parser.yyVAL.expr = &ast.FuncCallExpr{ - FnName: model.NewCIStr(yyS[yypt-3].ident), + FnName: ast.NewCIStr(yyS[yypt-3].ident), Args: []ast.ExprNode{yyS[yypt-1].expr}, } } case 1547: { parser.yyVAL.expr = &ast.FuncCallExpr{ - FnName: model.NewCIStr(yyS[yypt-5].ident), + FnName: ast.NewCIStr(yyS[yypt-5].ident), Args: []ast.ExprNode{yyS[yypt-1].expr, yyS[yypt-3].expr}, } } @@ -18575,7 +18574,7 @@ yynewstate: spaceVal := ast.NewValueExpr(" ", parser.charset, parser.collation) direction := &ast.TrimDirectionExpr{Direction: yyS[yypt-3].item.(ast.TrimDirectionType)} parser.yyVAL.expr = &ast.FuncCallExpr{ - FnName: model.NewCIStr(yyS[yypt-5].ident), + FnName: ast.NewCIStr(yyS[yypt-5].ident), Args: []ast.ExprNode{yyS[yypt-1].expr, spaceVal, direction}, } } @@ -18583,41 +18582,41 @@ yynewstate: { direction := &ast.TrimDirectionExpr{Direction: yyS[yypt-4].item.(ast.TrimDirectionType)} parser.yyVAL.expr = &ast.FuncCallExpr{ - FnName: model.NewCIStr(yyS[yypt-6].ident), + FnName: ast.NewCIStr(yyS[yypt-6].ident), Args: []ast.ExprNode{yyS[yypt-1].expr, yyS[yypt-3].expr, direction}, } } case 1550: { parser.yyVAL.expr = &ast.FuncCallExpr{ - FnName: model.NewCIStr(yyS[yypt-3].ident), + FnName: ast.NewCIStr(yyS[yypt-3].ident), Args: []ast.ExprNode{yyS[yypt-1].expr}, } } case 1551: { parser.yyVAL.expr = &ast.FuncCallExpr{ - FnName: model.NewCIStr(yyS[yypt-6].ident), + FnName: ast.NewCIStr(yyS[yypt-6].ident), Args: []ast.ExprNode{yyS[yypt-4].expr, ast.NewValueExpr("CHAR", parser.charset, parser.collation), ast.NewValueExpr(yyS[yypt-1].item, parser.charset, parser.collation)}, } } case 1552: { parser.yyVAL.expr = &ast.FuncCallExpr{ - FnName: model.NewCIStr(yyS[yypt-6].ident), + FnName: ast.NewCIStr(yyS[yypt-6].ident), Args: []ast.ExprNode{yyS[yypt-4].expr, ast.NewValueExpr("BINARY", parser.charset, parser.collation), ast.NewValueExpr(yyS[yypt-1].item, parser.charset, parser.collation)}, } } case 1554: { parser.yyVAL.expr = &ast.FuncCallExpr{ - FnName: model.NewCIStr(yyS[yypt-7].ident), + FnName: ast.NewCIStr(yyS[yypt-7].ident), Args: []ast.ExprNode{yyS[yypt-5].expr, yyS[yypt-3].expr, yyS[yypt-1].expr}, } } case 1555: { - parser.yyVAL.expr = &ast.FuncCallExpr{FnName: model.NewCIStr(yyS[yypt-3].ident), Args: yyS[yypt-1].item.([]ast.ExprNode)} + parser.yyVAL.expr = &ast.FuncCallExpr{FnName: ast.NewCIStr(yyS[yypt-3].ident), Args: yyS[yypt-1].item.([]ast.ExprNode)} } case 1556: { @@ -18653,7 +18652,7 @@ yynewstate: Name: yyS[yypt-1].item.(*ast.TableName), } parser.yyVAL.expr = &ast.FuncCallExpr{ - FnName: model.NewCIStr(ast.LastVal), + FnName: ast.NewCIStr(ast.LastVal), Args: []ast.ExprNode{objNameExpr}, } } @@ -18664,7 +18663,7 @@ yynewstate: } valueExpr := ast.NewValueExpr(yyS[yypt-1].item, parser.charset, parser.collation) parser.yyVAL.expr = &ast.FuncCallExpr{ - FnName: model.NewCIStr(ast.SetVal), + FnName: ast.NewCIStr(ast.SetVal), Args: []ast.ExprNode{objNameExpr, valueExpr}, } } @@ -18890,7 +18889,7 @@ yynewstate: case 1599: { parser.yyVAL.expr = &ast.FuncCallExpr{ - FnName: model.NewCIStr(yyS[yypt-3].ident), + FnName: ast.NewCIStr(yyS[yypt-3].ident), Args: yyS[yypt-1].item.([]ast.ExprNode), } } @@ -18904,8 +18903,8 @@ yynewstate: } parser.yyVAL.expr = &ast.FuncCallExpr{ Tp: tp, - Schema: model.NewCIStr(yyS[yypt-5].ident), - FnName: model.NewCIStr(yyS[yypt-3].ident), + Schema: ast.NewCIStr(yyS[yypt-5].ident), + FnName: ast.NewCIStr(yyS[yypt-3].ident), Args: yyS[yypt-1].item.([]ast.ExprNode), } } @@ -19250,7 +19249,7 @@ yynewstate: } case 1659: { - parser.yyVAL.item = &ast.TableName{Name: model.NewCIStr(yyS[yypt-0].ident)} + parser.yyVAL.item = &ast.TableName{Name: ast.NewCIStr(yyS[yypt-0].ident)} } case 1660: { @@ -19259,11 +19258,11 @@ yynewstate: yylex.AppendError(ErrWrongDBName.GenWithStackByArgs(schema)) return 1 } - parser.yyVAL.item = &ast.TableName{Schema: model.NewCIStr(schema), Name: model.NewCIStr(yyS[yypt-0].ident)} + parser.yyVAL.item = &ast.TableName{Schema: ast.NewCIStr(schema), Name: ast.NewCIStr(yyS[yypt-0].ident)} } case 1661: { - parser.yyVAL.item = &ast.TableName{Schema: model.NewCIStr("*"), Name: model.NewCIStr(yyS[yypt-0].ident)} + parser.yyVAL.item = &ast.TableName{Schema: ast.NewCIStr("*"), Name: ast.NewCIStr(yyS[yypt-0].ident)} } case 1662: { @@ -19276,11 +19275,11 @@ yynewstate: } case 1664: { - parser.yyVAL.item = &ast.TableName{Name: model.NewCIStr(yyS[yypt-1].ident)} + parser.yyVAL.item = &ast.TableName{Name: ast.NewCIStr(yyS[yypt-1].ident)} } case 1665: { - parser.yyVAL.item = &ast.TableName{Schema: model.NewCIStr(yyS[yypt-3].ident), Name: model.NewCIStr(yyS[yypt-1].ident)} + parser.yyVAL.item = &ast.TableName{Schema: ast.NewCIStr(yyS[yypt-3].ident), Name: ast.NewCIStr(yyS[yypt-1].ident)} } case 1666: { @@ -19671,8 +19670,8 @@ yynewstate: case 1722: { cte := &ast.CommonTableExpression{} - cte.Name = model.NewCIStr(yyS[yypt-3].ident) - cte.ColNameList = yyS[yypt-2].item.([]model.CIStr) + cte.Name = ast.NewCIStr(yyS[yypt-3].ident) + cte.ColNameList = yyS[yypt-2].item.([]ast.CIStr) cte.Query = yyS[yypt-0].expr.(*ast.SubqueryExpr) parser.yyVAL.item = cte } @@ -19695,12 +19694,12 @@ yynewstate: case 1728: { var spec = yyS[yypt-0].item.(ast.WindowSpec) - spec.Name = yyS[yypt-2].item.(model.CIStr) + spec.Name = yyS[yypt-2].item.(ast.CIStr) parser.yyVAL.item = spec } case 1729: { - parser.yyVAL.item = model.NewCIStr(yyS[yypt-0].ident) + parser.yyVAL.item = ast.NewCIStr(yyS[yypt-0].ident) } case 1730: { @@ -19708,7 +19707,7 @@ yynewstate: } case 1731: { - spec := ast.WindowSpec{Ref: yyS[yypt-3].item.(model.CIStr)} + spec := ast.WindowSpec{Ref: yyS[yypt-3].item.(ast.CIStr)} if yyS[yypt-2].item != nil { spec.PartitionBy = yyS[yypt-2].item.(*ast.PartitionByClause) } @@ -19722,7 +19721,7 @@ yynewstate: } case 1732: { - parser.yyVAL.item = model.CIStr{} + parser.yyVAL.item = ast.CIStr{} } case 1734: { @@ -19825,7 +19824,7 @@ yynewstate: } case 1759: { - parser.yyVAL.item = ast.WindowSpec{Name: yyS[yypt-0].item.(model.CIStr), OnlyAlias: true} + parser.yyVAL.item = ast.WindowSpec{Name: yyS[yypt-0].item.(ast.CIStr), OnlyAlias: true} } case 1761: { @@ -19960,7 +19959,7 @@ yynewstate: case 1790: { tn := yyS[yypt-5].item.(*ast.TableName) - tn.PartitionNames = yyS[yypt-4].item.([]model.CIStr) + tn.PartitionNames = yyS[yypt-4].item.([]ast.CIStr) tn.IndexHints = yyS[yypt-1].item.([]*ast.IndexHint) if yyS[yypt-0].item != nil { tn.TableSample = yyS[yypt-0].item.(*ast.TableSample) @@ -19968,12 +19967,12 @@ yynewstate: if yyS[yypt-2].item != nil { tn.AsOf = yyS[yypt-2].item.(*ast.AsOfClause) } - parser.yyVAL.item = &ast.TableSource{Source: tn, AsName: yyS[yypt-3].item.(model.CIStr)} + parser.yyVAL.item = &ast.TableSource{Source: tn, AsName: yyS[yypt-3].item.(ast.CIStr)} } case 1791: { resultNode := yyS[yypt-1].expr.(*ast.SubqueryExpr).Query - parser.yyVAL.item = &ast.TableSource{Source: resultNode, AsName: yyS[yypt-0].item.(model.CIStr)} + parser.yyVAL.item = &ast.TableSource{Source: resultNode, AsName: yyS[yypt-0].item.(ast.CIStr)} } case 1792: { @@ -19983,7 +19982,7 @@ yynewstate: } case 1793: { - parser.yyVAL.item = []model.CIStr{} + parser.yyVAL.item = []ast.CIStr{} } case 1794: { @@ -19991,15 +19990,15 @@ yynewstate: } case 1795: { - parser.yyVAL.item = model.CIStr{} + parser.yyVAL.item = ast.CIStr{} } case 1797: { - parser.yyVAL.item = model.NewCIStr(yyS[yypt-0].ident) + parser.yyVAL.item = ast.NewCIStr(yyS[yypt-0].ident) } case 1798: { - parser.yyVAL.item = model.NewCIStr(yyS[yypt-0].ident) + parser.yyVAL.item = ast.NewCIStr(yyS[yypt-0].ident) } case 1799: { @@ -20032,31 +20031,31 @@ yynewstate: case 1806: { parser.yyVAL.item = &ast.IndexHint{ - IndexNames: yyS[yypt-1].item.([]model.CIStr), + IndexNames: yyS[yypt-1].item.([]ast.CIStr), HintType: yyS[yypt-4].item.(ast.IndexHintType), HintScope: yyS[yypt-3].item.(ast.IndexHintScope), } } case 1807: { - var nameList []model.CIStr + var nameList []ast.CIStr parser.yyVAL.item = nameList } case 1808: { - parser.yyVAL.item = []model.CIStr{model.NewCIStr(yyS[yypt-0].ident)} + parser.yyVAL.item = []ast.CIStr{ast.NewCIStr(yyS[yypt-0].ident)} } case 1809: { - parser.yyVAL.item = append(yyS[yypt-2].item.([]model.CIStr), model.NewCIStr(yyS[yypt-0].ident)) + parser.yyVAL.item = append(yyS[yypt-2].item.([]ast.CIStr), ast.NewCIStr(yyS[yypt-0].ident)) } case 1810: { - parser.yyVAL.item = []model.CIStr{model.NewCIStr(yyS[yypt-0].ident)} + parser.yyVAL.item = []ast.CIStr{ast.NewCIStr(yyS[yypt-0].ident)} } case 1811: { - parser.yyVAL.item = append(yyS[yypt-2].item.([]model.CIStr), model.NewCIStr(yyS[yypt-0].ident)) + parser.yyVAL.item = append(yyS[yypt-2].item.([]ast.CIStr), ast.NewCIStr(yyS[yypt-0].ident)) } case 1812: { @@ -20742,7 +20741,7 @@ yynewstate: } case 1916: { - parser.yyVAL.statement = &ast.SetResourceGroupStmt{Name: model.NewCIStr(yyS[yypt-0].ident)} + parser.yyVAL.statement = &ast.SetResourceGroupStmt{Name: ast.NewCIStr(yyS[yypt-0].ident)} } case 1917: { @@ -21446,7 +21445,7 @@ yynewstate: Tp: ast.ShowRegions, Table: yyS[yypt-3].item.(*ast.TableName), } - stmt.Table.PartitionNames = yyS[yypt-2].item.([]model.CIStr) + stmt.Table.PartitionNames = yyS[yypt-2].item.([]ast.CIStr) if yyS[yypt-0].item != nil { stmt.Where = yyS[yypt-0].item.(ast.ExprNode) } @@ -21464,9 +21463,9 @@ yynewstate: stmt := &ast.ShowStmt{ Tp: ast.ShowRegions, Table: yyS[yypt-5].item.(*ast.TableName), - IndexName: model.NewCIStr(yyS[yypt-2].ident), + IndexName: ast.NewCIStr(yyS[yypt-2].ident), } - stmt.Table.PartitionNames = yyS[yypt-4].item.([]model.CIStr) + stmt.Table.PartitionNames = yyS[yypt-4].item.([]ast.CIStr) if yyS[yypt-0].item != nil { stmt.Where = yyS[yypt-0].item.(ast.ExprNode) } @@ -21591,7 +21590,7 @@ yynewstate: parser.yyVAL.item = &ast.ShowStmt{ Tp: ast.ShowPlacementForPartition, Table: yyS[yypt-2].item.(*ast.TableName), - Partition: model.NewCIStr(yyS[yypt-0].ident), + Partition: ast.NewCIStr(yyS[yypt-0].ident), } } case 2065: @@ -21710,7 +21709,7 @@ yynewstate: { show := &ast.ShowStmt{ Tp: ast.ShowIndex, - Table: &ast.TableName{Name: model.NewCIStr(yyS[yypt-2].ident), Schema: model.NewCIStr(yyS[yypt-0].ident)}, + Table: &ast.TableName{Name: ast.NewCIStr(yyS[yypt-2].ident), Schema: ast.NewCIStr(yyS[yypt-0].ident)}, } parser.yyVAL.item = show } @@ -21828,11 +21827,11 @@ yynewstate: } case 2114: { - parser.yyVAL.item = &ast.ShowStmt{Tp: ast.ShowStatsMeta, Table: &ast.TableName{Name: model.NewCIStr("STATS_META"), Schema: model.NewCIStr(mysql.SystemDB)}} + parser.yyVAL.item = &ast.ShowStmt{Tp: ast.ShowStatsMeta, Table: &ast.TableName{Name: ast.NewCIStr("STATS_META"), Schema: ast.NewCIStr(mysql.SystemDB)}} } case 2115: { - parser.yyVAL.item = &ast.ShowStmt{Tp: ast.ShowStatsHistograms, Table: &ast.TableName{Name: model.NewCIStr("STATS_HISTOGRAMS"), Schema: model.NewCIStr(mysql.SystemDB)}} + parser.yyVAL.item = &ast.ShowStmt{Tp: ast.ShowStatsHistograms, Table: &ast.TableName{Name: ast.NewCIStr("STATS_HISTOGRAMS"), Schema: ast.NewCIStr(mysql.SystemDB)}} } case 2116: { @@ -21840,7 +21839,7 @@ yynewstate: } case 2117: { - parser.yyVAL.item = &ast.ShowStmt{Tp: ast.ShowStatsBuckets, Table: &ast.TableName{Name: model.NewCIStr("STATS_BUCKETS"), Schema: model.NewCIStr(mysql.SystemDB)}} + parser.yyVAL.item = &ast.ShowStmt{Tp: ast.ShowStatsBuckets, Table: &ast.TableName{Name: ast.NewCIStr("STATS_BUCKETS"), Schema: ast.NewCIStr(mysql.SystemDB)}} } case 2118: { @@ -21848,7 +21847,7 @@ yynewstate: } case 2119: { - parser.yyVAL.item = &ast.ShowStmt{Tp: ast.ShowStatsLocked, Table: &ast.TableName{Name: model.NewCIStr("STATS_TABLE_LOCKED"), Schema: model.NewCIStr(mysql.SystemDB)}} + parser.yyVAL.item = &ast.ShowStmt{Tp: ast.ShowStatsLocked, Table: &ast.TableName{Name: ast.NewCIStr("STATS_TABLE_LOCKED"), Schema: ast.NewCIStr(mysql.SystemDB)}} } case 2120: { @@ -22139,14 +22138,14 @@ yynewstate: if c.Option == nil { c.Option = &ast.IndexOption{} } - c.Option.Tp = indexType.(model.IndexType) + c.Option.Tp = indexType.(ast.IndexType) } if c.Option == nil { - c.Option = &ast.IndexOption{Tp: model.IndexTypeHNSW} - } else if c.Option.Tp == model.IndexTypeInvalid { - c.Option.Tp = model.IndexTypeHNSW + c.Option = &ast.IndexOption{Tp: ast.IndexTypeHNSW} + } else if c.Option.Tp == ast.IndexTypeInvalid { + c.Option.Tp = ast.IndexTypeHNSW } - if c.Option.Tp != model.IndexTypeHNSW { + if c.Option.Tp != ast.IndexTypeHNSW { yylex.AppendError(ErrSyntax) return 1 } @@ -22391,7 +22390,7 @@ yynewstate: { parser.yyVAL.item = &ast.TableOption{ Tp: ast.TableOptionTTL, - ColumnName: &ast.ColumnName{Name: model.NewCIStr(yyS[yypt-4].ident)}, + ColumnName: &ast.ColumnName{Name: ast.NewCIStr(yyS[yypt-4].ident)}, Value: ast.NewValueExpr(yyS[yypt-1].expr, parser.charset, parser.collation), TimeUnitValue: &ast.TimeUnitExpr{Unit: yyS[yypt-0].item.(ast.TimeUnitType)}, } @@ -23247,7 +23246,7 @@ yynewstate: case 2510: { option := yyS[yypt-0].item.(*ast.PlacementOption) - parser.yyVAL.statement = &ast.AlterRangeStmt{RangeName: model.NewCIStr(yyS[yypt-1].ident), PlacementOption: option} + parser.yyVAL.statement = &ast.AlterRangeStmt{RangeName: ast.NewCIStr(yyS[yypt-1].ident), PlacementOption: option} } case 2511: { @@ -24510,24 +24509,24 @@ yynewstate: { parser.yyVAL.item = ast.TableLock{ Table: yyS[yypt-1].item.(*ast.TableName), - Type: yyS[yypt-0].item.(model.TableLockType), + Type: yyS[yypt-0].item.(ast.TableLockType), } } case 2725: { - parser.yyVAL.item = model.TableLockRead + parser.yyVAL.item = ast.TableLockRead } case 2726: { - parser.yyVAL.item = model.TableLockReadLocal + parser.yyVAL.item = ast.TableLockReadLocal } case 2727: { - parser.yyVAL.item = model.TableLockWrite + parser.yyVAL.item = ast.TableLockWrite } case 2728: { - parser.yyVAL.item = model.TableLockWriteLocal + parser.yyVAL.item = ast.TableLockWriteLocal } case 2729: { @@ -24625,7 +24624,7 @@ yynewstate: case 2750: { x := yyS[yypt-2].item.(*ast.TableName) - x.PartitionNames = yyS[yypt-0].item.([]model.CIStr) + x.PartitionNames = yyS[yypt-0].item.([]ast.CIStr) parser.yyVAL.statement = &ast.LockStatsStmt{ Tables: []*ast.TableName{x}, } @@ -24633,7 +24632,7 @@ yynewstate: case 2751: { x := yyS[yypt-4].item.(*ast.TableName) - x.PartitionNames = yyS[yypt-1].item.([]model.CIStr) + x.PartitionNames = yyS[yypt-1].item.([]ast.CIStr) parser.yyVAL.statement = &ast.LockStatsStmt{ Tables: []*ast.TableName{x}, } @@ -24647,7 +24646,7 @@ yynewstate: case 2753: { x := yyS[yypt-2].item.(*ast.TableName) - x.PartitionNames = yyS[yypt-0].item.([]model.CIStr) + x.PartitionNames = yyS[yypt-0].item.([]ast.CIStr) parser.yyVAL.statement = &ast.UnlockStatsStmt{ Tables: []*ast.TableName{x}, } @@ -24655,7 +24654,7 @@ yynewstate: case 2754: { x := yyS[yypt-4].item.(*ast.TableName) - x.PartitionNames = yyS[yypt-1].item.([]model.CIStr) + x.PartitionNames = yyS[yypt-1].item.([]ast.CIStr) parser.yyVAL.statement = &ast.UnlockStatsStmt{ Tables: []*ast.TableName{x}, } @@ -24664,14 +24663,14 @@ yynewstate: { parser.yyVAL.statement = &ast.DropPlacementPolicyStmt{ IfExists: yyS[yypt-1].item.(bool), - PolicyName: model.NewCIStr(yyS[yypt-0].ident), + PolicyName: ast.NewCIStr(yyS[yypt-0].ident), } } case 2756: { parser.yyVAL.statement = &ast.CreateResourceGroupStmt{ IfNotExists: yyS[yypt-2].item.(bool), - ResourceGroupName: model.NewCIStr(yyS[yypt-1].ident), + ResourceGroupName: ast.NewCIStr(yyS[yypt-1].ident), ResourceGroupOptionList: yyS[yypt-0].item.([]*ast.ResourceGroupOption), } } @@ -24679,7 +24678,7 @@ yynewstate: { parser.yyVAL.statement = &ast.AlterResourceGroupStmt{ IfExists: yyS[yypt-2].item.(bool), - ResourceGroupName: model.NewCIStr(yyS[yypt-1].ident), + ResourceGroupName: ast.NewCIStr(yyS[yypt-1].ident), ResourceGroupOptionList: yyS[yypt-0].item.([]*ast.ResourceGroupOption), } } @@ -24687,7 +24686,7 @@ yynewstate: { parser.yyVAL.statement = &ast.DropResourceGroupStmt{ IfExists: yyS[yypt-1].item.(bool), - ResourceGroupName: model.NewCIStr(yyS[yypt-0].ident), + ResourceGroupName: ast.NewCIStr(yyS[yypt-0].ident), } } case 2759: @@ -24695,7 +24694,7 @@ yynewstate: parser.yyVAL.statement = &ast.CreatePlacementPolicyStmt{ OrReplace: yyS[yypt-5].item.(bool), IfNotExists: yyS[yypt-2].item.(bool), - PolicyName: model.NewCIStr(yyS[yypt-1].ident), + PolicyName: ast.NewCIStr(yyS[yypt-1].ident), PlacementOptions: yyS[yypt-0].item.([]*ast.PlacementOption), } } @@ -24703,7 +24702,7 @@ yynewstate: { parser.yyVAL.statement = &ast.AlterPlacementPolicyStmt{ IfExists: yyS[yypt-2].item.(bool), - PolicyName: model.NewCIStr(yyS[yypt-1].ident), + PolicyName: ast.NewCIStr(yyS[yypt-1].ident), PlacementOptions: yyS[yypt-0].item.([]*ast.PlacementOption), } } @@ -25674,7 +25673,7 @@ yynewstate: parser.yyVAL.item = &ast.QueryWatchOption{ Tp: ast.QueryWatchResourceGroup, ResourceGroupOption: &ast.QueryWatchResourceGroupOption{ - GroupNameStr: model.NewCIStr(yyS[yypt-0].ident), + GroupNameStr: ast.NewCIStr(yyS[yypt-0].ident), }, } } @@ -25704,21 +25703,21 @@ yynewstate: case 2952: { parser.yyVAL.item = &ast.QueryWatchTextOption{ - Type: model.WatchSimilar, + Type: ast.WatchSimilar, PatternExpr: yyS[yypt-0].expr, } } case 2953: { parser.yyVAL.item = &ast.QueryWatchTextOption{ - Type: model.WatchPlan, + Type: ast.WatchPlan, PatternExpr: yyS[yypt-0].expr, } } case 2954: { parser.yyVAL.item = &ast.QueryWatchTextOption{ - Type: yyS[yypt-2].item.(model.RunawayWatchType), + Type: yyS[yypt-2].item.(ast.RunawayWatchType), PatternExpr: yyS[yypt-0].expr, TypeSpecified: true, } diff --git a/pkg/parser/parser.y b/pkg/parser/parser.y index ed1b282b04c8e..5095af57e0290 100644 --- a/pkg/parser/parser.y +++ b/pkg/parser/parser.y @@ -31,7 +31,6 @@ import ( "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/ast" - "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/opcode" "github.com/pingcap/tidb/pkg/parser/auth" "github.com/pingcap/tidb/pkg/parser/charset" @@ -1715,14 +1714,14 @@ AlterTableStmt: } | "ALTER" IgnoreOptional "TABLE" TableName "ANALYZE" "PARTITION" PartitionNameList AnalyzeOptionListOpt { - $$ = &ast.AnalyzeTableStmt{TableNames: []*ast.TableName{$4.(*ast.TableName)}, PartitionNames: $7.([]model.CIStr), AnalyzeOpts: $8.([]ast.AnalyzeOpt)} + $$ = &ast.AnalyzeTableStmt{TableNames: []*ast.TableName{$4.(*ast.TableName)}, PartitionNames: $7.([]ast.CIStr), AnalyzeOpts: $8.([]ast.AnalyzeOpt)} } | "ALTER" IgnoreOptional "TABLE" TableName "ANALYZE" "PARTITION" PartitionNameList "INDEX" IndexNameList AnalyzeOptionListOpt { $$ = &ast.AnalyzeTableStmt{ TableNames: []*ast.TableName{$4.(*ast.TableName)}, - PartitionNames: $7.([]model.CIStr), - IndexNames: $9.([]model.CIStr), + PartitionNames: $7.([]ast.CIStr), + IndexNames: $9.([]ast.CIStr), IndexFlag: true, AnalyzeOpts: $10.([]ast.AnalyzeOpt), } @@ -1745,7 +1744,7 @@ AlterTableStmt: { $$ = &ast.CompactTableStmt{ Table: $4.(*ast.TableName), - PartitionNames: $7.([]model.CIStr), + PartitionNames: $7.([]ast.CIStr), ReplicaKind: ast.CompactReplicaKindAll, } } @@ -1753,7 +1752,7 @@ AlterTableStmt: { $$ = &ast.CompactTableStmt{ Table: $4.(*ast.TableName), - PartitionNames: $7.([]model.CIStr), + PartitionNames: $7.([]ast.CIStr), ReplicaKind: ast.CompactReplicaKindTiFlash, } } @@ -1819,35 +1818,35 @@ ResourceGroupRunawayOptionList: ResourceGroupRunawayWatchOption: "EXACT" { - $$ = model.WatchExact + $$ = ast.WatchExact } | "SIMILAR" { - $$ = model.WatchSimilar + $$ = ast.WatchSimilar } | "PLAN" { - $$ = model.WatchPlan + $$ = ast.WatchPlan } ResourceGroupRunawayActionOption: "DRYRUN" { - $$ = &ast.ResourceGroupRunawayActionOption{Type: model.RunawayActionDryRun} + $$ = &ast.ResourceGroupRunawayActionOption{Type: ast.RunawayActionDryRun} } | "COOLDOWN" { - $$ = &ast.ResourceGroupRunawayActionOption{Type: model.RunawayActionCooldown} + $$ = &ast.ResourceGroupRunawayActionOption{Type: ast.RunawayActionCooldown} } | "KILL" { - $$ = &ast.ResourceGroupRunawayActionOption{Type: model.RunawayActionKill} + $$ = &ast.ResourceGroupRunawayActionOption{Type: ast.RunawayActionKill} } | "SWITCH_GROUP" '(' ResourceGroupName ')' { $$ = &ast.ResourceGroupRunawayActionOption{ - Type: model.RunawayActionSwitchGroup, - SwitchGroupName: model.NewCIStr($3), + Type: ast.RunawayActionSwitchGroup, + SwitchGroupName: ast.NewCIStr($3), } } @@ -1860,28 +1859,28 @@ DirectResourceGroupRunawayOption: return 1 } $$ = &ast.ResourceGroupRunawayOption{ - Tp: model.RunawayRule, + Tp: ast.RunawayRule, RuleOption: &ast.ResourceGroupRunawayRuleOption{Tp: ast.RunawayRuleExecElapsed, ExecElapsed: $3}, } } | "PROCESSED_KEYS" EqOpt intLit { $$ = &ast.ResourceGroupRunawayOption{ - Tp: model.RunawayRule, + Tp: ast.RunawayRule, RuleOption: &ast.ResourceGroupRunawayRuleOption{Tp: ast.RunawayRuleProcessedKeys, ProcessedKeys: $3.(int64)}, } } | "RU" EqOpt intLit { $$ = &ast.ResourceGroupRunawayOption{ - Tp: model.RunawayRule, + Tp: ast.RunawayRule, RuleOption: &ast.ResourceGroupRunawayRuleOption{Tp: ast.RunawayRuleRequestUnit, RequestUnit: $3.(int64)}, } } | "ACTION" EqOpt ResourceGroupRunawayActionOption { $$ = &ast.ResourceGroupRunawayOption{ - Tp: model.RunawayAction, + Tp: ast.RunawayAction, ActionOption: $3.(*ast.ResourceGroupRunawayActionOption), } } @@ -1899,9 +1898,9 @@ DirectResourceGroupRunawayOption: } } $$ = &ast.ResourceGroupRunawayOption{ - Tp: model.RunawayWatch, + Tp: ast.RunawayWatch, WatchOption: &ast.ResourceGroupRunawayWatchOption{ - Type: $3.(model.RunawayWatchType), + Type: $3.(ast.RunawayWatchType), Duration: dur, }, } @@ -2156,7 +2155,7 @@ AlterTableSpecSingleOpt: { $$ = &ast.AlterTableSpec{ Tp: ast.AlterTablePartitionAttributes, - PartitionNames: []model.CIStr{model.NewCIStr($2)}, + PartitionNames: []ast.CIStr{ast.NewCIStr($2)}, AttributesSpec: $3.(*ast.AttributesSpec), } } @@ -2164,7 +2163,7 @@ AlterTableSpecSingleOpt: { $$ = &ast.AlterTableSpec{ Tp: ast.AlterTablePartitionOptions, - PartitionNames: []model.CIStr{model.NewCIStr($2)}, + PartitionNames: []ast.CIStr{ast.NewCIStr($2)}, Options: $3.([]*ast.TableOption), } } @@ -2370,7 +2369,7 @@ AlterTableSpec: if $3 == nil { ret.OnAllPartitions = true } else { - ret.PartitionNames = $3.([]model.CIStr) + ret.PartitionNames = $3.([]ast.CIStr) } $$ = ret } @@ -2404,7 +2403,7 @@ AlterTableSpec: $$ = &ast.AlterTableSpec{ IfExists: $3.(bool), Tp: ast.AlterTableDropPartition, - PartitionNames: $4.([]model.CIStr), + PartitionNames: $4.([]ast.CIStr), } } | "FIRST" "PARTITION" "LESS" "THAN" '(' BitExpr ')' IfExists @@ -2434,7 +2433,7 @@ AlterTableSpec: { $$ = &ast.AlterTableSpec{ Tp: ast.AlterTableExchangePartition, - PartitionNames: []model.CIStr{model.NewCIStr($3)}, + PartitionNames: []ast.CIStr{ast.NewCIStr($3)}, NewTable: $6.(*ast.TableName), WithValidation: $7.(bool), } @@ -2447,7 +2446,7 @@ AlterTableSpec: if $3 == nil { ret.OnAllPartitions = true } else { - ret.PartitionNames = $3.([]model.CIStr) + ret.PartitionNames = $3.([]ast.CIStr) } $$ = ret } @@ -2460,7 +2459,7 @@ AlterTableSpec: if $4 == nil { ret.OnAllPartitions = true } else { - ret.PartitionNames = $4.([]model.CIStr) + ret.PartitionNames = $4.([]ast.CIStr) } $$ = ret } @@ -2473,7 +2472,7 @@ AlterTableSpec: if $4 == nil { ret.OnAllPartitions = true } else { - ret.PartitionNames = $4.([]model.CIStr) + ret.PartitionNames = $4.([]ast.CIStr) } $$ = ret } @@ -2485,7 +2484,7 @@ AlterTableSpec: if $3 == nil { ret.OnAllPartitions = true } else { - ret.PartitionNames = $3.([]model.CIStr) + ret.PartitionNames = $3.([]ast.CIStr) } $$ = ret yylex.AppendError(yylex.Errorf("The IMPORT PARTITION TABLESPACE clause is parsed but ignored by all storage engines.")) @@ -2499,7 +2498,7 @@ AlterTableSpec: if $3 == nil { ret.OnAllPartitions = true } else { - ret.PartitionNames = $3.([]model.CIStr) + ret.PartitionNames = $3.([]ast.CIStr) } $$ = ret yylex.AppendError(yylex.Errorf("The DISCARD PARTITION TABLESPACE clause is parsed but ignored by all storage engines.")) @@ -2532,7 +2531,7 @@ AlterTableSpec: if $4 == nil { ret.OnAllPartitions = true } else { - ret.PartitionNames = $4.([]model.CIStr) + ret.PartitionNames = $4.([]ast.CIStr) } $$ = ret } @@ -2625,8 +2624,8 @@ AlterTableSpec: } | "RENAME" "COLUMN" Identifier "TO" Identifier { - oldColName := &ast.ColumnName{Name: model.NewCIStr($3)} - newColName := &ast.ColumnName{Name: model.NewCIStr($5)} + oldColName := &ast.ColumnName{Name: ast.NewCIStr($3)} + newColName := &ast.ColumnName{Name: ast.NewCIStr($5)} $$ = &ast.AlterTableSpec{ Tp: ast.AlterTableRenameColumn, OldColumnName: oldColName, @@ -2658,8 +2657,8 @@ AlterTableSpec: { $$ = &ast.AlterTableSpec{ Tp: ast.AlterTableRenameIndex, - FromKey: model.NewCIStr($3), - ToKey: model.NewCIStr($5), + FromKey: ast.NewCIStr($3), + ToKey: ast.NewCIStr($5), } } | LockClause @@ -2751,7 +2750,7 @@ AlterTableSpec: { $$ = &ast.AlterTableSpec{ Tp: ast.AlterTableIndexInvisible, - IndexName: model.NewCIStr($3), + IndexName: ast.NewCIStr($3), Visibility: $4.(ast.IndexVisibility), } } @@ -2782,7 +2781,7 @@ ReorganizePartitionRuleOpt: { ret := &ast.AlterTableSpec{ Tp: ast.AlterTableReorganizePartition, - PartitionNames: $1.([]model.CIStr), + PartitionNames: $1.([]ast.CIStr), PartDefinitions: $4.([]*ast.PartitionDefinition), } $$ = ret @@ -2814,11 +2813,11 @@ WithValidation: WithClustered: "CLUSTERED" { - $$ = model.PrimaryKeyTypeClustered + $$ = ast.PrimaryKeyTypeClustered } | "NONCLUSTERED" { - $$ = model.PrimaryKeyTypeNonClustered + $$ = ast.PrimaryKeyTypeNonClustered } GlobalOrLocalOpt: @@ -2937,11 +2936,11 @@ AlterTableSpecList: PartitionNameList: Identifier { - $$ = []model.CIStr{model.NewCIStr($1)} + $$ = []ast.CIStr{ast.NewCIStr($1)} } | PartitionNameList ',' Identifier { - $$ = append($1.([]model.CIStr), model.NewCIStr($3)) + $$ = append($1.([]ast.CIStr), ast.NewCIStr($3)) } ConstraintKeywordOpt: @@ -3085,7 +3084,7 @@ FlashbackToTimestampStmt: | "FLASHBACK" DatabaseSym DBName toTimestamp stringLit { $$ = &ast.FlashBackToTimestampStmt{ - DBName: model.NewCIStr($3), + DBName: ast.NewCIStr($3), FlashbackTS: ast.NewValueExpr($5, "", ""), FlashbackTSO: 0, } @@ -3117,7 +3116,7 @@ FlashbackToTimestampStmt: { if tsoValue, ok := $5.(uint64); ok && tsoValue > 0 { $$ = &ast.FlashBackToTimestampStmt{ - DBName: model.NewCIStr($3), + DBName: ast.NewCIStr($3), FlashbackTSO: tsoValue, } } else { @@ -3163,7 +3162,7 @@ FlashbackDatabaseStmt: "FLASHBACK" DatabaseSym DBName FlashbackToNewName { $$ = &ast.FlashBackDatabaseStmt{ - DBName: model.NewCIStr($3), + DBName: ast.NewCIStr($3), NewName: $4, } } @@ -3182,7 +3181,7 @@ SplitRegionStmt: $$ = &ast.SplitRegionStmt{ SplitSyntaxOpt: $2.(*ast.SplitSyntaxOption), Table: $4.(*ast.TableName), - PartitionNames: $5.([]model.CIStr), + PartitionNames: $5.([]ast.CIStr), SplitOpt: $6.(*ast.SplitOption), } } @@ -3191,8 +3190,8 @@ SplitRegionStmt: $$ = &ast.SplitRegionStmt{ SplitSyntaxOpt: $2.(*ast.SplitSyntaxOption), Table: $4.(*ast.TableName), - PartitionNames: $5.([]model.CIStr), - IndexName: model.NewCIStr($7), + PartitionNames: $5.([]ast.CIStr), + IndexName: ast.NewCIStr($7), SplitOpt: $8.(*ast.SplitOption), } } @@ -3241,27 +3240,27 @@ SplitSyntaxOption: AnalyzeTableStmt: "ANALYZE" NoWriteToBinLogAliasOpt "TABLE" TableNameList AllColumnsOrPredicateColumnsOpt AnalyzeOptionListOpt { - $$ = &ast.AnalyzeTableStmt{TableNames: $4.([]*ast.TableName), NoWriteToBinLog: $2.(bool), ColumnChoice: $5.(model.ColumnChoice), AnalyzeOpts: $6.([]ast.AnalyzeOpt)} + $$ = &ast.AnalyzeTableStmt{TableNames: $4.([]*ast.TableName), NoWriteToBinLog: $2.(bool), ColumnChoice: $5.(ast.ColumnChoice), AnalyzeOpts: $6.([]ast.AnalyzeOpt)} } | "ANALYZE" NoWriteToBinLogAliasOpt "TABLE" TableName "INDEX" IndexNameList AnalyzeOptionListOpt { - $$ = &ast.AnalyzeTableStmt{TableNames: []*ast.TableName{$4.(*ast.TableName)}, NoWriteToBinLog: $2.(bool), IndexNames: $6.([]model.CIStr), IndexFlag: true, AnalyzeOpts: $7.([]ast.AnalyzeOpt)} + $$ = &ast.AnalyzeTableStmt{TableNames: []*ast.TableName{$4.(*ast.TableName)}, NoWriteToBinLog: $2.(bool), IndexNames: $6.([]ast.CIStr), IndexFlag: true, AnalyzeOpts: $7.([]ast.AnalyzeOpt)} } | "ANALYZE" NoWriteToBinLogAliasOpt "INCREMENTAL" "TABLE" TableName "INDEX" IndexNameList AnalyzeOptionListOpt { - $$ = &ast.AnalyzeTableStmt{TableNames: []*ast.TableName{$5.(*ast.TableName)}, NoWriteToBinLog: $2.(bool), IndexNames: $7.([]model.CIStr), IndexFlag: true, Incremental: true, AnalyzeOpts: $8.([]ast.AnalyzeOpt)} + $$ = &ast.AnalyzeTableStmt{TableNames: []*ast.TableName{$5.(*ast.TableName)}, NoWriteToBinLog: $2.(bool), IndexNames: $7.([]ast.CIStr), IndexFlag: true, Incremental: true, AnalyzeOpts: $8.([]ast.AnalyzeOpt)} } | "ANALYZE" NoWriteToBinLogAliasOpt "TABLE" TableName "PARTITION" PartitionNameList AllColumnsOrPredicateColumnsOpt AnalyzeOptionListOpt { - $$ = &ast.AnalyzeTableStmt{TableNames: []*ast.TableName{$4.(*ast.TableName)}, NoWriteToBinLog: $2.(bool), PartitionNames: $6.([]model.CIStr), ColumnChoice: $7.(model.ColumnChoice), AnalyzeOpts: $8.([]ast.AnalyzeOpt)} + $$ = &ast.AnalyzeTableStmt{TableNames: []*ast.TableName{$4.(*ast.TableName)}, NoWriteToBinLog: $2.(bool), PartitionNames: $6.([]ast.CIStr), ColumnChoice: $7.(ast.ColumnChoice), AnalyzeOpts: $8.([]ast.AnalyzeOpt)} } | "ANALYZE" NoWriteToBinLogAliasOpt "TABLE" TableName "PARTITION" PartitionNameList "INDEX" IndexNameList AnalyzeOptionListOpt { $$ = &ast.AnalyzeTableStmt{ TableNames: []*ast.TableName{$4.(*ast.TableName)}, NoWriteToBinLog: $2.(bool), - PartitionNames: $6.([]model.CIStr), - IndexNames: $8.([]model.CIStr), + PartitionNames: $6.([]ast.CIStr), + IndexNames: $8.([]ast.CIStr), IndexFlag: true, AnalyzeOpts: $9.([]ast.AnalyzeOpt), } @@ -3271,8 +3270,8 @@ AnalyzeTableStmt: $$ = &ast.AnalyzeTableStmt{ TableNames: []*ast.TableName{$5.(*ast.TableName)}, NoWriteToBinLog: $2.(bool), - PartitionNames: $7.([]model.CIStr), - IndexNames: $9.([]model.CIStr), + PartitionNames: $7.([]ast.CIStr), + IndexNames: $9.([]ast.CIStr), IndexFlag: true, Incremental: true, AnalyzeOpts: $10.([]ast.AnalyzeOpt), @@ -3283,7 +3282,7 @@ AnalyzeTableStmt: $$ = &ast.AnalyzeTableStmt{ TableNames: []*ast.TableName{$4.(*ast.TableName)}, NoWriteToBinLog: $2.(bool), - ColumnNames: $8.([]model.CIStr), + ColumnNames: $8.([]ast.CIStr), AnalyzeOpts: $9.([]ast.AnalyzeOpt), HistogramOperation: ast.HistogramOperationUpdate, } @@ -3293,7 +3292,7 @@ AnalyzeTableStmt: $$ = &ast.AnalyzeTableStmt{ TableNames: []*ast.TableName{$4.(*ast.TableName)}, NoWriteToBinLog: $2.(bool), - ColumnNames: $8.([]model.CIStr), + ColumnNames: $8.([]ast.CIStr), HistogramOperation: ast.HistogramOperationDrop, } } @@ -3302,8 +3301,8 @@ AnalyzeTableStmt: $$ = &ast.AnalyzeTableStmt{ TableNames: []*ast.TableName{$4.(*ast.TableName)}, NoWriteToBinLog: $2.(bool), - ColumnNames: $6.([]model.CIStr), - ColumnChoice: model.ColumnList, + ColumnNames: $6.([]ast.CIStr), + ColumnChoice: ast.ColumnList, AnalyzeOpts: $7.([]ast.AnalyzeOpt)} } | "ANALYZE" NoWriteToBinLogAliasOpt "TABLE" TableName "PARTITION" PartitionNameList "COLUMNS" IdentList AnalyzeOptionListOpt @@ -3311,24 +3310,24 @@ AnalyzeTableStmt: $$ = &ast.AnalyzeTableStmt{ TableNames: []*ast.TableName{$4.(*ast.TableName)}, NoWriteToBinLog: $2.(bool), - PartitionNames: $6.([]model.CIStr), - ColumnNames: $8.([]model.CIStr), - ColumnChoice: model.ColumnList, + PartitionNames: $6.([]ast.CIStr), + ColumnNames: $8.([]ast.CIStr), + ColumnChoice: ast.ColumnList, AnalyzeOpts: $9.([]ast.AnalyzeOpt)} } AllColumnsOrPredicateColumnsOpt: /* empty */ { - $$ = model.DefaultChoice + $$ = ast.DefaultChoice } | "ALL" "COLUMNS" { - $$ = model.AllColumns + $$ = ast.AllColumns } | "PREDICATE" "COLUMNS" { - $$ = model.PredicateColumns + $$ = ast.PredicateColumns } AnalyzeOptionListOpt: @@ -3476,15 +3475,15 @@ ColumnDef: ColumnName: Identifier { - $$ = &ast.ColumnName{Name: model.NewCIStr($1)} + $$ = &ast.ColumnName{Name: ast.NewCIStr($1)} } | Identifier '.' Identifier { - $$ = &ast.ColumnName{Table: model.NewCIStr($1), Name: model.NewCIStr($3)} + $$ = &ast.ColumnName{Table: ast.NewCIStr($1), Name: ast.NewCIStr($3)} } | Identifier '.' Identifier '.' Identifier { - $$ = &ast.ColumnName{Schema: model.NewCIStr($1), Table: model.NewCIStr($3), Name: model.NewCIStr($5)} + $$ = &ast.ColumnName{Schema: ast.NewCIStr($1), Table: ast.NewCIStr($3), Name: ast.NewCIStr($5)} } ColumnNameList: @@ -3507,7 +3506,7 @@ ColumnNameListOpt: IdentListWithParenOpt: /* EMPTY */ { - $$ = []model.CIStr{} + $$ = []ast.CIStr{} } | '(' IdentList ')' { @@ -3517,11 +3516,11 @@ IdentListWithParenOpt: IdentList: Identifier { - $$ = []model.CIStr{model.NewCIStr($1)} + $$ = []ast.CIStr{ast.NewCIStr($1)} } | IdentList ',' Identifier { - $$ = append($1.([]model.CIStr), model.NewCIStr($3)) + $$ = append($1.([]ast.CIStr), ast.NewCIStr($3)) } ColumnNameOrUserVarListOpt: @@ -3641,7 +3640,7 @@ ColumnOption: // KEY is normally a synonym for INDEX. The key attribute PRIMARY KEY // can also be specified as just KEY when given in a column definition. // See http://dev.mysql.com/doc/refman/5.7/en/create-table.html - $$ = &ast.ColumnOption{Tp: ast.ColumnOptionPrimaryKey, PrimaryKeyTp: $3.(model.PrimaryKeyType), StrValue: $4} + $$ = &ast.ColumnOption{Tp: ast.ColumnOptionPrimaryKey, PrimaryKeyTp: $3.(ast.PrimaryKeyType), StrValue: $4} } | "UNIQUE" "GLOBAL" { @@ -3830,7 +3829,7 @@ ConstraintElem: if c.Option == nil { c.Option = &ast.IndexOption{} } - c.Option.Tp = indexType.(model.IndexType) + c.Option.Tp = indexType.(ast.IndexType) } $$ = c } @@ -3863,7 +3862,7 @@ ConstraintElem: if c.Option == nil { c.Option = &ast.IndexOption{} } - c.Option.Tp = indexType.(model.IndexType) + c.Option.Tp = indexType.(ast.IndexType) } $$ = c } @@ -3883,7 +3882,7 @@ ConstraintElem: if c.Option == nil { c.Option = &ast.IndexOption{} } - c.Option.Tp = indexType.(model.IndexType) + c.Option.Tp = indexType.(ast.IndexType) } $$ = c } @@ -3948,13 +3947,13 @@ ReferDef: OnDelete: "ON" "DELETE" ReferOpt { - $$ = &ast.OnDeleteOpt{ReferOpt: $3.(model.ReferOptionType)} + $$ = &ast.OnDeleteOpt{ReferOpt: $3.(ast.ReferOptionType)} } OnUpdate: "ON" "UPDATE" ReferOpt { - $$ = &ast.OnUpdateOpt{ReferOpt: $3.(model.ReferOptionType)} + $$ = &ast.OnUpdateOpt{ReferOpt: $3.(ast.ReferOptionType)} } OnDeleteUpdateOpt: @@ -3982,23 +3981,23 @@ OnDeleteUpdateOpt: ReferOpt: "RESTRICT" { - $$ = model.ReferOptionRestrict + $$ = ast.ReferOptionRestrict } | "CASCADE" { - $$ = model.ReferOptionCascade + $$ = ast.ReferOptionCascade } | "SET" "NULL" { - $$ = model.ReferOptionSetNull + $$ = ast.ReferOptionSetNull } | "NO" "ACTION" { - $$ = model.ReferOptionNoAction + $$ = ast.ReferOptionNoAction } | "SET" "DEFAULT" { - $$ = model.ReferOptionSetDefault + $$ = ast.ReferOptionSetDefault yylex.AppendError(yylex.Errorf("The SET DEFAULT clause is parsed but ignored by all storage engines.")) parser.lastErrorAsWarn() } @@ -4028,20 +4027,20 @@ BuiltinFunction: | identifier '(' ')' { $$ = &ast.FuncCallExpr{ - FnName: model.NewCIStr($1), + FnName: ast.NewCIStr($1), } } | identifier '(' ExpressionList ')' { $$ = &ast.FuncCallExpr{ - FnName: model.NewCIStr($1), + FnName: ast.NewCIStr($1), Args: $3.([]ast.ExprNode), } } | "REPLACE" '(' ExpressionList ')' { $$ = &ast.FuncCallExpr{ - FnName: model.NewCIStr($1), + FnName: ast.NewCIStr($1), Args: $3.([]ast.ExprNode), } } @@ -4056,23 +4055,23 @@ NowSymOptionFractionParentheses: NowSymOptionFraction: NowSym { - $$ = &ast.FuncCallExpr{FnName: model.NewCIStr("CURRENT_TIMESTAMP")} + $$ = &ast.FuncCallExpr{FnName: ast.NewCIStr("CURRENT_TIMESTAMP")} } | NowSymFunc '(' ')' { - $$ = &ast.FuncCallExpr{FnName: model.NewCIStr("CURRENT_TIMESTAMP")} + $$ = &ast.FuncCallExpr{FnName: ast.NewCIStr("CURRENT_TIMESTAMP")} } | NowSymFunc '(' NUM ')' { - $$ = &ast.FuncCallExpr{FnName: model.NewCIStr("CURRENT_TIMESTAMP"), Args: []ast.ExprNode{ast.NewValueExpr($3, parser.charset, parser.collation)}} + $$ = &ast.FuncCallExpr{FnName: ast.NewCIStr("CURRENT_TIMESTAMP"), Args: []ast.ExprNode{ast.NewValueExpr($3, parser.charset, parser.collation)}} } | CurdateSym '(' ')' { - $$ = &ast.FuncCallExpr{FnName: model.NewCIStr("CURRENT_DATE")} + $$ = &ast.FuncCallExpr{FnName: ast.NewCIStr("CURRENT_DATE")} } | "CURRENT_DATE" { - $$ = &ast.FuncCallExpr{FnName: model.NewCIStr("CURRENT_DATE")} + $$ = &ast.FuncCallExpr{FnName: ast.NewCIStr("CURRENT_DATE")} } NextValueForSequenceParentheses: @@ -4089,7 +4088,7 @@ NextValueForSequence: Name: $4.(*ast.TableName), } $$ = &ast.FuncCallExpr{ - FnName: model.NewCIStr(ast.NextVal), + FnName: ast.NewCIStr(ast.NextVal), Args: []ast.ExprNode{objNameExpr}, } } @@ -4099,7 +4098,7 @@ NextValueForSequence: Name: $3.(*ast.TableName), } $$ = &ast.FuncCallExpr{ - FnName: model.NewCIStr(ast.NextVal), + FnName: ast.NewCIStr(ast.NextVal), Args: []ast.ExprNode{objNameExpr}, } } @@ -4220,15 +4219,15 @@ CreateIndexStmt: var indexOption *ast.IndexOption if $12 != nil { indexOption = $12.(*ast.IndexOption) - if indexOption.Tp == model.IndexTypeInvalid { + if indexOption.Tp == ast.IndexTypeInvalid { if $6 != nil { - indexOption.Tp = $6.(model.IndexType) + indexOption.Tp = $6.(ast.IndexType) } } } else { indexOption = &ast.IndexOption{} if $6 != nil { - indexOption.Tp = $6.(model.IndexType) + indexOption.Tp = $6.(ast.IndexType) } } var indexLockAndAlgorithm *ast.IndexLockAndAlgorithm @@ -4241,10 +4240,10 @@ CreateIndexStmt: keyType := $2.(ast.IndexKeyType) isVectorIndex := keyType == ast.IndexKeyTypeVector - if isVectorIndex && indexOption.Tp == model.IndexTypeInvalid { - indexOption.Tp = model.IndexTypeHNSW + if isVectorIndex && indexOption.Tp == ast.IndexTypeInvalid { + indexOption.Tp = ast.IndexTypeHNSW } - if (isVectorIndex && indexOption.Tp != model.IndexTypeHNSW) || (!isVectorIndex && indexOption.Tp == model.IndexTypeHNSW) { + if (isVectorIndex && indexOption.Tp != ast.IndexTypeHNSW) || (!isVectorIndex && indexOption.Tp == ast.IndexTypeHNSW) { yylex.AppendError(ErrSyntax) return 1 } @@ -4366,7 +4365,7 @@ AlterDatabaseStmt: "ALTER" DatabaseSym DBName DatabaseOptionList { $$ = &ast.AlterDatabaseStmt{ - Name: model.NewCIStr($3), + Name: ast.NewCIStr($3), AlterDefaultDatabase: false, Options: $4.([]*ast.DatabaseOption), } @@ -4374,7 +4373,7 @@ AlterDatabaseStmt: | "ALTER" DatabaseSym DatabaseOptionList { $$ = &ast.AlterDatabaseStmt{ - Name: model.NewCIStr(""), + Name: ast.NewCIStr(""), AlterDefaultDatabase: true, Options: $3.([]*ast.DatabaseOption), } @@ -4396,7 +4395,7 @@ CreateDatabaseStmt: { $$ = &ast.CreateDatabaseStmt{ IfNotExists: $3.(bool), - Name: model.NewCIStr($4), + Name: ast.NewCIStr($4), Options: $5.([]*ast.DatabaseOption), } } @@ -4613,7 +4612,7 @@ SubPartitionMethod: { keyAlgorithm, _ := $3.(*ast.PartitionKeyAlgorithm) $$ = &ast.PartitionMethod{ - Tp: model.PartitionTypeKey, + Tp: ast.PartitionTypeKey, Linear: len($1) != 0, ColumnNames: $5.([]*ast.ColumnName), KeyAlgorithm: keyAlgorithm, @@ -4622,7 +4621,7 @@ SubPartitionMethod: | LinearOpt "HASH" '(' BitExpr ')' { $$ = &ast.PartitionMethod{ - Tp: model.PartitionTypeHash, + Tp: ast.PartitionTypeHash, Linear: len($1) != 0, Expr: $4.(ast.ExprNode), } @@ -4651,7 +4650,7 @@ PartitionMethod: { partitionInterval, _ := $5.(*ast.PartitionInterval) $$ = &ast.PartitionMethod{ - Tp: model.PartitionTypeRange, + Tp: ast.PartitionTypeRange, Expr: $3.(ast.ExprNode), Interval: partitionInterval, } @@ -4660,7 +4659,7 @@ PartitionMethod: { partitionInterval, _ := $6.(*ast.PartitionInterval) $$ = &ast.PartitionMethod{ - Tp: model.PartitionTypeRange, + Tp: ast.PartitionTypeRange, ColumnNames: $4.([]*ast.ColumnName), Interval: partitionInterval, } @@ -4668,21 +4667,21 @@ PartitionMethod: | "LIST" '(' BitExpr ')' { $$ = &ast.PartitionMethod{ - Tp: model.PartitionTypeList, + Tp: ast.PartitionTypeList, Expr: $3.(ast.ExprNode), } } | "LIST" FieldsOrColumns '(' ColumnNameList ')' { $$ = &ast.PartitionMethod{ - Tp: model.PartitionTypeList, + Tp: ast.PartitionTypeList, ColumnNames: $4.([]*ast.ColumnName), } } | "SYSTEM_TIME" "INTERVAL" Expression TimeUnit { $$ = &ast.PartitionMethod{ - Tp: model.PartitionTypeSystemTime, + Tp: ast.PartitionTypeSystemTime, Expr: $3.(ast.ExprNode), Unit: $4.(ast.TimeUnitType), } @@ -4690,14 +4689,14 @@ PartitionMethod: | "SYSTEM_TIME" "LIMIT" LengthNum { $$ = &ast.PartitionMethod{ - Tp: model.PartitionTypeSystemTime, + Tp: ast.PartitionTypeSystemTime, Limit: $3.(uint64), } } | "SYSTEM_TIME" { $$ = &ast.PartitionMethod{ - Tp: model.PartitionTypeSystemTime, + Tp: ast.PartitionTypeSystemTime, } } @@ -4831,7 +4830,7 @@ PartitionDefinition: "PARTITION" Identifier PartDefValuesOpt PartDefOptionList SubPartDefinitionListOpt { $$ = &ast.PartitionDefinition{ - Name: model.NewCIStr($2), + Name: ast.NewCIStr($2), Clause: $3.(ast.PartitionDefinitionClause), Options: $4.([]*ast.TableOption), Sub: $5.([]*ast.SubPartitionDefinition), @@ -4863,7 +4862,7 @@ SubPartDefinition: "SUBPARTITION" Identifier PartDefOptionList { $$ = &ast.SubPartitionDefinition{ - Name: model.NewCIStr($2), + Name: ast.NewCIStr($2), Options: $3.([]*ast.TableOption), } } @@ -5070,18 +5069,18 @@ CreateViewStmt: OrReplace: $2.(bool), ViewName: $7.(*ast.TableName), Select: selStmt, - Algorithm: $3.(model.ViewAlgorithm), + Algorithm: $3.(ast.ViewAlgorithm), Definer: $4.(*auth.UserIdentity), - Security: $5.(model.ViewSecurity), + Security: $5.(ast.ViewSecurity), } if $8 != nil { - x.Cols = $8.([]model.CIStr) + x.Cols = $8.([]ast.CIStr) } if $11 != nil { - x.CheckOption = $11.(model.ViewCheckOption) + x.CheckOption = $11.(ast.ViewCheckOption) endOffset = parser.startOffset(&yyS[yypt]) } else { - x.CheckOption = model.CheckOptionCascaded + x.CheckOption = ast.CheckOptionCascaded } selStmt.SetText(parser.lexer.client, strings.TrimSpace(parser.src[startOffset:endOffset])) $$ = x @@ -5100,19 +5099,19 @@ OrReplace: ViewAlgorithm: /* EMPTY */ { - $$ = model.AlgorithmUndefined + $$ = ast.AlgorithmUndefined } | "ALGORITHM" "=" "UNDEFINED" { - $$ = model.AlgorithmUndefined + $$ = ast.AlgorithmUndefined } | "ALGORITHM" "=" "MERGE" { - $$ = model.AlgorithmMerge + $$ = ast.AlgorithmMerge } | "ALGORITHM" "=" "TEMPTABLE" { - $$ = model.AlgorithmTemptable + $$ = ast.AlgorithmTemptable } ViewDefiner: @@ -5128,15 +5127,15 @@ ViewDefiner: ViewSQLSecurity: /* EMPTY */ { - $$ = model.SecurityDefiner + $$ = ast.SecurityDefiner } | "SQL" "SECURITY" "DEFINER" { - $$ = model.SecurityDefiner + $$ = ast.SecurityDefiner } | "SQL" "SECURITY" "INVOKER" { - $$ = model.SecurityInvoker + $$ = ast.SecurityInvoker } ViewName: @@ -5149,17 +5148,17 @@ ViewFieldList: } | '(' ColumnList ')' { - $$ = $2.([]model.CIStr) + $$ = $2.([]ast.CIStr) } ColumnList: Identifier { - $$ = []model.CIStr{model.NewCIStr($1)} + $$ = []ast.CIStr{ast.NewCIStr($1)} } | ColumnList ',' Identifier { - $$ = append($1.([]model.CIStr), model.NewCIStr($3)) + $$ = append($1.([]ast.CIStr), ast.NewCIStr($3)) } ViewCheckOption: @@ -5169,11 +5168,11 @@ ViewCheckOption: } | "WITH" "CASCADED" "CHECK" "OPTION" { - $$ = model.CheckOptionCascaded + $$ = ast.CheckOptionCascaded } | "WITH" "LOCAL" "CHECK" "OPTION" { - $$ = model.CheckOptionLocal + $$ = ast.CheckOptionLocal } /****************************************************************** @@ -5199,8 +5198,8 @@ DeleteWithoutUsingStmt: // Single Table tn := $7.(*ast.TableName) tn.IndexHints = $10.([]*ast.IndexHint) - tn.PartitionNames = $8.([]model.CIStr) - join := &ast.Join{Left: &ast.TableSource{Source: tn, AsName: $9.(model.CIStr)}, Right: nil} + tn.PartitionNames = $8.([]ast.CIStr) + join := &ast.Join{Left: &ast.TableSource{Source: tn, AsName: $9.(ast.CIStr)}, Right: nil} x := &ast.DeleteStmt{ TableRefs: &ast.TableRefsClause{TableRefs: join}, Priority: $3.(mysql.PriorityEnum), @@ -5286,7 +5285,7 @@ DatabaseSym: DropDatabaseStmt: "DROP" DatabaseSym IfExists DBName { - $$ = &ast.DropDatabaseStmt{IfExists: $3.(bool), Name: model.NewCIStr($4)} + $$ = &ast.DropDatabaseStmt{IfExists: $3.(bool), Name: ast.NewCIStr($4)} } /****************************************************************** @@ -5390,7 +5389,7 @@ DropStatsStmt: parser.lastErrorAsWarn() $$ = &ast.DropStatsStmt{ Tables: []*ast.TableName{$3.(*ast.TableName)}, - PartitionNames: $5.([]model.CIStr), + PartitionNames: $5.([]ast.CIStr), } } | "DROP" "STATS" TableName "GLOBAL" @@ -6407,7 +6406,7 @@ PredicateExpr: } | BitExpr memberof '(' SimpleExpr ')' { - $$ = &ast.FuncCallExpr{FnName: model.NewCIStr(ast.JSONMemberOf), Args: []ast.ExprNode{$1, $4}} + $$ = &ast.FuncCallExpr{FnName: ast.NewCIStr(ast.JSONMemberOf), Args: []ast.ExprNode{$1, $4}} } | BitExpr @@ -6432,19 +6431,19 @@ Field: } | Identifier '.' '*' %prec '*' { - wildCard := &ast.WildCardField{Table: model.NewCIStr($1)} + wildCard := &ast.WildCardField{Table: ast.NewCIStr($1)} $$ = &ast.SelectField{WildCard: wildCard} } | Identifier '.' Identifier '.' '*' %prec '*' { - wildCard := &ast.WildCardField{Schema: model.NewCIStr($1), Table: model.NewCIStr($3)} + wildCard := &ast.WildCardField{Schema: ast.NewCIStr($1), Table: ast.NewCIStr($3)} $$ = &ast.SelectField{WildCard: wildCard} } | Expression FieldAsNameOpt { expr := $1 asName := $2 - $$ = &ast.SelectField{Expr: expr, AsName: model.NewCIStr(asName)} + $$ = &ast.SelectField{Expr: expr, AsName: ast.NewCIStr(asName)} } FieldAsNameOpt: @@ -6593,7 +6592,7 @@ IndexOptionList: opt1.ParserName = opt2.ParserName } else if opt2.Visibility != ast.IndexVisibilityDefault { opt1.Visibility = opt2.Visibility - } else if opt2.PrimaryKeyTp != model.PrimaryKeyTypeDefault { + } else if opt2.PrimaryKeyTp != ast.PrimaryKeyTypeDefault { opt1.PrimaryKeyTp = opt2.PrimaryKeyTp } else if opt2.Global { opt1.Global = true @@ -6614,13 +6613,13 @@ IndexOption: | IndexType { $$ = &ast.IndexOption{ - Tp: $1.(model.IndexType), + Tp: $1.(ast.IndexType), } } | "WITH" "PARSER" Identifier { $$ = &ast.IndexOption{ - ParserName: model.NewCIStr($3), + ParserName: ast.NewCIStr($3), } yylex.AppendError(yylex.Errorf("The WITH PARASER clause is parsed but ignored by all storage engines.")) parser.lastErrorAsWarn() @@ -6640,7 +6639,7 @@ IndexOption: | WithClustered { $$ = &ast.IndexOption{ - PrimaryKeyTp: $1.(model.PrimaryKeyType), + PrimaryKeyTp: $1.(ast.PrimaryKeyType), } } | "GLOBAL" @@ -6655,17 +6654,17 @@ IndexOption: Global: false, } } -| "PRE_SPLIT_REGIONS" EqOpt '(' SplitOption ')' - { +| "PRE_SPLIT_REGIONS" EqOpt '(' SplitOption ')' + { $$ = &ast.IndexOption{ SplitOpt: $4.(*ast.SplitOption), } } -| "PRE_SPLIT_REGIONS" EqOpt Int64Num +| "PRE_SPLIT_REGIONS" EqOpt Int64Num { $$ = &ast.IndexOption{ - SplitOpt: &ast.SplitOption{ - Num: $3.(int64), + SplitOpt: &ast.SplitOption{ + Num: $3.(int64), }, } } @@ -6722,23 +6721,23 @@ IndexType: IndexTypeName: "BTREE" { - $$ = model.IndexTypeBtree + $$ = ast.IndexTypeBtree } | "HASH" { - $$ = model.IndexTypeHash + $$ = ast.IndexTypeHash } | "RTREE" { - $$ = model.IndexTypeRtree + $$ = ast.IndexTypeRtree } | "HYPO" { - $$ = model.IndexTypeHypo + $$ = ast.IndexTypeHypo } | "HNSW" { - $$ = model.IndexTypeHNSW + $$ = ast.IndexTypeHNSW } IndexInvisible: @@ -7331,7 +7330,7 @@ ProcedureCall: { $$ = &ast.FuncCallExpr{ Tp: ast.FuncCallExprTypeGeneric, - FnName: model.NewCIStr($1), + FnName: ast.NewCIStr($1), Args: []ast.ExprNode{}, } } @@ -7339,8 +7338,8 @@ ProcedureCall: { $$ = &ast.FuncCallExpr{ Tp: ast.FuncCallExprTypeGeneric, - Schema: model.NewCIStr($1), - FnName: model.NewCIStr($3), + Schema: ast.NewCIStr($1), + FnName: ast.NewCIStr($3), Args: []ast.ExprNode{}, } } @@ -7348,7 +7347,7 @@ ProcedureCall: { $$ = &ast.FuncCallExpr{ Tp: ast.FuncCallExprTypeGeneric, - FnName: model.NewCIStr($1), + FnName: ast.NewCIStr($1), Args: $3.([]ast.ExprNode), } } @@ -7356,8 +7355,8 @@ ProcedureCall: { $$ = &ast.FuncCallExpr{ Tp: ast.FuncCallExprTypeGeneric, - Schema: model.NewCIStr($1), - FnName: model.NewCIStr($3), + Schema: ast.NewCIStr($1), + FnName: ast.NewCIStr($3), Args: $5.([]ast.ExprNode), } } @@ -7382,7 +7381,7 @@ InsertIntoStmt: if $2 != nil { x.TableHints = $2.([]*ast.TableOptimizerHint) } - x.PartitionNames = $7.([]model.CIStr) + x.PartitionNames = $7.([]ast.CIStr) $$ = x } @@ -7546,7 +7545,7 @@ ReplaceIntoStmt: x.Priority = $3.(mysql.PriorityEnum) ts := &ast.TableSource{Source: $5.(*ast.TableName)} x.Table = &ast.TableRefsClause{TableRefs: &ast.Join{Left: ts}} - x.PartitionNames = $6.([]model.CIStr) + x.PartitionNames = $6.([]ast.CIStr) $$ = x } @@ -7773,7 +7772,7 @@ BitExpr: | BitExpr '+' "INTERVAL" Expression TimeUnit %prec '+' { $$ = &ast.FuncCallExpr{ - FnName: model.NewCIStr("DATE_ADD"), + FnName: ast.NewCIStr("DATE_ADD"), Args: []ast.ExprNode{ $1, $4, @@ -7784,7 +7783,7 @@ BitExpr: | BitExpr '-' "INTERVAL" Expression TimeUnit %prec '+' { $$ = &ast.FuncCallExpr{ - FnName: model.NewCIStr("DATE_SUB"), + FnName: ast.NewCIStr("DATE_SUB"), Args: []ast.ExprNode{ $1, $4, @@ -7795,7 +7794,7 @@ BitExpr: | "INTERVAL" Expression TimeUnit '+' BitExpr %prec '+' { $$ = &ast.FuncCallExpr{ - FnName: model.NewCIStr("DATE_ADD"), + FnName: ast.NewCIStr("DATE_ADD"), Args: []ast.ExprNode{ $5, $2, @@ -7833,22 +7832,22 @@ SimpleIdent: Identifier { $$ = &ast.ColumnNameExpr{Name: &ast.ColumnName{ - Name: model.NewCIStr($1), + Name: ast.NewCIStr($1), }} } | Identifier '.' Identifier { $$ = &ast.ColumnNameExpr{Name: &ast.ColumnName{ - Table: model.NewCIStr($1), - Name: model.NewCIStr($3), + Table: ast.NewCIStr($1), + Name: ast.NewCIStr($3), }} } | Identifier '.' Identifier '.' Identifier { $$ = &ast.ColumnNameExpr{Name: &ast.ColumnName{ - Schema: model.NewCIStr($1), - Table: model.NewCIStr($3), - Name: model.NewCIStr($5), + Schema: ast.NewCIStr($1), + Table: ast.NewCIStr($3), + Name: ast.NewCIStr($5), }} } @@ -7887,7 +7886,7 @@ SimpleExpr: } | SimpleExpr pipes SimpleExpr { - $$ = &ast.FuncCallExpr{FnName: model.NewCIStr(ast.Concat), Args: []ast.ExprNode{$1, $3}} + $$ = &ast.FuncCallExpr{FnName: ast.NewCIStr(ast.Concat), Args: []ast.ExprNode{$1, $3}} } | not2 SimpleExpr %prec neg { @@ -7929,15 +7928,15 @@ SimpleExpr: case "d": tp.SetCharset("") tp.SetCollate("") - $$ = &ast.FuncCallExpr{FnName: model.NewCIStr(ast.DateLiteral), Args: []ast.ExprNode{$3}} + $$ = &ast.FuncCallExpr{FnName: ast.NewCIStr(ast.DateLiteral), Args: []ast.ExprNode{$3}} case "t": tp.SetCharset("") tp.SetCollate("") - $$ = &ast.FuncCallExpr{FnName: model.NewCIStr(ast.TimeLiteral), Args: []ast.ExprNode{$3}} + $$ = &ast.FuncCallExpr{FnName: ast.NewCIStr(ast.TimeLiteral), Args: []ast.ExprNode{$3}} case "ts": tp.SetCharset("") tp.SetCollate("") - $$ = &ast.FuncCallExpr{FnName: model.NewCIStr(ast.TimestampLiteral), Args: []ast.ExprNode{$3}} + $$ = &ast.FuncCallExpr{FnName: ast.NewCIStr(ast.TimestampLiteral), Args: []ast.ExprNode{$3}} default: $$ = $3 } @@ -8017,7 +8016,7 @@ SimpleExpr: // See https://dev.mysql.com/doc/refman/5.7/en/cast-functions.html#function_convert charset1 := ast.NewValueExpr($5, "", "") $$ = &ast.FuncCallExpr{ - FnName: model.NewCIStr($1), + FnName: ast.NewCIStr($1), Args: []ast.ExprNode{$3, charset1}, } } @@ -8032,13 +8031,13 @@ SimpleExpr: | SimpleIdent jss stringLit { expr := ast.NewValueExpr($3, parser.charset, parser.collation) - $$ = &ast.FuncCallExpr{FnName: model.NewCIStr(ast.JSONExtract), Args: []ast.ExprNode{$1, expr}} + $$ = &ast.FuncCallExpr{FnName: ast.NewCIStr(ast.JSONExtract), Args: []ast.ExprNode{$1, expr}} } | SimpleIdent juss stringLit { expr := ast.NewValueExpr($3, parser.charset, parser.collation) - extract := &ast.FuncCallExpr{FnName: model.NewCIStr(ast.JSONExtract), Args: []ast.ExprNode{$1, expr}} - $$ = &ast.FuncCallExpr{FnName: model.NewCIStr(ast.JSONUnquote), Args: []ast.ExprNode{extract}} + extract := &ast.FuncCallExpr{FnName: ast.NewCIStr(ast.JSONExtract), Args: []ast.ExprNode{$1, expr}} + $$ = &ast.FuncCallExpr{FnName: ast.NewCIStr(ast.JSONUnquote), Args: []ast.ExprNode{extract}} } ArrayKwdOpt: @@ -8139,19 +8138,19 @@ FunctionNameDatetimePrecision: FunctionCallKeyword: FunctionNameConflict '(' ExpressionListOpt ')' { - $$ = &ast.FuncCallExpr{FnName: model.NewCIStr($1), Args: $3.([]ast.ExprNode)} + $$ = &ast.FuncCallExpr{FnName: ast.NewCIStr($1), Args: $3.([]ast.ExprNode)} } | builtinUser '(' ExpressionListOpt ')' { - $$ = &ast.FuncCallExpr{FnName: model.NewCIStr($1), Args: $3.([]ast.ExprNode)} + $$ = &ast.FuncCallExpr{FnName: ast.NewCIStr($1), Args: $3.([]ast.ExprNode)} } | FunctionNameOptionalBraces OptionalBraces { - $$ = &ast.FuncCallExpr{FnName: model.NewCIStr($1)} + $$ = &ast.FuncCallExpr{FnName: ast.NewCIStr($1)} } | builtinCurDate '(' ')' { - $$ = &ast.FuncCallExpr{FnName: model.NewCIStr($1)} + $$ = &ast.FuncCallExpr{FnName: ast.NewCIStr($1)} } | FunctionNameDatetimePrecision FuncDatetimePrec { @@ -8159,14 +8158,14 @@ FunctionCallKeyword: if $2 != nil { args = append(args, $2.(ast.ExprNode)) } - $$ = &ast.FuncCallExpr{FnName: model.NewCIStr($1), Args: args} + $$ = &ast.FuncCallExpr{FnName: ast.NewCIStr($1), Args: args} } | "CHAR" '(' ExpressionList ')' { nilVal := ast.NewValueExpr(nil, parser.charset, parser.collation) args := $3.([]ast.ExprNode) $$ = &ast.FuncCallExpr{ - FnName: model.NewCIStr(ast.CharFunc), + FnName: ast.NewCIStr(ast.CharFunc), Args: append(args, nilVal), } } @@ -8175,28 +8174,28 @@ FunctionCallKeyword: charset1 := ast.NewValueExpr($5, "", "") args := $3.([]ast.ExprNode) $$ = &ast.FuncCallExpr{ - FnName: model.NewCIStr(ast.CharFunc), + FnName: ast.NewCIStr(ast.CharFunc), Args: append(args, charset1), } } | "DATE" stringLit { expr := ast.NewValueExpr($2, "", "") - $$ = &ast.FuncCallExpr{FnName: model.NewCIStr(ast.DateLiteral), Args: []ast.ExprNode{expr}} + $$ = &ast.FuncCallExpr{FnName: ast.NewCIStr(ast.DateLiteral), Args: []ast.ExprNode{expr}} } | "TIME" stringLit { expr := ast.NewValueExpr($2, "", "") - $$ = &ast.FuncCallExpr{FnName: model.NewCIStr(ast.TimeLiteral), Args: []ast.ExprNode{expr}} + $$ = &ast.FuncCallExpr{FnName: ast.NewCIStr(ast.TimeLiteral), Args: []ast.ExprNode{expr}} } | "TIMESTAMP" stringLit { expr := ast.NewValueExpr($2, "", "") - $$ = &ast.FuncCallExpr{FnName: model.NewCIStr(ast.TimestampLiteral), Args: []ast.ExprNode{expr}} + $$ = &ast.FuncCallExpr{FnName: ast.NewCIStr(ast.TimestampLiteral), Args: []ast.ExprNode{expr}} } | "INSERT" '(' ExpressionListOpt ')' { - $$ = &ast.FuncCallExpr{FnName: model.NewCIStr(ast.InsertFunc), Args: $3.([]ast.ExprNode)} + $$ = &ast.FuncCallExpr{FnName: ast.NewCIStr(ast.InsertFunc), Args: $3.([]ast.ExprNode)} } | "MOD" '(' BitExpr ',' BitExpr ')' { @@ -8204,22 +8203,22 @@ FunctionCallKeyword: } | "PASSWORD" '(' ExpressionListOpt ')' { - $$ = &ast.FuncCallExpr{FnName: model.NewCIStr(ast.PasswordFunc), Args: $3.([]ast.ExprNode)} + $$ = &ast.FuncCallExpr{FnName: ast.NewCIStr(ast.PasswordFunc), Args: $3.([]ast.ExprNode)} } FunctionCallNonKeyword: builtinCurTime '(' FuncDatetimePrecListOpt ')' { - $$ = &ast.FuncCallExpr{FnName: model.NewCIStr($1), Args: $3.([]ast.ExprNode)} + $$ = &ast.FuncCallExpr{FnName: ast.NewCIStr($1), Args: $3.([]ast.ExprNode)} } | builtinSysDate '(' FuncDatetimePrecListOpt ')' { - $$ = &ast.FuncCallExpr{FnName: model.NewCIStr($1), Args: $3.([]ast.ExprNode)} + $$ = &ast.FuncCallExpr{FnName: ast.NewCIStr($1), Args: $3.([]ast.ExprNode)} } | FunctionNameDateArithMultiForms '(' Expression ',' Expression ')' { $$ = &ast.FuncCallExpr{ - FnName: model.NewCIStr($1), + FnName: ast.NewCIStr($1), Args: []ast.ExprNode{ $3, $5, @@ -8230,7 +8229,7 @@ FunctionCallNonKeyword: | FunctionNameDateArithMultiForms '(' Expression ',' "INTERVAL" Expression TimeUnit ')' { $$ = &ast.FuncCallExpr{ - FnName: model.NewCIStr($1), + FnName: ast.NewCIStr($1), Args: []ast.ExprNode{ $3, $6, @@ -8241,7 +8240,7 @@ FunctionCallNonKeyword: | FunctionNameDateArith '(' Expression ',' "INTERVAL" Expression TimeUnit ')' { $$ = &ast.FuncCallExpr{ - FnName: model.NewCIStr($1), + FnName: ast.NewCIStr($1), Args: []ast.ExprNode{ $3, $6, @@ -8253,14 +8252,14 @@ FunctionCallNonKeyword: { timeUnit := &ast.TimeUnitExpr{Unit: $3.(ast.TimeUnitType)} $$ = &ast.FuncCallExpr{ - FnName: model.NewCIStr($1), + FnName: ast.NewCIStr($1), Args: []ast.ExprNode{timeUnit, $5}, } } | "GET_FORMAT" '(' GetFormatSelector ',' Expression ')' { $$ = &ast.FuncCallExpr{ - FnName: model.NewCIStr($1), + FnName: ast.NewCIStr($1), Args: []ast.ExprNode{ &ast.GetFormatSelectorExpr{Selector: $3.(ast.GetFormatSelectorType)}, $5, @@ -8269,61 +8268,61 @@ FunctionCallNonKeyword: } | builtinPosition '(' BitExpr "IN" Expression ')' { - $$ = &ast.FuncCallExpr{FnName: model.NewCIStr($1), Args: []ast.ExprNode{$3, $5}} + $$ = &ast.FuncCallExpr{FnName: ast.NewCIStr($1), Args: []ast.ExprNode{$3, $5}} } | builtinSubstring '(' Expression ',' Expression ')' { $$ = &ast.FuncCallExpr{ - FnName: model.NewCIStr($1), + FnName: ast.NewCIStr($1), Args: []ast.ExprNode{$3, $5}, } } | builtinSubstring '(' Expression "FROM" Expression ')' { $$ = &ast.FuncCallExpr{ - FnName: model.NewCIStr($1), + FnName: ast.NewCIStr($1), Args: []ast.ExprNode{$3, $5}, } } | builtinSubstring '(' Expression ',' Expression ',' Expression ')' { $$ = &ast.FuncCallExpr{ - FnName: model.NewCIStr($1), + FnName: ast.NewCIStr($1), Args: []ast.ExprNode{$3, $5, $7}, } } | builtinSubstring '(' Expression "FROM" Expression "FOR" Expression ')' { $$ = &ast.FuncCallExpr{ - FnName: model.NewCIStr($1), + FnName: ast.NewCIStr($1), Args: []ast.ExprNode{$3, $5, $7}, } } | "TIMESTAMPADD" '(' TimestampUnit ',' Expression ',' Expression ')' { $$ = &ast.FuncCallExpr{ - FnName: model.NewCIStr($1), + FnName: ast.NewCIStr($1), Args: []ast.ExprNode{&ast.TimeUnitExpr{Unit: $3.(ast.TimeUnitType)}, $5, $7}, } } | "TIMESTAMPDIFF" '(' TimestampUnit ',' Expression ',' Expression ')' { $$ = &ast.FuncCallExpr{ - FnName: model.NewCIStr($1), + FnName: ast.NewCIStr($1), Args: []ast.ExprNode{&ast.TimeUnitExpr{Unit: $3.(ast.TimeUnitType)}, $5, $7}, } } | builtinTrim '(' Expression ')' { $$ = &ast.FuncCallExpr{ - FnName: model.NewCIStr($1), + FnName: ast.NewCIStr($1), Args: []ast.ExprNode{$3}, } } | builtinTrim '(' Expression "FROM" Expression ')' { $$ = &ast.FuncCallExpr{ - FnName: model.NewCIStr($1), + FnName: ast.NewCIStr($1), Args: []ast.ExprNode{$5, $3}, } } @@ -8332,7 +8331,7 @@ FunctionCallNonKeyword: spaceVal := ast.NewValueExpr(" ", parser.charset, parser.collation) direction := &ast.TrimDirectionExpr{Direction: $3.(ast.TrimDirectionType)} $$ = &ast.FuncCallExpr{ - FnName: model.NewCIStr($1), + FnName: ast.NewCIStr($1), Args: []ast.ExprNode{$5, spaceVal, direction}, } } @@ -8340,28 +8339,28 @@ FunctionCallNonKeyword: { direction := &ast.TrimDirectionExpr{Direction: $3.(ast.TrimDirectionType)} $$ = &ast.FuncCallExpr{ - FnName: model.NewCIStr($1), + FnName: ast.NewCIStr($1), Args: []ast.ExprNode{$6, $4, direction}, } } | weightString '(' Expression ')' { $$ = &ast.FuncCallExpr{ - FnName: model.NewCIStr($1), + FnName: ast.NewCIStr($1), Args: []ast.ExprNode{$3}, } } | weightString '(' Expression "AS" Char FieldLen ')' { $$ = &ast.FuncCallExpr{ - FnName: model.NewCIStr($1), + FnName: ast.NewCIStr($1), Args: []ast.ExprNode{$3, ast.NewValueExpr("CHAR", parser.charset, parser.collation), ast.NewValueExpr($6, parser.charset, parser.collation)}, } } | weightString '(' Expression "AS" "BINARY" FieldLen ')' { $$ = &ast.FuncCallExpr{ - FnName: model.NewCIStr($1), + FnName: ast.NewCIStr($1), Args: []ast.ExprNode{$3, ast.NewValueExpr("BINARY", parser.charset, parser.collation), ast.NewValueExpr($6, parser.charset, parser.collation)}, } } @@ -8369,13 +8368,13 @@ FunctionCallNonKeyword: | builtinTranslate '(' Expression ',' Expression ',' Expression ')' { $$ = &ast.FuncCallExpr{ - FnName: model.NewCIStr($1), + FnName: ast.NewCIStr($1), Args: []ast.ExprNode{$3, $5, $7}, } } | "COMPRESS" '(' ExpressionListOpt ')' { - $$ = &ast.FuncCallExpr{FnName: model.NewCIStr($1), Args: $3.([]ast.ExprNode)} + $$ = &ast.FuncCallExpr{FnName: ast.NewCIStr($1), Args: $3.([]ast.ExprNode)} } GetFormatSelector: @@ -8425,7 +8424,7 @@ FunctionNameSequence: Name: $3.(*ast.TableName), } $$ = &ast.FuncCallExpr{ - FnName: model.NewCIStr(ast.LastVal), + FnName: ast.NewCIStr(ast.LastVal), Args: []ast.ExprNode{objNameExpr}, } } @@ -8436,7 +8435,7 @@ FunctionNameSequence: } valueExpr := ast.NewValueExpr($5, parser.charset, parser.collation) $$ = &ast.FuncCallExpr{ - FnName: model.NewCIStr(ast.SetVal), + FnName: ast.NewCIStr(ast.SetVal), Args: []ast.ExprNode{objNameExpr, valueExpr}, } } @@ -8668,7 +8667,7 @@ FunctionCallGeneric: identifier '(' ExpressionListOpt ')' { $$ = &ast.FuncCallExpr{ - FnName: model.NewCIStr($1), + FnName: ast.NewCIStr($1), Args: $3.([]ast.ExprNode), } } @@ -8682,8 +8681,8 @@ FunctionCallGeneric: } $$ = &ast.FuncCallExpr{ Tp: tp, - Schema: model.NewCIStr($1), - FnName: model.NewCIStr($3), + Schema: ast.NewCIStr($1), + FnName: ast.NewCIStr($3), Args: $5.([]ast.ExprNode), } } @@ -9050,7 +9049,7 @@ PriorityOpt: TableName: Identifier { - $$ = &ast.TableName{Name: model.NewCIStr($1)} + $$ = &ast.TableName{Name: ast.NewCIStr($1)} } | Identifier '.' Identifier { @@ -9059,11 +9058,11 @@ TableName: yylex.AppendError(ErrWrongDBName.GenWithStackByArgs(schema)) return 1 } - $$ = &ast.TableName{Schema: model.NewCIStr(schema), Name: model.NewCIStr($3)} + $$ = &ast.TableName{Schema: ast.NewCIStr(schema), Name: ast.NewCIStr($3)} } | '*' '.' Identifier { - $$ = &ast.TableName{Schema: model.NewCIStr("*"), Name: model.NewCIStr($3)} + $$ = &ast.TableName{Schema: ast.NewCIStr("*"), Name: ast.NewCIStr($3)} } TableNameList: @@ -9080,11 +9079,11 @@ TableNameList: TableNameOptWild: Identifier OptWild { - $$ = &ast.TableName{Name: model.NewCIStr($1)} + $$ = &ast.TableName{Name: ast.NewCIStr($1)} } | Identifier '.' Identifier OptWild { - $$ = &ast.TableName{Schema: model.NewCIStr($1), Name: model.NewCIStr($3)} + $$ = &ast.TableName{Schema: ast.NewCIStr($1), Name: ast.NewCIStr($3)} } TableAliasRefList: @@ -9548,8 +9547,8 @@ CommonTableExpr: Identifier IdentListWithParenOpt "AS" SubSelect { cte := &ast.CommonTableExpression{} - cte.Name = model.NewCIStr($1) - cte.ColNameList = $2.([]model.CIStr) + cte.Name = ast.NewCIStr($1) + cte.ColNameList = $2.([]ast.CIStr) cte.Query = $4.(*ast.SubqueryExpr) $$ = cte } @@ -9580,14 +9579,14 @@ WindowDefinition: WindowName "AS" WindowSpec { var spec = $3.(ast.WindowSpec) - spec.Name = $1.(model.CIStr) + spec.Name = $1.(ast.CIStr) $$ = spec } WindowName: Identifier { - $$ = model.NewCIStr($1) + $$ = ast.NewCIStr($1) } WindowSpec: @@ -9599,7 +9598,7 @@ WindowSpec: WindowSpecDetails: OptExistingWindowName OptPartitionClause OptWindowOrderByClause OptWindowFrameClause { - spec := ast.WindowSpec{Ref: $1.(model.CIStr)} + spec := ast.WindowSpec{Ref: $1.(ast.CIStr)} if $2 != nil { spec.PartitionBy = $2.(*ast.PartitionByClause) } @@ -9614,7 +9613,7 @@ WindowSpecDetails: OptExistingWindowName: { - $$ = model.CIStr{} + $$ = ast.CIStr{} } | WindowName @@ -9738,7 +9737,7 @@ WindowingClause: WindowNameOrSpec: WindowName { - $$ = ast.WindowSpec{Name: $1.(model.CIStr), OnlyAlias: true} + $$ = ast.WindowSpec{Name: $1.(ast.CIStr), OnlyAlias: true} } | WindowSpec @@ -9893,7 +9892,7 @@ TableFactor: TableName PartitionNameListOpt TableAsNameOpt AsOfClauseOpt IndexHintListOpt TableSampleOpt { tn := $1.(*ast.TableName) - tn.PartitionNames = $2.([]model.CIStr) + tn.PartitionNames = $2.([]ast.CIStr) tn.IndexHints = $5.([]*ast.IndexHint) if $6 != nil { tn.TableSample = $6.(*ast.TableSample) @@ -9901,12 +9900,12 @@ TableFactor: if $4 != nil { tn.AsOf = $4.(*ast.AsOfClause) } - $$ = &ast.TableSource{Source: tn, AsName: $3.(model.CIStr)} + $$ = &ast.TableSource{Source: tn, AsName: $3.(ast.CIStr)} } | SubSelect TableAsNameOpt { resultNode := $1.(*ast.SubqueryExpr).Query - $$ = &ast.TableSource{Source: resultNode, AsName: $2.(model.CIStr)} + $$ = &ast.TableSource{Source: resultNode, AsName: $2.(ast.CIStr)} } | '(' TableRefs ')' { @@ -9918,7 +9917,7 @@ TableFactor: PartitionNameListOpt: /* empty */ { - $$ = []model.CIStr{} + $$ = []ast.CIStr{} } | "PARTITION" '(' PartitionNameList ')' { @@ -9928,18 +9927,18 @@ PartitionNameListOpt: TableAsNameOpt: %prec empty { - $$ = model.CIStr{} + $$ = ast.CIStr{} } | TableAsName TableAsName: Identifier { - $$ = model.NewCIStr($1) + $$ = ast.NewCIStr($1) } | "AS" Identifier { - $$ = model.NewCIStr($2) + $$ = ast.NewCIStr($2) } IndexHintType: @@ -9977,7 +9976,7 @@ IndexHint: IndexHintType IndexHintScope '(' IndexNameList ')' { $$ = &ast.IndexHint{ - IndexNames: $4.([]model.CIStr), + IndexNames: $4.([]ast.CIStr), HintType: $1.(ast.IndexHintType), HintScope: $2.(ast.IndexHintScope), } @@ -9985,24 +9984,24 @@ IndexHint: IndexNameList: { - var nameList []model.CIStr + var nameList []ast.CIStr $$ = nameList } | Identifier { - $$ = []model.CIStr{model.NewCIStr($1)} + $$ = []ast.CIStr{ast.NewCIStr($1)} } | IndexNameList ',' Identifier { - $$ = append($1.([]model.CIStr), model.NewCIStr($3)) + $$ = append($1.([]ast.CIStr), ast.NewCIStr($3)) } | "PRIMARY" { - $$ = []model.CIStr{model.NewCIStr($1)} + $$ = []ast.CIStr{ast.NewCIStr($1)} } | IndexNameList ',' "PRIMARY" { - $$ = append($1.([]model.CIStr), model.NewCIStr($3)) + $$ = append($1.([]ast.CIStr), ast.NewCIStr($3)) } IndexHintList: @@ -10776,7 +10775,7 @@ SetStmt: } | "SET" "RESOURCE" "GROUP" ResourceGroupName { - $$ = &ast.SetResourceGroupStmt{Name: model.NewCIStr($4)} + $$ = &ast.SetResourceGroupStmt{Name: ast.NewCIStr($4)} } SetRoleStmt: @@ -11570,7 +11569,7 @@ ShowStmt: Tp: ast.ShowRegions, Table: $3.(*ast.TableName), } - stmt.Table.PartitionNames = $4.([]model.CIStr) + stmt.Table.PartitionNames = $4.([]ast.CIStr) if $6 != nil { stmt.Where = $6.(ast.ExprNode) } @@ -11588,9 +11587,9 @@ ShowStmt: stmt := &ast.ShowStmt{ Tp: ast.ShowRegions, Table: $3.(*ast.TableName), - IndexName: model.NewCIStr($6), + IndexName: ast.NewCIStr($6), } - stmt.Table.PartitionNames = $4.([]model.CIStr) + stmt.Table.PartitionNames = $4.([]ast.CIStr) if $8 != nil { stmt.Where = $8.(ast.ExprNode) } @@ -11721,7 +11720,7 @@ ShowPlacementTarget: $$ = &ast.ShowStmt{ Tp: ast.ShowPlacementForPartition, Table: $2.(*ast.TableName), - Partition: model.NewCIStr($4), + Partition: ast.NewCIStr($4), } } @@ -11859,7 +11858,7 @@ ShowTargetFilterable: { show := &ast.ShowStmt{ Tp: ast.ShowIndex, - Table: &ast.TableName{Name: model.NewCIStr($3), Schema: model.NewCIStr($5)}, + Table: &ast.TableName{Name: ast.NewCIStr($3), Schema: ast.NewCIStr($5)}, } $$ = show } @@ -11977,11 +11976,11 @@ ShowTargetFilterable: } | "STATS_META" { - $$ = &ast.ShowStmt{Tp: ast.ShowStatsMeta, Table: &ast.TableName{Name: model.NewCIStr("STATS_META"), Schema: model.NewCIStr(mysql.SystemDB)}} + $$ = &ast.ShowStmt{Tp: ast.ShowStatsMeta, Table: &ast.TableName{Name: ast.NewCIStr("STATS_META"), Schema: ast.NewCIStr(mysql.SystemDB)}} } | "STATS_HISTOGRAMS" { - $$ = &ast.ShowStmt{Tp: ast.ShowStatsHistograms, Table: &ast.TableName{Name: model.NewCIStr("STATS_HISTOGRAMS"), Schema: model.NewCIStr(mysql.SystemDB)}} + $$ = &ast.ShowStmt{Tp: ast.ShowStatsHistograms, Table: &ast.TableName{Name: ast.NewCIStr("STATS_HISTOGRAMS"), Schema: ast.NewCIStr(mysql.SystemDB)}} } | "STATS_TOPN" { @@ -11989,7 +11988,7 @@ ShowTargetFilterable: } | "STATS_BUCKETS" { - $$ = &ast.ShowStmt{Tp: ast.ShowStatsBuckets, Table: &ast.TableName{Name: model.NewCIStr("STATS_BUCKETS"), Schema: model.NewCIStr(mysql.SystemDB)}} + $$ = &ast.ShowStmt{Tp: ast.ShowStatsBuckets, Table: &ast.TableName{Name: ast.NewCIStr("STATS_BUCKETS"), Schema: ast.NewCIStr(mysql.SystemDB)}} } | "STATS_HEALTHY" { @@ -11997,7 +11996,7 @@ ShowTargetFilterable: } | "STATS_LOCKED" { - $$ = &ast.ShowStmt{Tp: ast.ShowStatsLocked, Table: &ast.TableName{Name: model.NewCIStr("STATS_TABLE_LOCKED"), Schema: model.NewCIStr(mysql.SystemDB)}} + $$ = &ast.ShowStmt{Tp: ast.ShowStatsLocked, Table: &ast.TableName{Name: ast.NewCIStr("STATS_TABLE_LOCKED"), Schema: ast.NewCIStr(mysql.SystemDB)}} } | "HISTOGRAMS_IN_FLIGHT" { @@ -12447,14 +12446,14 @@ ConstraintVectorIndex: if c.Option == nil { c.Option = &ast.IndexOption{} } - c.Option.Tp = indexType.(model.IndexType) + c.Option.Tp = indexType.(ast.IndexType) } if c.Option == nil { - c.Option = &ast.IndexOption{Tp: model.IndexTypeHNSW} - } else if c.Option.Tp == model.IndexTypeInvalid { - c.Option.Tp = model.IndexTypeHNSW + c.Option = &ast.IndexOption{Tp: ast.IndexTypeHNSW} + } else if c.Option.Tp == ast.IndexTypeInvalid { + c.Option.Tp = ast.IndexTypeHNSW } - if c.Option.Tp != model.IndexTypeHNSW { + if c.Option.Tp != ast.IndexTypeHNSW { yylex.AppendError(ErrSyntax) return 1 } @@ -12716,7 +12715,7 @@ TableOption: { $$ = &ast.TableOption{ Tp: ast.TableOptionTTL, - ColumnName: &ast.ColumnName{Name: model.NewCIStr($3)}, + ColumnName: &ast.ColumnName{Name: ast.NewCIStr($3)}, Value: ast.NewValueExpr($6, parser.charset, parser.collation), TimeUnitValue: &ast.TimeUnitExpr{Unit: $7.(ast.TimeUnitType)}, } @@ -13723,7 +13722,7 @@ AlterRangeStmt: "ALTER" "RANGE" Identifier PlacementPolicyOption { option := $4.(*ast.PlacementOption) - $$ = &ast.AlterRangeStmt{RangeName: model.NewCIStr($3), PlacementOption: option} + $$ = &ast.AlterRangeStmt{RangeName: ast.NewCIStr($3), PlacementOption: option} } InstanceOption: @@ -15151,26 +15150,26 @@ TableLock: { $$ = ast.TableLock{ Table: $1.(*ast.TableName), - Type: $2.(model.TableLockType), + Type: $2.(ast.TableLockType), } } LockType: "READ" { - $$ = model.TableLockRead + $$ = ast.TableLockRead } | "READ" "LOCAL" { - $$ = model.TableLockReadLocal + $$ = ast.TableLockReadLocal } | "WRITE" { - $$ = model.TableLockWrite + $$ = ast.TableLockWrite } | "WRITE" "LOCAL" { - $$ = model.TableLockWriteLocal + $$ = ast.TableLockWriteLocal } TableLockList: @@ -15306,7 +15305,7 @@ LockStatsStmt: | "LOCK" "STATS" TableName "PARTITION" PartitionNameList { x := $3.(*ast.TableName) - x.PartitionNames = $5.([]model.CIStr) + x.PartitionNames = $5.([]ast.CIStr) $$ = &ast.LockStatsStmt{ Tables: []*ast.TableName{x}, } @@ -15314,7 +15313,7 @@ LockStatsStmt: | "LOCK" "STATS" TableName "PARTITION" '(' PartitionNameList ')' { x := $3.(*ast.TableName) - x.PartitionNames = $6.([]model.CIStr) + x.PartitionNames = $6.([]ast.CIStr) $$ = &ast.LockStatsStmt{ Tables: []*ast.TableName{x}, } @@ -15330,7 +15329,7 @@ UnlockStatsStmt: | "UNLOCK" "STATS" TableName "PARTITION" PartitionNameList { x := $3.(*ast.TableName) - x.PartitionNames = $5.([]model.CIStr) + x.PartitionNames = $5.([]ast.CIStr) $$ = &ast.UnlockStatsStmt{ Tables: []*ast.TableName{x}, } @@ -15338,7 +15337,7 @@ UnlockStatsStmt: | "UNLOCK" "STATS" TableName "PARTITION" '(' PartitionNameList ')' { x := $3.(*ast.TableName) - x.PartitionNames = $6.([]model.CIStr) + x.PartitionNames = $6.([]ast.CIStr) $$ = &ast.UnlockStatsStmt{ Tables: []*ast.TableName{x}, } @@ -15349,7 +15348,7 @@ DropPolicyStmt: { $$ = &ast.DropPlacementPolicyStmt{ IfExists: $4.(bool), - PolicyName: model.NewCIStr($5), + PolicyName: ast.NewCIStr($5), } } @@ -15358,7 +15357,7 @@ CreateResourceGroupStmt: { $$ = &ast.CreateResourceGroupStmt{ IfNotExists: $4.(bool), - ResourceGroupName: model.NewCIStr($5), + ResourceGroupName: ast.NewCIStr($5), ResourceGroupOptionList: $6.([]*ast.ResourceGroupOption), } } @@ -15368,7 +15367,7 @@ AlterResourceGroupStmt: { $$ = &ast.AlterResourceGroupStmt{ IfExists: $4.(bool), - ResourceGroupName: model.NewCIStr($5), + ResourceGroupName: ast.NewCIStr($5), ResourceGroupOptionList: $6.([]*ast.ResourceGroupOption), } } @@ -15378,7 +15377,7 @@ DropResourceGroupStmt: { $$ = &ast.DropResourceGroupStmt{ IfExists: $4.(bool), - ResourceGroupName: model.NewCIStr($5), + ResourceGroupName: ast.NewCIStr($5), } } @@ -15388,7 +15387,7 @@ CreatePolicyStmt: $$ = &ast.CreatePlacementPolicyStmt{ OrReplace: $2.(bool), IfNotExists: $5.(bool), - PolicyName: model.NewCIStr($6), + PolicyName: ast.NewCIStr($6), PlacementOptions: $7.([]*ast.PlacementOption), } } @@ -15398,7 +15397,7 @@ AlterPolicyStmt: { $$ = &ast.AlterPlacementPolicyStmt{ IfExists: $4.(bool), - PolicyName: model.NewCIStr($5), + PolicyName: ast.NewCIStr($5), PlacementOptions: $6.([]*ast.PlacementOption), } } @@ -16637,7 +16636,7 @@ QueryWatchOption: $$ = &ast.QueryWatchOption{ Tp: ast.QueryWatchResourceGroup, ResourceGroupOption: &ast.QueryWatchResourceGroupOption{ - GroupNameStr: model.NewCIStr($3), + GroupNameStr: ast.NewCIStr($3), }, } } @@ -16669,21 +16668,21 @@ QueryWatchTextOption: "SQL" "DIGEST" SimpleExpr { $$ = &ast.QueryWatchTextOption{ - Type: model.WatchSimilar, + Type: ast.WatchSimilar, PatternExpr: $3, } } | "PLAN" "DIGEST" SimpleExpr { $$ = &ast.QueryWatchTextOption{ - Type: model.WatchPlan, + Type: ast.WatchPlan, PatternExpr: $3, } } | "SQL" "TEXT" ResourceGroupRunawayWatchOption "TO" SimpleExpr { $$ = &ast.QueryWatchTextOption{ - Type: $3.(model.RunawayWatchType), + Type: $3.(ast.RunawayWatchType), PatternExpr: $5, TypeSpecified: true, } diff --git a/pkg/parser/parser_test.go b/pkg/parser/parser_test.go index 70ef142424913..a979cabf31e1f 100644 --- a/pkg/parser/parser_test.go +++ b/pkg/parser/parser_test.go @@ -25,7 +25,6 @@ import ( "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/charset" . "github.com/pingcap/tidb/pkg/parser/format" - "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/opcode" "github.com/pingcap/tidb/pkg/parser/terror" @@ -4579,12 +4578,12 @@ func TestOptimizerHints(t *testing.T) { hints = selectStmt.TableHints require.Len(t, hints, 2) require.Equal(t, "read_from_storage", hints[0].HintName.L) - require.Equal(t, "tiflash", hints[0].HintData.(model.CIStr).L) + require.Equal(t, "tiflash", hints[0].HintData.(ast.CIStr).L) require.Len(t, hints[0].Tables, 2) require.Equal(t, "t1", hints[0].Tables[0].TableName.L) require.Equal(t, "t2", hints[0].Tables[1].TableName.L) require.Equal(t, "read_from_storage", hints[1].HintName.L) - require.Equal(t, "tikv", hints[1].HintData.(model.CIStr).L) + require.Equal(t, "tikv", hints[1].HintData.(ast.CIStr).L) require.Len(t, hints[1].Tables, 1) require.Equal(t, "t3", hints[1].Tables[0].TableName.L) @@ -4657,9 +4656,9 @@ func TestOptimizerHints(t *testing.T) { hints = selectStmt.TableHints require.Len(t, hints, 2) require.Equal(t, "query_type", hints[0].HintName.L) - require.Equal(t, "olap", hints[0].HintData.(model.CIStr).L) + require.Equal(t, "olap", hints[0].HintData.(ast.CIStr).L) require.Equal(t, "query_type", hints[1].HintName.L) - require.Equal(t, "oltp", hints[1].HintData.(model.CIStr).L) + require.Equal(t, "oltp", hints[1].HintData.(ast.CIStr).L) // Test MEMORY_QUOTA stmt, _, err = p.Parse("select /*+ MEMORY_QUOTA(1 MB), memory_quota(1 GB) */ c1, c2 from t1, t2 where t1.c1 = t2.c1", "", "") @@ -5822,10 +5821,10 @@ func TestView(t *testing.T) { require.NoError(t, err) v, ok := sms[0].(*ast.CreateViewStmt) require.True(t, ok) - require.Equal(t, model.AlgorithmUndefined, v.Algorithm) + require.Equal(t, ast.AlgorithmUndefined, v.Algorithm) require.Equal(t, "select * from t", v.Select.Text()) - require.Equal(t, model.SecurityDefiner, v.Security) - require.Equal(t, model.CheckOptionCascaded, v.CheckOption) + require.Equal(t, ast.SecurityDefiner, v.Security) + require.Equal(t, ast.CheckOptionCascaded, v.CheckOption) src := `CREATE OR REPLACE ALGORITHM = UNDEFINED DEFINER = root@localhost SQL SECURITY DEFINER @@ -5838,15 +5837,15 @@ func TestView(t *testing.T) { v, ok = st.(*ast.CreateViewStmt) require.True(t, ok) require.True(t, v.OrReplace) - require.Equal(t, model.AlgorithmUndefined, v.Algorithm) + require.Equal(t, ast.AlgorithmUndefined, v.Algorithm) require.Equal(t, "root", v.Definer.Username) require.Equal(t, "localhost", v.Definer.Hostname) - require.Equal(t, model.NewCIStr("a"), v.Cols[0]) - require.Equal(t, model.NewCIStr("b"), v.Cols[1]) - require.Equal(t, model.NewCIStr("c"), v.Cols[2]) + require.Equal(t, ast.NewCIStr("a"), v.Cols[0]) + require.Equal(t, ast.NewCIStr("b"), v.Cols[1]) + require.Equal(t, ast.NewCIStr("c"), v.Cols[2]) require.Equal(t, "select c,d,e from t", v.Select.Text()) - require.Equal(t, model.SecurityDefiner, v.Security) - require.Equal(t, model.CheckOptionCascaded, v.CheckOption) + require.Equal(t, ast.SecurityDefiner, v.Security) + require.Equal(t, ast.CheckOptionCascaded, v.CheckOption) src = ` CREATE VIEW v1 AS SELECT * FROM t; @@ -6459,8 +6458,8 @@ func TestTablePartitionNameList(t *testing.T) { tableName, ok := source.Source.(*ast.TableName) require.True(t, ok) require.Len(t, tableName.PartitionNames, 2) - require.Equal(t, model.CIStr{O: "p0", L: "p0"}, tableName.PartitionNames[0]) - require.Equal(t, model.CIStr{O: "p1", L: "p1"}, tableName.PartitionNames[1]) + require.Equal(t, ast.CIStr{O: "p0", L: "p0"}, tableName.PartitionNames[0]) + require.Equal(t, ast.CIStr{O: "p1", L: "p1"}, tableName.PartitionNames[1]) } } @@ -7062,11 +7061,11 @@ func TestStatisticsOps(t *testing.T) { require.True(t, v.IfNotExists) require.Equal(t, "stats1", v.StatsName) require.Equal(t, ast.StatsTypeCardinality, v.StatsType) - require.Equal(t, model.CIStr{O: "t", L: "t"}, v.Table.Name) + require.Equal(t, ast.CIStr{O: "t", L: "t"}, v.Table.Name) require.Len(t, v.Columns, 3) - require.Equal(t, model.CIStr{O: "a", L: "a"}, v.Columns[0].Name) - require.Equal(t, model.CIStr{O: "b", L: "b"}, v.Columns[1].Name) - require.Equal(t, model.CIStr{O: "c", L: "c"}, v.Columns[2].Name) + require.Equal(t, ast.CIStr{O: "a", L: "a"}, v.Columns[0].Name) + require.Equal(t, ast.CIStr{O: "b", L: "b"}, v.Columns[1].Name) + require.Equal(t, ast.CIStr{O: "c", L: "c"}, v.Columns[2].Name) } func TestHighNotPrecedenceMode(t *testing.T) { diff --git a/pkg/planner/BUILD.bazel b/pkg/planner/BUILD.bazel index d3c7b9233f010..46c1f4afae915 100644 --- a/pkg/planner/BUILD.bazel +++ b/pkg/planner/BUILD.bazel @@ -11,7 +11,6 @@ go_library( "//pkg/infoschema", "//pkg/kv", "//pkg/parser/ast", - "//pkg/parser/model", "//pkg/planner/core", "//pkg/planner/core/base", "//pkg/planner/core/resolve", diff --git a/pkg/planner/cardinality/BUILD.bazel b/pkg/planner/cardinality/BUILD.bazel index bebf6b8dccfd2..3a4a955203806 100644 --- a/pkg/planner/cardinality/BUILD.bazel +++ b/pkg/planner/cardinality/BUILD.bazel @@ -69,7 +69,7 @@ go_test( "//pkg/kv", "//pkg/meta/model", "//pkg/parser", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/parser/mysql", "//pkg/planner/core", "//pkg/planner/core/base", diff --git a/pkg/planner/cardinality/row_size_test.go b/pkg/planner/cardinality/row_size_test.go index 76d4350cee7bb..70bcdefc9b450 100644 --- a/pkg/planner/cardinality/row_size_test.go +++ b/pkg/planner/cardinality/row_size_test.go @@ -20,7 +20,7 @@ import ( "testing" "unsafe" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/planner/cardinality" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/types" @@ -36,7 +36,7 @@ func TestAvgColLen(t *testing.T) { testKit.MustExec("analyze table t") do := dom is := do.InfoSchema() - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo := tbl.Meta() statsTbl := do.StatsHandle().GetTableStats(tableInfo) diff --git a/pkg/planner/cardinality/selectivity_test.go b/pkg/planner/cardinality/selectivity_test.go index e645f3d863e79..290fdbda8b320 100644 --- a/pkg/planner/cardinality/selectivity_test.go +++ b/pkg/planner/cardinality/selectivity_test.go @@ -29,7 +29,7 @@ import ( "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/cardinality" plannercore "github.com/pingcap/tidb/pkg/planner/core" @@ -129,7 +129,7 @@ func TestOutOfRangeEstimation(t *testing.T) { testKit.MustExec("analyze table t with 2000 samples") h := dom.StatsHandle() - table, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + table, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) statsTbl := h.GetTableStats(table.Meta()) sctx := mock.NewContext() @@ -233,7 +233,7 @@ func TestEstimationForUnknownValues(t *testing.T) { } require.Nil(t, h.DumpStatsDeltaToKV(true)) require.Nil(t, h.Update(context.Background(), dom.InfoSchema())) - table, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + table, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) statsTbl := h.GetTableStats(table.Meta()) @@ -263,7 +263,7 @@ func TestEstimationForUnknownValues(t *testing.T) { testKit.MustExec("truncate table t") testKit.MustExec("insert into t values (null, null)") testKit.MustExec("analyze table t") - table, err = dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + table, err = dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) statsTbl = h.GetTableStats(table.Meta()) @@ -276,7 +276,7 @@ func TestEstimationForUnknownValues(t *testing.T) { testKit.MustExec("create table t(a int, b int, index idx(b))") testKit.MustExec("insert into t values (1,1)") testKit.MustExec("analyze table t") - table, err = dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + table, err = dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) statsTbl = h.GetTableStats(table.Meta()) @@ -311,7 +311,7 @@ func TestEstimationForUnknownValuesAfterModify(t *testing.T) { h := dom.StatsHandle() require.Nil(t, h.DumpStatsDeltaToKV(true)) - table, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + table, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) statsTbl := h.GetTableStats(table.Meta()) @@ -371,7 +371,7 @@ func TestEstimationUniqueKeyEqualConds(t *testing.T) { testKit.MustExec("create table t(a int, b int, c int, unique key(b))") testKit.MustExec("insert into t values (1,1,1),(2,2,2),(3,3,3),(4,4,4),(5,5,5),(6,6,6),(7,7,7)") testKit.MustExec("analyze table t all columns with 4 cmsketch width, 1 cmsketch depth;") - table, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + table, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) statsTbl := dom.StatsHandle().GetTableStats(table.Meta()) @@ -561,7 +561,7 @@ func TestDNFCondSelectivity(t *testing.T) { ctx := context.Background() h := dom.StatsHandle() - tb, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tb, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tblInfo := tb.Meta() statsTbl := h.GetTableStats(tblInfo) @@ -673,7 +673,7 @@ func TestSmallRangeEstimation(t *testing.T) { testKit.MustExec("analyze table t with 0 topn") h := dom.StatsHandle() - table, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + table, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) statsTbl := h.GetTableStats(table.Meta()) sctx := mock.NewContext() @@ -757,7 +757,7 @@ func prepareSelectivity(testKit *testkit.TestKit, dom *domain.Domain) (*statisti testKit.MustExec("create table t(a int primary key, b int, c int, d int, e int, index idx_cd(c, d), index idx_de(d, e))") is := dom.InfoSchema() - tb, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tb, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) if err != nil { return nil, err } @@ -982,7 +982,7 @@ func TestIssue39593(t *testing.T) { testKit.MustExec("drop table if exists t") testKit.MustExec("create table t(a int, b int, index idx(a, b))") is := dom.InfoSchema() - tb, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tb, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tblInfo := tb.Meta() @@ -1033,7 +1033,7 @@ func TestIndexJoinInnerRowCountUpperBound(t *testing.T) { err := statstestutil.HandleNextDDLEventWithTxn(h) require.NoError(t, err) is := dom.InfoSchema() - tb, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tb, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tblInfo := tb.Meta() @@ -1105,7 +1105,7 @@ func TestOrderingIdxSelectivityThreshold(t *testing.T) { err := statstestutil.HandleNextDDLEventWithTxn(h) require.NoError(t, err) is := dom.InfoSchema() - tb, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tb, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tblInfo := tb.Meta() @@ -1189,7 +1189,7 @@ func TestOrderingIdxSelectivityRatio(t *testing.T) { err := statstestutil.HandleNextDDLEventWithTxn(h) require.NoError(t, err) is := dom.InfoSchema() - tb, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tb, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tblInfo := tb.Meta() @@ -1431,7 +1431,7 @@ func TestBuiltinInEstWithoutStats(t *testing.T) { tk.MustQuery("explain format='brief' select * from t where a in (1, 2, 3, 4, 5, 6, 7, 8)").Check(expectedA) tk.MustQuery("explain format='brief' select * from t where b in (1, 2, 3, 4, 5, 6, 7, 8)").Check(expectedB) require.NoError(t, h.Update(context.Background(), is)) - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) statsTbl, found := h.Get(tbl.Meta().ID) require.True(t, found) diff --git a/pkg/planner/cardinality/trace_test.go b/pkg/planner/cardinality/trace_test.go index 1b8a4ac370f72..c07c5857d5386 100644 --- a/pkg/planner/cardinality/trace_test.go +++ b/pkg/planner/cardinality/trace_test.go @@ -28,7 +28,7 @@ import ( "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/planner/cardinality" plannercore "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/planner/core/base" @@ -194,7 +194,7 @@ func TestTraceDebugSelectivity(t *testing.T) { require.NoError(t, err) sctx := tk.Session().(sessionctx.Context) - tb, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tb, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tblInfo := tb.Meta() statsTbl := statsHandle.GetTableStats(tblInfo) diff --git a/pkg/planner/cascades/memo/BUILD.bazel b/pkg/planner/cascades/memo/BUILD.bazel index 61414bcd21fea..0801ff9008644 100644 --- a/pkg/planner/cascades/memo/BUILD.bazel +++ b/pkg/planner/cascades/memo/BUILD.bazel @@ -38,7 +38,7 @@ go_test( shard_count = 12, deps = [ "//pkg/expression", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/planner/cascades/base", "//pkg/planner/core/operator/logicalop", "//pkg/planner/util", diff --git a/pkg/planner/cascades/memo/memo_test.go b/pkg/planner/cascades/memo/memo_test.go index 2985bab2298a4..fe893c299cb3a 100644 --- a/pkg/planner/cascades/memo/memo_test.go +++ b/pkg/planner/cascades/memo/memo_test.go @@ -21,7 +21,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb/pkg/expression" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/planner/core/operator/logicalop" "github.com/pingcap/tidb/pkg/planner/util" "github.com/pingcap/tidb/pkg/util/mock" @@ -358,15 +358,15 @@ func TestIteratorLogicalPlan(t *testing.T) { require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/planner/cascades/memo/MockPlanSkipMemoDeriveStats")) }() ctx := mock.NewContext() - asT1 := pmodel.NewCIStr("t1") - asT2 := pmodel.NewCIStr("t2") + asT1 := ast.NewCIStr("t1") + asT2 := ast.NewCIStr("t2") t1 := logicalop.DataSource{TableAsName: &asT1}.Init(ctx, 0) t2 := logicalop.DataSource{TableAsName: &asT2}.Init(ctx, 0) join1 := logicalop.LogicalJoin{}.Init(ctx, 0) join1.SetChildren(t1, t2) - asT3 := pmodel.NewCIStr("t3") - asT4 := pmodel.NewCIStr("t4") + asT3 := ast.NewCIStr("t3") + asT4 := ast.NewCIStr("t4") t3 := logicalop.DataSource{TableAsName: &asT3}.Init(ctx, 0) t4 := logicalop.DataSource{TableAsName: &asT4}.Init(ctx, 0) diff --git a/pkg/planner/cascades/rule/BUILD.bazel b/pkg/planner/cascades/rule/BUILD.bazel index 3b03895534728..7cb5e7cdc923f 100644 --- a/pkg/planner/cascades/rule/BUILD.bazel +++ b/pkg/planner/cascades/rule/BUILD.bazel @@ -25,7 +25,7 @@ go_test( flaky = True, shard_count = 8, deps = [ - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/planner/cascades/memo", "//pkg/planner/cascades/pattern", "//pkg/planner/cascades/util", diff --git a/pkg/planner/cascades/rule/binder_test.go b/pkg/planner/cascades/rule/binder_test.go index 56d3290b2507d..a24a3c936378c 100644 --- a/pkg/planner/cascades/rule/binder_test.go +++ b/pkg/planner/cascades/rule/binder_test.go @@ -19,7 +19,7 @@ import ( "testing" "github.com/pingcap/failpoint" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/planner/cascades/memo" "github.com/pingcap/tidb/pkg/planner/cascades/pattern" "github.com/pingcap/tidb/pkg/planner/cascades/util" @@ -226,15 +226,15 @@ func TestBinderMultiNext(t *testing.T) { require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/planner/cascades/memo/MockPlanSkipMemoDeriveStats")) }() ctx := mock.NewContext() - asT1 := pmodel.NewCIStr("t1") - asT2 := pmodel.NewCIStr("t2") + asT1 := ast.NewCIStr("t1") + asT2 := ast.NewCIStr("t2") t1 := logicalop.DataSource{TableAsName: &asT1}.Init(ctx, 0) t2 := logicalop.DataSource{TableAsName: &asT2}.Init(ctx, 0) join1 := logicalop.LogicalJoin{}.Init(ctx, 0) join1.SetChildren(t1, t2) - asT3 := pmodel.NewCIStr("t3") - asT4 := pmodel.NewCIStr("t4") + asT3 := ast.NewCIStr("t3") + asT4 := ast.NewCIStr("t4") t3 := logicalop.DataSource{TableAsName: &asT3}.Init(ctx, 0) t4 := logicalop.DataSource{TableAsName: &asT4}.Init(ctx, 0) @@ -328,15 +328,15 @@ func TestBinderAny(t *testing.T) { require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/planner/cascades/memo/MockPlanSkipMemoDeriveStats")) }() ctx := mock.NewContext() - asT1 := pmodel.NewCIStr("t1") - asT2 := pmodel.NewCIStr("t2") + asT1 := ast.NewCIStr("t1") + asT2 := ast.NewCIStr("t2") t1 := logicalop.DataSource{TableAsName: &asT1}.Init(ctx, 0) t2 := logicalop.DataSource{TableAsName: &asT2}.Init(ctx, 0) join1 := logicalop.LogicalJoin{}.Init(ctx, 0) join1.SetChildren(t1, t2) - asT3 := pmodel.NewCIStr("t3") - asT4 := pmodel.NewCIStr("t4") + asT3 := ast.NewCIStr("t3") + asT4 := ast.NewCIStr("t4") t3 := logicalop.DataSource{TableAsName: &asT3}.Init(ctx, 0) t4 := logicalop.DataSource{TableAsName: &asT4}.Init(ctx, 0) @@ -416,15 +416,15 @@ func TestBinderMultiAny(t *testing.T) { require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/planner/cascades/memo/MockPlanSkipMemoDeriveStats")) }() ctx := mock.NewContext() - asT1 := pmodel.NewCIStr("t1") - asT2 := pmodel.NewCIStr("t2") + asT1 := ast.NewCIStr("t1") + asT2 := ast.NewCIStr("t2") t1 := logicalop.DataSource{TableAsName: &asT1}.Init(ctx, 0) t2 := logicalop.DataSource{TableAsName: &asT2}.Init(ctx, 0) join1 := logicalop.LogicalJoin{}.Init(ctx, 0) join1.SetChildren(t1, t2) - asT3 := pmodel.NewCIStr("t3") - asT4 := pmodel.NewCIStr("t4") + asT3 := ast.NewCIStr("t3") + asT4 := ast.NewCIStr("t4") t3 := logicalop.DataSource{TableAsName: &asT3}.Init(ctx, 0) t4 := logicalop.DataSource{TableAsName: &asT4}.Init(ctx, 0) diff --git a/pkg/planner/core/BUILD.bazel b/pkg/planner/core/BUILD.bazel index 025d047da8ae8..9e5b65f0fa0f7 100644 --- a/pkg/planner/core/BUILD.bazel +++ b/pkg/planner/core/BUILD.bazel @@ -114,7 +114,6 @@ go_library( "//pkg/parser/auth", "//pkg/parser/charset", "//pkg/parser/format", - "//pkg/parser/model", "//pkg/parser/mysql", "//pkg/parser/opcode", "//pkg/parser/terror", @@ -281,7 +280,6 @@ go_test( "//pkg/parser/auth", "//pkg/parser/charset", "//pkg/parser/format", - "//pkg/parser/model", "//pkg/parser/mysql", "//pkg/parser/terror", "//pkg/planner", diff --git a/pkg/planner/core/casetest/BUILD.bazel b/pkg/planner/core/casetest/BUILD.bazel index 79d028e6a7dd5..1ecd84f0b43fb 100644 --- a/pkg/planner/core/casetest/BUILD.bazel +++ b/pkg/planner/core/casetest/BUILD.bazel @@ -18,7 +18,7 @@ go_test( "//pkg/errno", "//pkg/meta/model", "//pkg/parser", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/planner/core", "//pkg/planner/core/base", "//pkg/planner/core/operator/logicalop", diff --git a/pkg/planner/core/casetest/cbotest/BUILD.bazel b/pkg/planner/core/casetest/cbotest/BUILD.bazel index 255649ae34d83..c9f523e77c84e 100644 --- a/pkg/planner/core/casetest/cbotest/BUILD.bazel +++ b/pkg/planner/core/casetest/cbotest/BUILD.bazel @@ -15,7 +15,7 @@ go_test( "//pkg/domain", "//pkg/executor", "//pkg/meta/model", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/planner", "//pkg/planner/core", "//pkg/planner/core/resolve", diff --git a/pkg/planner/core/casetest/cbotest/cbo_test.go b/pkg/planner/core/casetest/cbotest/cbo_test.go index e66e9cf5a1cc0..5f9b6baa847e7 100644 --- a/pkg/planner/core/casetest/cbotest/cbo_test.go +++ b/pkg/planner/core/casetest/cbotest/cbo_test.go @@ -25,7 +25,7 @@ import ( "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/executor" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/planner" "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/planner/core/resolve" @@ -566,7 +566,7 @@ func TestTiFlashCostModel(t *testing.T) { tk.MustExec("create table t (a int, b int, c int, primary key(a))") tk.MustExec("insert into t values(1,1,1), (2,2,2), (3,3,3)") - tbl, err := dom.InfoSchema().TableByName(context.Background(), pmodel.CIStr{O: "test", L: "test"}, pmodel.CIStr{O: "t", L: "t"}) + tbl, err := dom.InfoSchema().TableByName(context.Background(), ast.CIStr{O: "test", L: "test"}, ast.CIStr{O: "t", L: "t"}) require.NoError(t, err) // Set the hacked TiFlash replica for explain tests. tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{Count: 1, Available: true} diff --git a/pkg/planner/core/casetest/enforcempp/BUILD.bazel b/pkg/planner/core/casetest/enforcempp/BUILD.bazel index 7a314567e73b7..132a2fa516949 100644 --- a/pkg/planner/core/casetest/enforcempp/BUILD.bazel +++ b/pkg/planner/core/casetest/enforcempp/BUILD.bazel @@ -13,7 +13,7 @@ go_test( deps = [ "//pkg/domain", "//pkg/meta/model", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/store/mockstore", "//pkg/testkit", "//pkg/testkit/external", diff --git a/pkg/planner/core/casetest/enforcempp/enforce_mpp_test.go b/pkg/planner/core/casetest/enforcempp/enforce_mpp_test.go index 623ea33589e6a..5e93702e2b8c0 100644 --- a/pkg/planner/core/casetest/enforcempp/enforce_mpp_test.go +++ b/pkg/planner/core/casetest/enforcempp/enforce_mpp_test.go @@ -22,7 +22,7 @@ import ( "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/store/mockstore" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/testkit/external" @@ -53,7 +53,7 @@ func TestEnforceMPP(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - db, exists := is.SchemaByName(pmodel.NewCIStr("test")) + db, exists := is.SchemaByName(ast.NewCIStr("test")) require.True(t, exists) testkit.SetTiFlashReplica(t, dom, db.Name.L, "t") testkit.SetTiFlashReplica(t, dom, db.Name.L, "s") @@ -131,7 +131,7 @@ func TestEnforceMPPWarning1(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - tblInfo, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tblInfo, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tblInfo.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ Count: 1, @@ -170,7 +170,7 @@ func TestEnforceMPPWarning2(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ Count: 1, @@ -218,7 +218,7 @@ func TestEnforceMPPWarning3(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ Count: 1, @@ -375,7 +375,7 @@ func TestMPPSkewedGroupDistinctRewrite(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ Count: 1, @@ -423,7 +423,7 @@ func TestMPPSingleDistinct3Stage(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ Count: 1, diff --git a/pkg/planner/core/casetest/hint/BUILD.bazel b/pkg/planner/core/casetest/hint/BUILD.bazel index 6c42d940fdf98..07eb2699906f4 100644 --- a/pkg/planner/core/casetest/hint/BUILD.bazel +++ b/pkg/planner/core/casetest/hint/BUILD.bazel @@ -14,7 +14,7 @@ go_test( "//pkg/config", "//pkg/domain", "//pkg/meta/model", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/sessionctx/variable", "//pkg/store/mockstore", "//pkg/testkit", diff --git a/pkg/planner/core/casetest/hint/hint_test.go b/pkg/planner/core/casetest/hint/hint_test.go index b9e8d1ce90fc1..e374e7fd430fd 100644 --- a/pkg/planner/core/casetest/hint/hint_test.go +++ b/pkg/planner/core/casetest/hint/hint_test.go @@ -20,7 +20,7 @@ import ( "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/store/mockstore" "github.com/pingcap/tidb/pkg/testkit" @@ -88,7 +88,7 @@ func TestAllViewHintType(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ Count: 1, @@ -271,7 +271,7 @@ func TestOptimizeHintOnPartitionTable(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ Count: 1, diff --git a/pkg/planner/core/casetest/index/BUILD.bazel b/pkg/planner/core/casetest/index/BUILD.bazel index e4bb65791f767..2db3ab19a14bd 100644 --- a/pkg/planner/core/casetest/index/BUILD.bazel +++ b/pkg/planner/core/casetest/index/BUILD.bazel @@ -13,7 +13,7 @@ go_test( deps = [ "//pkg/domain", "//pkg/domain/infosync", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/store/mockstore", "//pkg/testkit", "//pkg/testkit/testdata", diff --git a/pkg/planner/core/casetest/index/index_test.go b/pkg/planner/core/casetest/index/index_test.go index d336e302a7b17..c9cc0425956e4 100644 --- a/pkg/planner/core/casetest/index/index_test.go +++ b/pkg/planner/core/casetest/index/index_test.go @@ -22,7 +22,7 @@ import ( "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/domain/infosync" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/store/mockstore" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/testkit/testdata" @@ -285,7 +285,7 @@ func TestAnalyzeVectorIndex(t *testing.T) { tk.MustExec(`create table t(a int, b vector(2), c vector(3), j json, index(a))`) tk.MustExec("insert into t values(1, '[1, 0]', '[1, 0, 0]', '{\"a\": 1}')") tk.MustExec("alter table t set tiflash replica 2 location labels 'a','b';") - tbl, err := dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tblInfo := tbl.Meta() err = domain.GetDomain(tk.Session()).DDLExecutor().UpdateTableReplicaInfo(tk.Session(), tblInfo.ID, true) diff --git a/pkg/planner/core/casetest/integration_test.go b/pkg/planner/core/casetest/integration_test.go index 72b00393b6d83..85154d685f4f6 100644 --- a/pkg/planner/core/casetest/integration_test.go +++ b/pkg/planner/core/casetest/integration_test.go @@ -22,7 +22,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" statstestutil "github.com/pingcap/tidb/pkg/statistics/handle/ddl/testutil" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/testkit/testdata" @@ -88,7 +88,7 @@ func TestIsolationReadTiFlashNotChoosePointGet(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - tblInfo, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tblInfo, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tblInfo.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ Count: 1, @@ -176,7 +176,7 @@ func TestIssue31240(t *testing.T) { // since allow-mpp is adjusted to false, there will be no physical plan if TiFlash cop is banned. tk.MustExec("set @@session.tidb_allow_tiflash_cop=ON") - tbl, err := dom.InfoSchema().TableByName(context.Background(), pmodel.CIStr{O: "test", L: "test"}, pmodel.CIStr{O: "t31240", L: "t31240"}) + tbl, err := dom.InfoSchema().TableByName(context.Background(), ast.CIStr{O: "test", L: "test"}, ast.CIStr{O: "t31240", L: "t31240"}) require.NoError(t, err) // Set the hacked TiFlash replica for explain tests. tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{Count: 1, Available: true} @@ -230,9 +230,9 @@ func TestIssue32632(t *testing.T) { require.NoError(t, err) tk.MustExec("set @@tidb_enforce_mpp = 1") - tbl1, err := dom.InfoSchema().TableByName(context.Background(), pmodel.CIStr{O: "test", L: "test"}, pmodel.CIStr{O: "partsupp", L: "partsupp"}) + tbl1, err := dom.InfoSchema().TableByName(context.Background(), ast.CIStr{O: "test", L: "test"}, ast.CIStr{O: "partsupp", L: "partsupp"}) require.NoError(t, err) - tbl2, err := dom.InfoSchema().TableByName(context.Background(), pmodel.CIStr{O: "test", L: "test"}, pmodel.CIStr{O: "supplier", L: "supplier"}) + tbl2, err := dom.InfoSchema().TableByName(context.Background(), ast.CIStr{O: "test", L: "test"}, ast.CIStr{O: "supplier", L: "supplier"}) require.NoError(t, err) // Set the hacked TiFlash replica for explain tests. tbl1.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{Count: 1, Available: true} @@ -276,9 +276,9 @@ func TestTiFlashPartitionTableScan(t *testing.T) { tk.MustExec("drop table if exists hp_t;") tk.MustExec("create table rp_t(a int) partition by RANGE (a) (PARTITION p0 VALUES LESS THAN (6),PARTITION p1 VALUES LESS THAN (11), PARTITION p2 VALUES LESS THAN (16), PARTITION p3 VALUES LESS THAN (21));") tk.MustExec("create table hp_t(a int) partition by hash(a) partitions 4;") - tbl1, err := dom.InfoSchema().TableByName(context.Background(), pmodel.CIStr{O: "test", L: "test"}, pmodel.CIStr{O: "rp_t", L: "rp_t"}) + tbl1, err := dom.InfoSchema().TableByName(context.Background(), ast.CIStr{O: "test", L: "test"}, ast.CIStr{O: "rp_t", L: "rp_t"}) require.NoError(t, err) - tbl2, err := dom.InfoSchema().TableByName(context.Background(), pmodel.CIStr{O: "test", L: "test"}, pmodel.CIStr{O: "hp_t", L: "hp_t"}) + tbl2, err := dom.InfoSchema().TableByName(context.Background(), ast.CIStr{O: "test", L: "test"}, ast.CIStr{O: "hp_t", L: "hp_t"}) require.NoError(t, err) // Set the hacked TiFlash replica for explain tests. tbl1.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{Count: 1, Available: true} @@ -311,7 +311,7 @@ func TestTiFlashFineGrainedShuffle(t *testing.T) { tk.MustExec("drop table if exists t1;") tk.MustExec("create table t1(c1 int, c2 int)") - tbl1, err := dom.InfoSchema().TableByName(context.Background(), pmodel.CIStr{O: "test", L: "test"}, pmodel.CIStr{O: "t1", L: "t1"}) + tbl1, err := dom.InfoSchema().TableByName(context.Background(), ast.CIStr{O: "test", L: "test"}, ast.CIStr{O: "t1", L: "t1"}) require.NoError(t, err) // Set the hacked TiFlash replica for explain tests. tbl1.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{Count: 1, Available: true} @@ -475,7 +475,7 @@ func TestTiFlashExtraColumnPrune(t *testing.T) { tk.MustExec("drop table if exists t1;") tk.MustExec("create table t1(c1 int, c2 int)") - tbl1, err := dom.InfoSchema().TableByName(context.Background(), pmodel.CIStr{O: "test", L: "test"}, pmodel.CIStr{O: "t1", L: "t1"}) + tbl1, err := dom.InfoSchema().TableByName(context.Background(), ast.CIStr{O: "test", L: "test"}, ast.CIStr{O: "t1", L: "t1"}) require.NoError(t, err) // Set the hacked TiFlash replica for explain tests. tbl1.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{Count: 1, Available: true} diff --git a/pkg/planner/core/casetest/mpp/BUILD.bazel b/pkg/planner/core/casetest/mpp/BUILD.bazel index 0a7bcdf11e164..a9c2f26f8372f 100644 --- a/pkg/planner/core/casetest/mpp/BUILD.bazel +++ b/pkg/planner/core/casetest/mpp/BUILD.bazel @@ -14,7 +14,7 @@ go_test( "//pkg/config", "//pkg/domain", "//pkg/meta/model", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/testkit", "//pkg/testkit/testdata", "//pkg/testkit/testmain", diff --git a/pkg/planner/core/casetest/mpp/mpp_test.go b/pkg/planner/core/casetest/mpp/mpp_test.go index 24dbfdb2d30dc..71a75b961acbf 100644 --- a/pkg/planner/core/casetest/mpp/mpp_test.go +++ b/pkg/planner/core/casetest/mpp/mpp_test.go @@ -22,7 +22,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/testkit/testdata" "github.com/stretchr/testify/require" @@ -89,7 +89,7 @@ func TestMPPLeftSemiJoin(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ Count: 1, @@ -613,7 +613,7 @@ func TestMppAggTopNWithJoin(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ Count: 1, @@ -651,7 +651,7 @@ func TestRejectSortForMPP(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ Count: 1, @@ -689,7 +689,7 @@ func TestPushDownSelectionForMPP(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ Count: 1, @@ -728,7 +728,7 @@ func TestPushDownProjectionForMPP(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ Count: 1, @@ -766,7 +766,7 @@ func TestPushDownAggForMPP(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ Count: 1, @@ -803,7 +803,7 @@ func TestMppVersion(t *testing.T) { // Create virtual tiflash replica info. is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ Count: 1, diff --git a/pkg/planner/core/casetest/plan_test.go b/pkg/planner/core/casetest/plan_test.go index 6e7478a8661f1..84c8a58263f72 100644 --- a/pkg/planner/core/casetest/plan_test.go +++ b/pkg/planner/core/casetest/plan_test.go @@ -22,7 +22,7 @@ import ( "github.com/pingcap/tidb/pkg/errno" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/testkit" @@ -240,7 +240,7 @@ func TestNormalizedPlanForDiffStore(t *testing.T) { tk.MustExec("drop table if exists t1") tk.MustExec("create table t1 (a int, b int, c int, primary key(a))") tk.MustExec("insert into t1 values(1,1,1), (2,2,2), (3,3,3)") - tbl, err := dom.InfoSchema().TableByName(context.Background(), pmodel.CIStr{O: "test", L: "test"}, pmodel.CIStr{O: "t1", L: "t1"}) + tbl, err := dom.InfoSchema().TableByName(context.Background(), ast.CIStr{O: "test", L: "test"}, ast.CIStr{O: "t1", L: "t1"}) require.NoError(t, err) // Set the hacked TiFlash replica for explain tests. tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{Count: 1, Available: true} diff --git a/pkg/planner/core/casetest/planstats/BUILD.bazel b/pkg/planner/core/casetest/planstats/BUILD.bazel index 2244bb71edf61..23ca92a70e88e 100644 --- a/pkg/planner/core/casetest/planstats/BUILD.bazel +++ b/pkg/planner/core/casetest/planstats/BUILD.bazel @@ -16,7 +16,7 @@ go_test( "//pkg/executor", "//pkg/meta/model", "//pkg/parser", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/planner", "//pkg/planner/core", "//pkg/planner/core/base", diff --git a/pkg/planner/core/casetest/planstats/plan_stats_test.go b/pkg/planner/core/casetest/planstats/plan_stats_test.go index 66945d644b08b..ce713e55ad414 100644 --- a/pkg/planner/core/casetest/planstats/plan_stats_test.go +++ b/pkg/planner/core/casetest/planstats/plan_stats_test.go @@ -27,7 +27,7 @@ import ( "github.com/pingcap/tidb/pkg/executor" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/planner" plannercore "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/planner/core/base" @@ -203,7 +203,7 @@ func TestPlanStatsLoad(t *testing.T) { nodeW := resolve.NewNodeW(stmt) p, _, err := planner.Optimize(context.TODO(), ctx, nodeW, is) require.NoError(t, err) - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo := tbl.Meta() testCase.check(p, tableInfo) @@ -296,7 +296,7 @@ func TestPlanStatsLoadTimeout(t *testing.T) { tk.MustExec("analyze table t all columns") is := dom.InfoSchema() require.NoError(t, dom.StatsHandle().Update(context.Background(), is)) - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo := tbl.Meta() neededColumn := model.StatsLoadItem{TableItemID: model.TableItemID{TableID: tableInfo.ID, ID: tableInfo.Columns[0].ID, IsIndex: false}, FullLoad: true} @@ -392,7 +392,7 @@ func TestCollectDependingVirtualCols(t *testing.T) { tblName2TblID := make(map[string]int64) tblID2Tbl := make(map[int64]*model.TableInfo) for _, tblName := range tableNames { - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr(tblName)) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr(tblName)) require.NoError(t, err) tblName2TblID[tblName] = tbl.Meta().ID tblID2Tbl[tbl.Meta().ID] = tbl.Meta() diff --git a/pkg/planner/core/casetest/pushdown/BUILD.bazel b/pkg/planner/core/casetest/pushdown/BUILD.bazel index 34638b9c5ec93..fc92b41445b98 100644 --- a/pkg/planner/core/casetest/pushdown/BUILD.bazel +++ b/pkg/planner/core/casetest/pushdown/BUILD.bazel @@ -13,7 +13,7 @@ go_test( deps = [ "//pkg/domain", "//pkg/meta/model", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/testkit", "//pkg/testkit/testdata", "//pkg/testkit/testmain", diff --git a/pkg/planner/core/casetest/pushdown/push_down_test.go b/pkg/planner/core/casetest/pushdown/push_down_test.go index bdc22b7d7dbb6..afa603afc668d 100644 --- a/pkg/planner/core/casetest/pushdown/push_down_test.go +++ b/pkg/planner/core/casetest/pushdown/push_down_test.go @@ -20,7 +20,7 @@ import ( "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/testkit/testdata" "github.com/stretchr/testify/require" @@ -38,7 +38,7 @@ func TestPushDownToTiFlashWithKeepOrder(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ Count: 1, @@ -78,7 +78,7 @@ func TestPushDownToTiFlashWithKeepOrderInFastMode(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ Count: 1, @@ -151,7 +151,7 @@ func TestPushDownProjectionForTiFlashCoprocessor(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ Count: 1, @@ -187,7 +187,7 @@ func TestSelPushDownTiFlash(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ Count: 1, diff --git a/pkg/planner/core/casetest/vectorsearch/BUILD.bazel b/pkg/planner/core/casetest/vectorsearch/BUILD.bazel index 1a9a81c20ebf5..65461c38f7a9d 100644 --- a/pkg/planner/core/casetest/vectorsearch/BUILD.bazel +++ b/pkg/planner/core/casetest/vectorsearch/BUILD.bazel @@ -15,7 +15,7 @@ go_test( "//pkg/domain", "//pkg/domain/infosync", "//pkg/meta/model", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/planner", "//pkg/planner/core", "//pkg/planner/core/base", diff --git a/pkg/planner/core/casetest/vectorsearch/vector_index_test.go b/pkg/planner/core/casetest/vectorsearch/vector_index_test.go index 5efd71b8dca76..f3ca6143b417c 100644 --- a/pkg/planner/core/casetest/vectorsearch/vector_index_test.go +++ b/pkg/planner/core/casetest/vectorsearch/vector_index_test.go @@ -24,7 +24,7 @@ import ( "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/domain/infosync" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/planner" "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/planner/core/base" @@ -199,7 +199,7 @@ func TestANNIndexNormalizedPlan(t *testing.T) { require.NotEqual(t, d1, dx1) // test for TiFlashReplica's Available - tbl, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tbl.Meta().TiFlashReplica.Available = false tk.MustExec("explain select * from t order by vec_cosine_distance(vec, '[1,2,3]') limit 3") diff --git a/pkg/planner/core/collect_column_stats_usage_test.go b/pkg/planner/core/collect_column_stats_usage_test.go index bf5194ef2789b..7752a072de710 100644 --- a/pkg/planner/core/collect_column_stats_usage_test.go +++ b/pkg/planner/core/collect_column_stats_usage_test.go @@ -23,7 +23,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/core/resolve" "github.com/pingcap/tidb/pkg/planner/core/rule" @@ -36,7 +36,7 @@ func getTblInfoByPhyID(t *testing.T, is infoschema.InfoSchema, physicalTblID int tblInfo := tbl.Meta() return tblInfo, tblInfo.Name.L } - db, exists := is.SchemaByName(pmodel.NewCIStr("test")) + db, exists := is.SchemaByName(ast.NewCIStr("test")) require.True(t, exists) tblInfos, err := is.SchemaTableInfos(context.Background(), db.Name) require.NoError(t, err) diff --git a/pkg/planner/core/common_plans.go b/pkg/planner/core/common_plans.go index 28dce1a4a0d4b..63f7926229b24 100644 --- a/pkg/planner/core/common_plans.go +++ b/pkg/planner/core/common_plans.go @@ -26,7 +26,6 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/ast" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/core/operator/physicalop" @@ -583,7 +582,7 @@ type V2AnalyzeOptions struct { PhyTableID int64 RawOpts map[ast.AnalyzeOptionType]uint64 FilledOpts map[ast.AnalyzeOptionType]uint64 - ColChoice pmodel.ColumnChoice + ColChoice ast.ColumnChoice ColumnList []*model.ColumnInfo IsPartition bool } @@ -712,7 +711,7 @@ type SplitRegion struct { baseSchemaProducer TableInfo *model.TableInfo - PartitionNames []pmodel.CIStr + PartitionNames []ast.CIStr IndexInfo *model.IndexInfo Lower []types.Datum Upper []types.Datum @@ -734,7 +733,7 @@ type CompactTable struct { ReplicaKind ast.CompactReplicaKind TableInfo *model.TableInfo - PartitionNames []pmodel.CIStr + PartitionNames []ast.CIStr } // DDL represents a DDL statement plan. diff --git a/pkg/planner/core/enforce_mpp_test.go b/pkg/planner/core/enforce_mpp_test.go index 92fc684ae3043..d9649cccaddc9 100644 --- a/pkg/planner/core/enforce_mpp_test.go +++ b/pkg/planner/core/enforce_mpp_test.go @@ -22,7 +22,7 @@ import ( "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/testkit" "github.com/stretchr/testify/require" ) @@ -40,7 +40,7 @@ func TestRowSizeInMPP(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - db, exists := is.SchemaByName(pmodel.NewCIStr("test")) + db, exists := is.SchemaByName(ast.NewCIStr("test")) require.True(t, exists) tblInfos, err := is.SchemaTableInfos(context.Background(), db.Name) require.NoError(t, err) diff --git a/pkg/planner/core/exhaust_physical_plans_test.go b/pkg/planner/core/exhaust_physical_plans_test.go index e66a046dc4633..7eee582850433 100644 --- a/pkg/planner/core/exhaust_physical_plans_test.go +++ b/pkg/planner/core/exhaust_physical_plans_test.go @@ -23,7 +23,6 @@ import ( "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/charset" - "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/core/operator/logicalop" @@ -74,45 +73,45 @@ func prepareForAnalyzeLookUpFilters() *indexJoinContext { RetType: types.NewFieldType(mysql.TypeLonglong), }) dsNames = append(dsNames, &types.FieldName{ - ColName: model.NewCIStr("a"), - TblName: model.NewCIStr("t"), - DBName: model.NewCIStr("test"), + ColName: ast.NewCIStr("a"), + TblName: ast.NewCIStr("t"), + DBName: ast.NewCIStr("test"), }) dsSchema.Append(&expression.Column{ UniqueID: ctx.GetSessionVars().AllocPlanColumnID(), RetType: types.NewFieldType(mysql.TypeLonglong), }) dsNames = append(dsNames, &types.FieldName{ - ColName: model.NewCIStr("b"), - TblName: model.NewCIStr("t"), - DBName: model.NewCIStr("test"), + ColName: ast.NewCIStr("b"), + TblName: ast.NewCIStr("t"), + DBName: ast.NewCIStr("test"), }) dsSchema.Append(&expression.Column{ UniqueID: ctx.GetSessionVars().AllocPlanColumnID(), RetType: types.NewFieldTypeWithCollation(mysql.TypeVarchar, mysql.DefaultCollationName, types.UnspecifiedLength), }) dsNames = append(dsNames, &types.FieldName{ - ColName: model.NewCIStr("c"), - TblName: model.NewCIStr("t"), - DBName: model.NewCIStr("test"), + ColName: ast.NewCIStr("c"), + TblName: ast.NewCIStr("t"), + DBName: ast.NewCIStr("test"), }) dsSchema.Append(&expression.Column{ UniqueID: ctx.GetSessionVars().AllocPlanColumnID(), RetType: types.NewFieldType(mysql.TypeLonglong), }) dsNames = append(dsNames, &types.FieldName{ - ColName: model.NewCIStr("d"), - TblName: model.NewCIStr("t"), - DBName: model.NewCIStr("test"), + ColName: ast.NewCIStr("d"), + TblName: ast.NewCIStr("t"), + DBName: ast.NewCIStr("test"), }) dsSchema.Append(&expression.Column{ UniqueID: ctx.GetSessionVars().AllocPlanColumnID(), RetType: types.NewFieldTypeWithCollation(mysql.TypeVarchar, charset.CollationASCII, types.UnspecifiedLength), }) dsNames = append(dsNames, &types.FieldName{ - ColName: model.NewCIStr("c_ascii"), - TblName: model.NewCIStr("t"), - DBName: model.NewCIStr("test"), + ColName: ast.NewCIStr("c_ascii"), + TblName: ast.NewCIStr("t"), + DBName: ast.NewCIStr("test"), }) dataSourceNode.SetSchema(dsSchema) dataSourceNode.SetStats(&property.StatsInfo{StatsVersion: statistics.PseudoVersion}) @@ -127,36 +126,36 @@ func prepareForAnalyzeLookUpFilters() *indexJoinContext { RetType: types.NewFieldType(mysql.TypeLonglong), }) outerChildNames = append(outerChildNames, &types.FieldName{ - ColName: model.NewCIStr("e"), - TblName: model.NewCIStr("t1"), - DBName: model.NewCIStr("test"), + ColName: ast.NewCIStr("e"), + TblName: ast.NewCIStr("t1"), + DBName: ast.NewCIStr("test"), }) outerChildSchema.Append(&expression.Column{ UniqueID: ctx.GetSessionVars().AllocPlanColumnID(), RetType: types.NewFieldType(mysql.TypeLonglong), }) outerChildNames = append(outerChildNames, &types.FieldName{ - ColName: model.NewCIStr("f"), - TblName: model.NewCIStr("t1"), - DBName: model.NewCIStr("test"), + ColName: ast.NewCIStr("f"), + TblName: ast.NewCIStr("t1"), + DBName: ast.NewCIStr("test"), }) outerChildSchema.Append(&expression.Column{ UniqueID: ctx.GetSessionVars().AllocPlanColumnID(), RetType: types.NewFieldTypeWithCollation(mysql.TypeVarchar, mysql.DefaultCollationName, types.UnspecifiedLength), }) outerChildNames = append(outerChildNames, &types.FieldName{ - ColName: model.NewCIStr("g"), - TblName: model.NewCIStr("t1"), - DBName: model.NewCIStr("test"), + ColName: ast.NewCIStr("g"), + TblName: ast.NewCIStr("t1"), + DBName: ast.NewCIStr("test"), }) outerChildSchema.Append(&expression.Column{ UniqueID: ctx.GetSessionVars().AllocPlanColumnID(), RetType: types.NewFieldType(mysql.TypeLonglong), }) outerChildNames = append(outerChildNames, &types.FieldName{ - ColName: model.NewCIStr("h"), - TblName: model.NewCIStr("t1"), - DBName: model.NewCIStr("test"), + ColName: ast.NewCIStr("h"), + TblName: ast.NewCIStr("t1"), + DBName: ast.NewCIStr("test"), }) joinNode.SetSchema(expression.MergeSchema(dsSchema, outerChildSchema)) joinColNames := append(dsNames.Shallow(), outerChildNames...) diff --git a/pkg/planner/core/expression_codec_fn.go b/pkg/planner/core/expression_codec_fn.go index 02d7e0fa91495..37ab4497f82f4 100644 --- a/pkg/planner/core/expression_codec_fn.go +++ b/pkg/planner/core/expression_codec_fn.go @@ -29,7 +29,7 @@ import ( infoschemactx "github.com/pingcap/tidb/pkg/infoschema/context" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/table/tables" @@ -84,7 +84,7 @@ func (h tidbCodecFuncHelper) findCommonOrPartitionedTable( tblName string, ) (table.Table, int64, error) { tblName, partName := h.extractTablePartition(tblName) - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr(dbName), pmodel.NewCIStr(tblName)) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr(dbName), ast.NewCIStr(tblName)) if err != nil { return nil, 0, err } diff --git a/pkg/planner/core/expression_rewriter.go b/pkg/planner/core/expression_rewriter.go index 42306c205d43c..80669d0870933 100644 --- a/pkg/planner/core/expression_rewriter.go +++ b/pkg/planner/core/expression_rewriter.go @@ -29,7 +29,6 @@ import ( "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/charset" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/opcode" "github.com/pingcap/tidb/pkg/planner/core/base" @@ -2574,7 +2573,7 @@ func (er *expressionRewriter) evalDefaultExprWithPlanCtx(planCtx *exprRewriterPl dbName := name.DBName if dbName.O == "" { // if database name is not specified, use current database name - dbName = pmodel.NewCIStr(planCtx.builder.ctx.GetSessionVars().CurrentDB) + dbName = ast.NewCIStr(planCtx.builder.ctx.GetSessionVars().CurrentDB) } if name.OrigTblName.O == "" { // column is evaluated by some expressions, for example: diff --git a/pkg/planner/core/expression_test.go b/pkg/planner/core/expression_test.go index 4d56aa96ff260..67dcaa15c3331 100644 --- a/pkg/planner/core/expression_test.go +++ b/pkg/planner/core/expression_test.go @@ -27,7 +27,6 @@ import ( "github.com/pingcap/tidb/pkg/parser" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/charset" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/testkit/testutil" @@ -386,7 +385,7 @@ func TestBuildExpression(t *testing.T) { tbl := &model.TableInfo{ Columns: []*model.ColumnInfo{ { - Name: pmodel.NewCIStr("id"), + Name: ast.NewCIStr("id"), Offset: 0, State: model.StatePublic, FieldType: *types.NewFieldType(mysql.TypeString), @@ -394,13 +393,13 @@ func TestBuildExpression(t *testing.T) { DefaultValue: "uuid()", }, { - Name: pmodel.NewCIStr("a"), + Name: ast.NewCIStr("a"), Offset: 1, State: model.StatePublic, FieldType: *types.NewFieldType(mysql.TypeLonglong), }, { - Name: pmodel.NewCIStr("b"), + Name: ast.NewCIStr("b"), Offset: 2, State: model.StatePublic, FieldType: *types.NewFieldType(mysql.TypeLonglong), @@ -411,7 +410,7 @@ func TestBuildExpression(t *testing.T) { ctx := exprstatic.NewExprContext() evalCtx := ctx.GetStaticEvalCtx() - cols, names, err := expression.ColumnInfos2ColumnsAndNames(ctx, pmodel.NewCIStr(""), tbl.Name, tbl.Cols(), tbl) + cols, names, err := expression.ColumnInfos2ColumnsAndNames(ctx, ast.NewCIStr(""), tbl.Name, tbl.Cols(), tbl) require.NoError(t, err) schema := expression.NewSchema(cols...) diff --git a/pkg/planner/core/foreign_key.go b/pkg/planner/core/foreign_key.go index c1cdd302c60c0..9add6da5991df 100644 --- a/pkg/planner/core/foreign_key.go +++ b/pkg/planner/core/foreign_key.go @@ -22,7 +22,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/core/operator/physicalop" @@ -37,7 +37,7 @@ type FKCheck struct { ReferredFK *model.ReferredFKInfo Tbl table.Table Idx table.Index - Cols []pmodel.CIStr + Cols []ast.CIStr IdxIsPrimaryKey bool IdxIsExclusive bool @@ -123,9 +123,9 @@ func (f *FKCascade) AccessObject() base.AccessObject { func (f *FKCascade) OperatorInfo(bool) string { switch f.Tp { case FKCascadeOnDelete: - return fmt.Sprintf("foreign_key:%s, on_delete:%s", f.FK.Name, pmodel.ReferOptionType(f.FK.OnDelete).String()) + return fmt.Sprintf("foreign_key:%s, on_delete:%s", f.FK.Name, ast.ReferOptionType(f.FK.OnDelete).String()) case FKCascadeOnUpdate: - return fmt.Sprintf("foreign_key:%s, on_update:%s", f.FK.Name, pmodel.ReferOptionType(f.FK.OnUpdate).String()) + return fmt.Sprintf("foreign_key:%s, on_update:%s", f.FK.Name, ast.ReferOptionType(f.FK.OnUpdate).String()) } return "" } @@ -384,19 +384,19 @@ func buildOnDeleteOrUpdateFKTrigger(ctx base.PlanContext, is infoschema.InfoSche if fk == nil || fk.Version < 1 { return nil, nil, nil } - var fkReferOption pmodel.ReferOptionType + var fkReferOption ast.ReferOptionType if fk.State != model.StatePublic { - fkReferOption = pmodel.ReferOptionRestrict + fkReferOption = ast.ReferOptionRestrict } else { switch tp { case FKCascadeOnDelete: - fkReferOption = pmodel.ReferOptionType(fk.OnDelete) + fkReferOption = ast.ReferOptionType(fk.OnDelete) case FKCascadeOnUpdate: - fkReferOption = pmodel.ReferOptionType(fk.OnUpdate) + fkReferOption = ast.ReferOptionType(fk.OnUpdate) } } switch fkReferOption { - case pmodel.ReferOptionCascade, pmodel.ReferOptionSetNull: + case ast.ReferOptionCascade, ast.ReferOptionSetNull: fkCascade, err := buildFKCascade(ctx, tp, referredFK, childTable, fk) return nil, fkCascade, err default: @@ -405,7 +405,7 @@ func buildOnDeleteOrUpdateFKTrigger(ctx base.PlanContext, is infoschema.InfoSche } } -func isMapContainAnyCols(colsMap map[string]struct{}, cols ...pmodel.CIStr) bool { +func isMapContainAnyCols(colsMap map[string]struct{}, cols ...ast.CIStr) bool { for _, col := range cols { _, exist := colsMap[col.L] if exist { @@ -440,7 +440,7 @@ func buildFKCheckForReferredFK(ctx base.PlanContext, childTable table.Table, fk return fkCheck, nil } -func buildFKCheck(ctx base.PlanContext, tbl table.Table, cols []pmodel.CIStr, failedErr error) (*FKCheck, error) { +func buildFKCheck(ctx base.PlanContext, tbl table.Table, cols []ast.CIStr, failedErr error) (*FKCheck, error) { tblInfo := tbl.Meta() if tblInfo.PKIsHandle && len(cols) == 1 { refColInfo := model.FindColumnInfo(tblInfo.Columns, cols[0].L) diff --git a/pkg/planner/core/fragment.go b/pkg/planner/core/fragment.go index 5fa55c7d44436..6f40acd146323 100644 --- a/pkg/planner/core/fragment.go +++ b/pkg/planner/core/fragment.go @@ -29,7 +29,7 @@ import ( "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/kv" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/sessionctx" @@ -491,7 +491,7 @@ func (e *mppTaskGenerator) addReaderTasksForCTEStorage(storageID int, tasks ...* } } -func partitionPruning(ctx base.PlanContext, tbl table.PartitionedTable, conds []expression.Expression, partitionNames []model.CIStr, +func partitionPruning(ctx base.PlanContext, tbl table.PartitionedTable, conds []expression.Expression, partitionNames []ast.CIStr, columns []*expression.Column, columnNames types.NameSlice) ([]table.PhysicalTable, error) { idxArr, err := PartitionPruning(ctx, tbl, conds, partitionNames, columns, columnNames) if err != nil { diff --git a/pkg/planner/core/hint_utils.go b/pkg/planner/core/hint_utils.go index 62e63f6fe5813..8f5e8cb68dd69 100644 --- a/pkg/planner/core/hint_utils.go +++ b/pkg/planner/core/hint_utils.go @@ -19,7 +19,6 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser/ast" - "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/core/operator/logicalop" h "github.com/pingcap/tidb/pkg/util/hint" @@ -89,7 +88,7 @@ func genHintsFromSingle(p base.PhysicalPlan, nodeType h.NodeType, storeType kv.S if storeType == kv.TiKV { res = append(res, &ast.TableOptimizerHint{ QBName: qbName, - HintName: model.NewCIStr(h.HintLimitToCop), + HintName: ast.NewCIStr(h.HintLimitToCop), }) } case *PhysicalTableReader: @@ -100,14 +99,14 @@ func genHintsFromSingle(p base.PhysicalPlan, nodeType h.NodeType, storeType kv.S if tbl.StoreType == kv.TiFlash { res = append(res, &ast.TableOptimizerHint{ QBName: qbName, - HintName: model.NewCIStr(h.HintReadFromStorage), - HintData: model.NewCIStr(kv.TiFlash.Name()), + HintName: ast.NewCIStr(h.HintReadFromStorage), + HintData: ast.NewCIStr(kv.TiFlash.Name()), Tables: []ast.HintTable{{DBName: tbl.DBName, TableName: getTableName(tbl.Table.Name, tbl.TableAsName)}}, }) } else { res = append(res, &ast.TableOptimizerHint{ QBName: qbName, - HintName: model.NewCIStr(h.HintUseIndex), + HintName: ast.NewCIStr(h.HintUseIndex), Tables: []ast.HintTable{{DBName: tbl.DBName, TableName: getTableName(tbl.Table.Name, tbl.TableAsName)}}, }) if tbl.Table.PKIsHandle || tbl.Table.IsCommonHandle { // it's a primary key @@ -117,9 +116,9 @@ func genHintsFromSingle(p base.PhysicalPlan, nodeType h.NodeType, storeType kv.S } res = append(res, &ast.TableOptimizerHint{ QBName: qbName, - HintName: model.NewCIStr(orderHint), + HintName: ast.NewCIStr(orderHint), Tables: []ast.HintTable{{DBName: tbl.DBName, TableName: getTableName(tbl.Table.Name, tbl.TableAsName)}}, - Indexes: []model.CIStr{model.NewCIStr("primary")}, + Indexes: []ast.CIStr{ast.NewCIStr("primary")}, }) } } @@ -127,9 +126,9 @@ func genHintsFromSingle(p base.PhysicalPlan, nodeType h.NodeType, storeType kv.S index := pp.IndexPlans[0].(*PhysicalIndexScan) res = append(res, &ast.TableOptimizerHint{ QBName: qbName, - HintName: model.NewCIStr(h.HintUseIndex), + HintName: ast.NewCIStr(h.HintUseIndex), Tables: []ast.HintTable{{DBName: index.DBName, TableName: getTableName(index.Table.Name, index.TableAsName)}}, - Indexes: []model.CIStr{index.Index.Name}, + Indexes: []ast.CIStr{index.Index.Name}, }) orderHint := h.HintOrderIndex if !index.KeepOrder { @@ -137,17 +136,17 @@ func genHintsFromSingle(p base.PhysicalPlan, nodeType h.NodeType, storeType kv.S } res = append(res, &ast.TableOptimizerHint{ QBName: qbName, - HintName: model.NewCIStr(orderHint), + HintName: ast.NewCIStr(orderHint), Tables: []ast.HintTable{{DBName: index.DBName, TableName: getTableName(index.Table.Name, index.TableAsName)}}, - Indexes: []model.CIStr{index.Index.Name}, + Indexes: []ast.CIStr{index.Index.Name}, }) case *PhysicalIndexReader: index := pp.IndexPlans[0].(*PhysicalIndexScan) res = append(res, &ast.TableOptimizerHint{ QBName: qbName, - HintName: model.NewCIStr(h.HintUseIndex), + HintName: ast.NewCIStr(h.HintUseIndex), Tables: []ast.HintTable{{DBName: index.DBName, TableName: getTableName(index.Table.Name, index.TableAsName)}}, - Indexes: []model.CIStr{index.Index.Name}, + Indexes: []ast.CIStr{index.Index.Name}, }) orderHint := h.HintOrderIndex if !index.KeepOrder { @@ -155,50 +154,50 @@ func genHintsFromSingle(p base.PhysicalPlan, nodeType h.NodeType, storeType kv.S } res = append(res, &ast.TableOptimizerHint{ QBName: qbName, - HintName: model.NewCIStr(orderHint), + HintName: ast.NewCIStr(orderHint), Tables: []ast.HintTable{{DBName: index.DBName, TableName: getTableName(index.Table.Name, index.TableAsName)}}, - Indexes: []model.CIStr{index.Index.Name}, + Indexes: []ast.CIStr{index.Index.Name}, }) case *PhysicalIndexMergeReader: - indexs := make([]model.CIStr, 0, 2) - var tableName model.CIStr - var tableAsName *model.CIStr + indexs := make([]ast.CIStr, 0, 2) + var tableName ast.CIStr + var tableAsName *ast.CIStr for _, partialPlan := range pp.PartialPlans { if index, ok := partialPlan[0].(*PhysicalIndexScan); ok { indexs = append(indexs, index.Index.Name) tableName = index.Table.Name tableAsName = index.TableAsName } else { - indexName := model.NewCIStr("PRIMARY") + indexName := ast.NewCIStr("PRIMARY") indexs = append(indexs, indexName) } } res = append(res, &ast.TableOptimizerHint{ QBName: qbName, - HintName: model.NewCIStr(h.HintIndexMerge), + HintName: ast.NewCIStr(h.HintIndexMerge), Tables: []ast.HintTable{{TableName: getTableName(tableName, tableAsName)}}, Indexes: indexs, }) case *PhysicalHashAgg: res = append(res, &ast.TableOptimizerHint{ QBName: qbName, - HintName: model.NewCIStr(h.HintHashAgg), + HintName: ast.NewCIStr(h.HintHashAgg), }) if storeType == kv.TiKV { res = append(res, &ast.TableOptimizerHint{ QBName: qbName, - HintName: model.NewCIStr(h.HintAggToCop), + HintName: ast.NewCIStr(h.HintAggToCop), }) } case *PhysicalStreamAgg: res = append(res, &ast.TableOptimizerHint{ QBName: qbName, - HintName: model.NewCIStr(h.HintStreamAgg), + HintName: ast.NewCIStr(h.HintStreamAgg), }) if storeType == kv.TiKV { res = append(res, &ast.TableOptimizerHint{ QBName: qbName, - HintName: model.NewCIStr(h.HintAggToCop), + HintName: ast.NewCIStr(h.HintAggToCop), }) } case *PhysicalMergeJoin: @@ -306,7 +305,7 @@ func genHintsFromSingle(p base.PhysicalPlan, nodeType h.NodeType, storeType kv.S return res } -func getTableName(tblName model.CIStr, asName *model.CIStr) model.CIStr { +func getTableName(tblName ast.CIStr, asName *ast.CIStr) ast.CIStr { if asName != nil && asName.L != "" { return *asName } @@ -345,7 +344,7 @@ func genJoinMethodHintForSinglePhysicalJoin( } newHint := &ast.TableOptimizerHint{ - HintName: model.NewCIStr(joinType), + HintName: ast.NewCIStr(joinType), Tables: []ast.HintTable{*effectiveHintTbls[0]}, } @@ -366,7 +365,7 @@ func genHintTblForJoinNodes( joinedNodes []base.PhysicalPlan, parentQBOffset int, nodeType h.NodeType, -) (hintTbls []*ast.HintTable, hintQBNamePtr *model.CIStr) { +) (hintTbls []*ast.HintTable, hintQBNamePtr *ast.CIStr) { // 1. Use genHintTblForSingleJoinNode() to generate QB offset and table name for each join node. // Note that if we failed to generate valid information for one element in joinedNodes, we append -1 and nil instead @@ -462,7 +461,7 @@ func genHintTblForSingleJoinNode( return -1, false, nil } guessQBOffset = false - var dbName, tableName *model.CIStr + var dbName, tableName *ast.CIStr // For sub-queries like `(select * from t) t1`, t1 should belong to its surrounding select block. if qbOffset != parentOffset { var blockAsNames []ast.HintTable @@ -495,7 +494,7 @@ func genHintTblForSingleJoinNode( return qbOffset, guessQBOffset, &ast.HintTable{DBName: *dbName, TableName: *tableName} } -func extractTableAsName(p base.PhysicalPlan) (*model.CIStr, *model.CIStr) { +func extractTableAsName(p base.PhysicalPlan) (*ast.CIStr, *ast.CIStr) { if len(p.Children()) > 1 { return nil, nil } @@ -555,7 +554,7 @@ func genJoinOrderHintFromRootPhysicalJoin( hintTblVals = append(hintTblVals, *ht) } res := &ast.TableOptimizerHint{ - HintName: model.NewCIStr(h.HintLeading), + HintName: ast.NewCIStr(h.HintLeading), Tables: hintTblVals, } if hintQBName != nil { diff --git a/pkg/planner/core/indexmerge_path_test.go b/pkg/planner/core/indexmerge_path_test.go index 776c449f5228f..80969e3d777b4 100644 --- a/pkg/planner/core/indexmerge_path_test.go +++ b/pkg/planner/core/indexmerge_path_test.go @@ -28,7 +28,6 @@ import ( "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/parser" "github.com/pingcap/tidb/pkg/parser/ast" - "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/core/operator/logicalop" @@ -79,7 +78,7 @@ func TestCollectFilters4MVIndexMutations(t *testing.T) { ds, ok = p.(*logicalop.DataSource) } cnfs := ds.AllConds - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) idxCols, ok := core.PrepareIdxColsAndUnwrapArrayType( tbl.Meta(), @@ -457,7 +456,7 @@ func TestAnalyzeVectorIndex(t *testing.T) { }() tk.MustExec(`create table t(a int, b vector(2), c vector(3), j json, index(a))`) tk.MustExec("alter table t set tiflash replica 2 location labels 'a','b';") - tblInfo, err := dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tblInfo, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) err = domain.GetDomain(tk.Session()).DDLExecutor().UpdateTableReplicaInfo(tk.Session(), tblInfo.Meta().ID, true) require.NoError(t, err) diff --git a/pkg/planner/core/integration_test.go b/pkg/planner/core/integration_test.go index c653a4e67e6b8..4efe5629186e6 100644 --- a/pkg/planner/core/integration_test.go +++ b/pkg/planner/core/integration_test.go @@ -30,8 +30,8 @@ import ( "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/meta/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/auth" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/sessionctx/variable" @@ -84,7 +84,7 @@ func TestAggPushDownEngine(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - tblInfo, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tblInfo, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tblInfo.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ Count: 1, @@ -128,9 +128,9 @@ func TestIssue15110And49616(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - db, exists := is.SchemaByName(pmodel.NewCIStr("test")) + db, exists := is.SchemaByName(ast.NewCIStr("test")) require.True(t, exists) - tblInfo, err := is.TableByName(context.Background(), db.Name, pmodel.NewCIStr("crm_rd_150m")) + tblInfo, err := is.TableByName(context.Background(), db.Name, ast.NewCIStr("crm_rd_150m")) require.NoError(t, err) tblInfo.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ Count: 1, @@ -159,12 +159,12 @@ func TestPartitionPruningForEQ(t *testing.T) { tk.MustExec("create table t(a datetime, b int) partition by range(weekday(a)) (partition p0 values less than(10), partition p1 values less than (100))") is := tk.Session().GetInfoSchema().(infoschema.InfoSchema) - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) pt := tbl.(table.PartitionedTable) query, err := expression.ParseSimpleExpr(tk.Session().GetExprCtx(), "a = '2020-01-01 00:00:00'", expression.WithTableInfo("", tbl.Meta())) require.NoError(t, err) - dbName := pmodel.NewCIStr(tk.Session().GetSessionVars().CurrentDB) + dbName := ast.NewCIStr(tk.Session().GetSessionVars().CurrentDB) columns, names, err := expression.ColumnInfos2ColumnsAndNames(tk.Session().GetExprCtx(), dbName, tbl.Meta().Name, tbl.Meta().Cols(), tbl.Meta()) require.NoError(t, err) // Even the partition is not monotonous, EQ condition should be prune! @@ -186,7 +186,7 @@ func TestNotReadOnlySQLOnTiFlash(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - tblInfo, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tblInfo, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tblInfo.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ Count: 1, @@ -218,7 +218,7 @@ func TestTimeToSecPushDownToTiFlash(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ Count: 1, @@ -248,7 +248,7 @@ func TestRightShiftPushDownToTiFlash(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ Count: 1, @@ -684,7 +684,7 @@ func TestReverseUTF8PushDownToTiFlash(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ Count: 1, @@ -714,7 +714,7 @@ func TestReversePushDownToTiFlash(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ Count: 1, @@ -744,7 +744,7 @@ func TestSpacePushDownToTiFlash(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ Count: 1, @@ -880,7 +880,7 @@ func TestConflictReadFromStorage(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ Count: 1, @@ -942,7 +942,7 @@ func TestIssue31202(t *testing.T) { tk.MustExec("use test") tk.MustExec("create table t31202(a int primary key, b int);") - tbl, err := dom.InfoSchema().TableByName(context.Background(), pmodel.CIStr{O: "test", L: "test"}, pmodel.CIStr{O: "t31202", L: "t31202"}) + tbl, err := dom.InfoSchema().TableByName(context.Background(), ast.CIStr{O: "test", L: "test"}, ast.CIStr{O: "t31202", L: "t31202"}) require.NoError(t, err) // Set the hacked TiFlash replica for explain tests. tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{Count: 1, Available: true} @@ -999,7 +999,7 @@ func TestTiFlashFineGrainedShuffleWithMaxTiFlashThreads(t *testing.T) { tk.MustExec("set @@tidb_enforce_mpp = on") tk.MustExec("drop table if exists t1;") tk.MustExec("create table t1(c1 int, c2 int)") - tbl1, err := dom.InfoSchema().TableByName(context.Background(), pmodel.CIStr{O: "test", L: "test"}, pmodel.CIStr{O: "t1", L: "t1"}) + tbl1, err := dom.InfoSchema().TableByName(context.Background(), ast.CIStr{O: "test", L: "test"}, ast.CIStr{O: "t1", L: "t1"}) require.NoError(t, err) // Set the hacked TiFlash replica for explain tests. tbl1.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{Count: 1, Available: true} @@ -1188,7 +1188,7 @@ func TestRepeatPushDownToTiFlash(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ Count: 1, @@ -1223,7 +1223,7 @@ func TestIssue36194(t *testing.T) { // create virtual tiflash replica. is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ Count: 1, @@ -1248,7 +1248,7 @@ func TestGetFormatPushDownToTiFlash(t *testing.T) { tk.MustExec("set @@tidb_enforce_mpp=1;") tk.MustExec("set @@tidb_isolation_read_engines = 'tiflash';") - tbl, err := dom.InfoSchema().TableByName(context.Background(), pmodel.CIStr{O: "test", L: "test"}, pmodel.CIStr{O: "t", L: "t"}) + tbl, err := dom.InfoSchema().TableByName(context.Background(), ast.CIStr{O: "test", L: "test"}, ast.CIStr{O: "t", L: "t"}) require.NoError(t, err) // Set the hacked TiFlash replica for explain tests. tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{Count: 1, Available: true} @@ -1275,7 +1275,7 @@ func TestAggWithJsonPushDownToTiFlash(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ Count: 1, @@ -1320,7 +1320,7 @@ func TestLeftShiftPushDownToTiFlash(t *testing.T) { // Create virtual tiflash replica info. is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ Count: 1, @@ -1347,7 +1347,7 @@ func TestHexIntOrStrPushDownToTiFlash(t *testing.T) { tk.MustExec("set @@tidb_allow_mpp=1; set @@tidb_enforce_mpp=1;") tk.MustExec("set @@tidb_isolation_read_engines = 'tiflash'") - tbl, err := dom.InfoSchema().TableByName(context.Background(), pmodel.CIStr{O: "test", L: "test"}, pmodel.CIStr{O: "t", L: "t"}) + tbl, err := dom.InfoSchema().TableByName(context.Background(), ast.CIStr{O: "test", L: "test"}, ast.CIStr{O: "t", L: "t"}) require.NoError(t, err) // Set the hacked TiFlash replica for explain tests. tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{Count: 1, Available: true} @@ -1380,7 +1380,7 @@ func TestBinPushDownToTiFlash(t *testing.T) { tk.MustExec("set @@tidb_allow_mpp=1; set @@tidb_enforce_mpp=1;") tk.MustExec("set @@tidb_isolation_read_engines = 'tiflash'") - tbl, err := dom.InfoSchema().TableByName(context.Background(), pmodel.CIStr{O: "test", L: "test"}, pmodel.CIStr{O: "t", L: "t"}) + tbl, err := dom.InfoSchema().TableByName(context.Background(), ast.CIStr{O: "test", L: "test"}, ast.CIStr{O: "t", L: "t"}) require.NoError(t, err) // Set the hacked TiFlash replica for explain tests. tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{Count: 1, Available: true} @@ -1408,7 +1408,7 @@ func TestEltPushDownToTiFlash(t *testing.T) { // Create virtual tiflash replica info. is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ Count: 1, @@ -1437,7 +1437,7 @@ func TestRegexpInstrPushDownToTiFlash(t *testing.T) { // Create virtual tiflash replica info. is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ Count: 1, @@ -1466,7 +1466,7 @@ func TestRegexpSubstrPushDownToTiFlash(t *testing.T) { // Create virtual tiflash replica info. is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ Count: 1, @@ -1495,7 +1495,7 @@ func TestRegexpReplacePushDownToTiFlash(t *testing.T) { // Create virtual tiflash replica info. is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ Count: 1, @@ -1528,7 +1528,7 @@ func TestCastTimeAsDurationToTiFlash(t *testing.T) { // Create virtual tiflash replica info. is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ Count: 1, @@ -1555,7 +1555,7 @@ func TestUnhexPushDownToTiFlash(t *testing.T) { tk.MustExec("set @@tidb_allow_mpp=1; set @@tidb_enforce_mpp=1;") tk.MustExec("set @@tidb_isolation_read_engines = 'tiflash'") - tbl, err := dom.InfoSchema().TableByName(context.Background(), pmodel.CIStr{O: "test", L: "test"}, pmodel.CIStr{O: "t", L: "t"}) + tbl, err := dom.InfoSchema().TableByName(context.Background(), ast.CIStr{O: "test", L: "test"}, ast.CIStr{O: "t", L: "t"}) require.NoError(t, err) // Set the hacked TiFlash replica for explain tests. tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{Count: 1, Available: true} @@ -1588,7 +1588,7 @@ func TestLeastGretestStringPushDownToTiFlash(t *testing.T) { tk.MustExec("set @@tidb_allow_mpp=1; set @@tidb_enforce_mpp=1") tk.MustExec("set @@tidb_isolation_read_engines = 'tiflash'") - tbl, err := dom.InfoSchema().TableByName(context.Background(), pmodel.CIStr{O: "test", L: "test"}, pmodel.CIStr{O: "t", L: "t"}) + tbl, err := dom.InfoSchema().TableByName(context.Background(), ast.CIStr{O: "test", L: "test"}, ast.CIStr{O: "t", L: "t"}) require.NoError(t, err) // Set the hacked TiFlash replica for explain tests. tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{Count: 1, Available: true} @@ -1633,12 +1633,12 @@ func TestTiFlashReadForWriteStmt(t *testing.T) { tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 tidb_enable_tiflash_read_for_write_stmt is always turned on. This variable has been deprecated and will be removed in the future releases")) tk.MustQuery("select @@tidb_enable_tiflash_read_for_write_stmt").Check(testkit.Rows("1")) - tbl, err := dom.InfoSchema().TableByName(context.Background(), pmodel.CIStr{O: "test", L: "test"}, pmodel.CIStr{O: "t", L: "t"}) + tbl, err := dom.InfoSchema().TableByName(context.Background(), ast.CIStr{O: "test", L: "test"}, ast.CIStr{O: "t", L: "t"}) require.NoError(t, err) // Set the hacked TiFlash replica for explain tests. tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{Count: 1, Available: true} - tbl2, err := dom.InfoSchema().TableByName(context.Background(), pmodel.CIStr{O: "test", L: "test"}, pmodel.CIStr{O: "t2", L: "t2"}) + tbl2, err := dom.InfoSchema().TableByName(context.Background(), ast.CIStr{O: "test", L: "test"}, ast.CIStr{O: "t2", L: "t2"}) require.NoError(t, err) // Set the hacked TiFlash replica for explain tests. tbl2.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{Count: 1, Available: true} @@ -1694,11 +1694,11 @@ func TestPointGetWithSelectLock(t *testing.T) { tk.MustExec("use test") tk.MustExec("create table t(a int, b int, primary key(a, b));") tk.MustExec("create table t1(c int unique, d int);") - tbl, err := dom.InfoSchema().TableByName(context.Background(), pmodel.CIStr{O: "test", L: "test"}, pmodel.CIStr{O: "t", L: "t"}) + tbl, err := dom.InfoSchema().TableByName(context.Background(), ast.CIStr{O: "test", L: "test"}, ast.CIStr{O: "t", L: "t"}) require.NoError(t, err) // Set the hacked TiFlash replica for explain tests. tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{Count: 1, Available: true} - tbl1, err := dom.InfoSchema().TableByName(context.Background(), pmodel.CIStr{O: "test", L: "test"}, pmodel.CIStr{O: "t1", L: "t1"}) + tbl1, err := dom.InfoSchema().TableByName(context.Background(), ast.CIStr{O: "test", L: "test"}, ast.CIStr{O: "t1", L: "t1"}) require.NoError(t, err) // Set the hacked TiFlash replica for explain tests. tbl1.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{Count: 1, Available: true} @@ -1906,7 +1906,7 @@ func TestIsIPv4ToTiFlash(t *testing.T) { // Create virtual tiflash replica info. is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ Count: 1, @@ -1937,7 +1937,7 @@ func TestIsIPv6ToTiFlash(t *testing.T) { // Create virtual tiflash replica info. is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ Count: 1, @@ -1995,7 +1995,7 @@ func TestVirtualExprPushDown(t *testing.T) { tk.MustExec("set @@tidb_allow_mpp=1; set @@tidb_enforce_mpp=1") tk.MustExec("set @@tidb_isolation_read_engines = 'tiflash'") is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ Count: 1, diff --git a/pkg/planner/core/logical_plan_builder.go b/pkg/planner/core/logical_plan_builder.go index 074bfd2734280..cd5058a5a785a 100644 --- a/pkg/planner/core/logical_plan_builder.go +++ b/pkg/planner/core/logical_plan_builder.go @@ -40,7 +40,6 @@ import ( "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/charset" "github.com/pingcap/tidb/pkg/parser/format" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/opcode" "github.com/pingcap/tidb/pkg/parser/terror" @@ -983,7 +982,7 @@ func (b *PlanBuilder) buildSelection(ctx context.Context, p base.LogicalPlan, wh } // buildProjectionFieldNameFromColumns builds the field name, table name and database name when field expression is a column reference. -func (*PlanBuilder) buildProjectionFieldNameFromColumns(origField *ast.SelectField, colNameField *ast.ColumnNameExpr, name *types.FieldName) (colName, origColName, tblName, origTblName, dbName pmodel.CIStr) { +func (*PlanBuilder) buildProjectionFieldNameFromColumns(origField *ast.SelectField, colNameField *ast.ColumnNameExpr, name *types.FieldName) (colName, origColName, tblName, origTblName, dbName ast.CIStr) { origTblName, origColName, dbName = name.OrigTblName, name.OrigColName, name.DBName if origField.AsName.L == "" { colName = colNameField.Name.Name @@ -999,7 +998,7 @@ func (*PlanBuilder) buildProjectionFieldNameFromColumns(origField *ast.SelectFie } // buildProjectionFieldNameFromExpressions builds the field name when field expression is a normal expression. -func (b *PlanBuilder) buildProjectionFieldNameFromExpressions(_ context.Context, field *ast.SelectField) (pmodel.CIStr, error) { +func (b *PlanBuilder) buildProjectionFieldNameFromExpressions(_ context.Context, field *ast.SelectField) (ast.CIStr, error) { if agg, ok := field.Expr.(*ast.AggregateFuncExpr); ok && agg.F == ast.AggFuncFirstRow { // When the query is select t.a from t group by a; The Column Name should be a but not t.a; return agg.Args[0].(*ast.ColumnNameExpr).Name.Name, nil @@ -1012,16 +1011,16 @@ func (b *PlanBuilder) buildProjectionFieldNameFromExpressions(_ context.Context, if isFuncCall && funcCall.FnName.L == ast.NameConst { if v, err := evalAstExpr(b.ctx.GetExprCtx(), funcCall.Args[0]); err == nil { if s, err := v.ToString(); err == nil { - return pmodel.NewCIStr(s), nil + return ast.NewCIStr(s), nil } } - return pmodel.NewCIStr(""), plannererrors.ErrWrongArguments.GenWithStackByArgs("NAME_CONST") + return ast.NewCIStr(""), plannererrors.ErrWrongArguments.GenWithStackByArgs("NAME_CONST") } valueExpr, isValueExpr := innerExpr.(*driver.ValueExpr) // Non-literal: Output as inputed, except that comments need to be removed. if !isValueExpr { - return pmodel.NewCIStr(parser.SpecFieldPattern.ReplaceAllStringFunc(field.Text(), parser.TrimComment)), nil + return ast.NewCIStr(parser.SpecFieldPattern.ReplaceAllStringFunc(field.Text(), parser.TrimComment)), nil } // Literal: Need special processing @@ -1037,21 +1036,21 @@ func (b *PlanBuilder) buildProjectionFieldNameFromExpressions(_ context.Context, fieldName := strings.TrimLeftFunc(projName, func(r rune) bool { return !unicode.IsOneOf(mysql.RangeGraph, r) }) - return pmodel.NewCIStr(fieldName), nil + return ast.NewCIStr(fieldName), nil case types.KindNull: // See #4053, #3685 - return pmodel.NewCIStr("NULL"), nil + return ast.NewCIStr("NULL"), nil case types.KindBinaryLiteral: // Don't rewrite BIT literal or HEX literals - return pmodel.NewCIStr(field.Text()), nil + return ast.NewCIStr(field.Text()), nil case types.KindInt64: // See #9683 // TRUE or FALSE can be a int64 if mysql.HasIsBooleanFlag(valueExpr.Type.GetFlag()) { if i := valueExpr.GetValue().(int64); i == 0 { - return pmodel.NewCIStr("FALSE"), nil + return ast.NewCIStr("FALSE"), nil } - return pmodel.NewCIStr("TRUE"), nil + return ast.NewCIStr("TRUE"), nil } fallthrough @@ -1059,24 +1058,24 @@ func (b *PlanBuilder) buildProjectionFieldNameFromExpressions(_ context.Context, fieldName := field.Text() fieldName = strings.TrimLeft(fieldName, "\t\n +(") fieldName = strings.TrimRight(fieldName, "\t\n )") - return pmodel.NewCIStr(fieldName), nil + return ast.NewCIStr(fieldName), nil } } func buildExpandFieldName(ctx expression.EvalContext, expr expression.Expression, name *types.FieldName, genName string) *types.FieldName { _, isCol := expr.(*expression.Column) - var origTblName, origColName, dbName, colName, tblName pmodel.CIStr + var origTblName, origColName, dbName, colName, tblName ast.CIStr if genName != "" { // for case like: gid_, gpos_ - colName = pmodel.NewCIStr(expr.StringWithCtx(ctx, errors.RedactLogDisable)) + colName = ast.NewCIStr(expr.StringWithCtx(ctx, errors.RedactLogDisable)) } else if isCol { // col ref to original col, while its nullability may be changed. origTblName, origColName, dbName = name.OrigTblName, name.OrigColName, name.DBName - colName = pmodel.NewCIStr("ex_" + name.ColName.O) - tblName = pmodel.NewCIStr("ex_" + name.TblName.O) + colName = ast.NewCIStr("ex_" + name.ColName.O) + tblName = ast.NewCIStr("ex_" + name.TblName.O) } else { // Other: complicated expression. - colName = pmodel.NewCIStr("ex_" + expr.StringWithCtx(ctx, errors.RedactLogDisable)) + colName = ast.NewCIStr("ex_" + expr.StringWithCtx(ctx, errors.RedactLogDisable)) } newName := &types.FieldName{ TblName: tblName, @@ -1090,7 +1089,7 @@ func buildExpandFieldName(ctx expression.EvalContext, expr expression.Expression // buildProjectionField builds the field object according to SelectField in projection. func (b *PlanBuilder) buildProjectionField(ctx context.Context, p base.LogicalPlan, field *ast.SelectField, expr expression.Expression) (*expression.Column, *types.FieldName, error) { - var origTblName, tblName, origColName, colName, dbName pmodel.CIStr + var origTblName, tblName, origColName, colName, dbName ast.CIStr innerNode := getInnerFromParenthesesAndUnaryPlus(field.Expr) col, isCol := expr.(*expression.Column) // Correlated column won't affect the final output names. So we can put it in any of the three logic block. @@ -2288,7 +2287,7 @@ func (a *havingWindowAndOrderbyExprResolver) Leave(n ast.Node) (node ast.Node, o a.selectFields = append(a.selectFields, &ast.SelectField{ Auxiliary: true, Expr: v, - AsName: pmodel.NewCIStr(fmt.Sprintf("sel_agg_%d", len(a.selectFields))), + AsName: ast.NewCIStr(fmt.Sprintf("sel_agg_%d", len(a.selectFields))), }) case *ast.WindowFuncExpr: a.inWindowFunc = false @@ -2300,7 +2299,7 @@ func (a *havingWindowAndOrderbyExprResolver) Leave(n ast.Node) (node ast.Node, o a.selectFields = append(a.selectFields, &ast.SelectField{ Auxiliary: true, Expr: v, - AsName: pmodel.NewCIStr(fmt.Sprintf("sel_window_%d", len(a.selectFields))), + AsName: ast.NewCIStr(fmt.Sprintf("sel_window_%d", len(a.selectFields))), }) } case *ast.WindowSpec: @@ -2829,7 +2828,7 @@ func (b *PlanBuilder) resolveCorrelatedAggregates(ctx context.Context, sel *ast. sel.Fields.Fields = append(sel.Fields.Fields, &ast.SelectField{ Auxiliary: true, Expr: aggFunc, - AsName: pmodel.NewCIStr(fmt.Sprintf("sel_subq_agg_%d", len(sel.Fields.Fields))), + AsName: ast.NewCIStr(fmt.Sprintf("sel_subq_agg_%d", len(sel.Fields.Fields))), }) } return correlatedAggMap, nil @@ -3616,9 +3615,9 @@ func (b *PlanBuilder) addAliasName(ctx context.Context, selectStmt *ast.SelectSt oldName := field.AsName if dup, ok := dedupMap[field.AsName.L]; ok { if dup == 0 { - field.AsName = pmodel.NewCIStr(fmt.Sprintf("Name_exp_%s", field.AsName.O)) + field.AsName = ast.NewCIStr(fmt.Sprintf("Name_exp_%s", field.AsName.O)) } else { - field.AsName = pmodel.NewCIStr(fmt.Sprintf("Name_exp_%d_%s", dup, field.AsName.O)) + field.AsName = ast.NewCIStr(fmt.Sprintf("Name_exp_%d_%s", dup, field.AsName.O)) } dedupMap[oldName.L] = dup + 1 } else { @@ -4183,7 +4182,7 @@ func getLatestVersionFromStatsTable(ctx sessionctx.Context, tblInfo *model.Table // tryBuildCTE considers the input tn as a reference to a CTE and tries to build the logical plan for it like building // DataSource for normal tables. // tryBuildCTE will push an entry into handleHelper when successful. -func (b *PlanBuilder) tryBuildCTE(ctx context.Context, tn *ast.TableName, asName *pmodel.CIStr) (base.LogicalPlan, error) { +func (b *PlanBuilder) tryBuildCTE(ctx context.Context, tn *ast.TableName, asName *ast.CIStr) (base.LogicalPlan, error) { for i := len(b.outerCTEs) - 1; i >= 0; i-- { cte := b.outerCTEs[i] if cte.def.Name.L == tn.Name.L { @@ -4339,7 +4338,7 @@ func (b *PlanBuilder) buildDataSourceFromCTEMerge(ctx context.Context, cte *ast. outPutNames := p.OutputNames() for _, name := range outPutNames { name.TblName = cte.Name - name.DBName = pmodel.NewCIStr(b.ctx.GetSessionVars().CurrentDB) + name.DBName = ast.NewCIStr(b.ctx.GetSessionVars().CurrentDB) } if len(cte.ColNameList) > 0 { @@ -4354,7 +4353,7 @@ func (b *PlanBuilder) buildDataSourceFromCTEMerge(ctx context.Context, cte *ast. return p, nil } -func (b *PlanBuilder) buildDataSource(ctx context.Context, tn *ast.TableName, asName *pmodel.CIStr) (base.LogicalPlan, error) { +func (b *PlanBuilder) buildDataSource(ctx context.Context, tn *ast.TableName, asName *ast.CIStr) (base.LogicalPlan, error) { b.optFlag |= rule.FlagPredicateSimplification dbName := tn.Schema sessionVars := b.ctx.GetSessionVars() @@ -4365,7 +4364,7 @@ func (b *PlanBuilder) buildDataSource(ctx context.Context, tn *ast.TableName, as if err != nil || p != nil { return p, err } - dbName = pmodel.NewCIStr(sessionVars.CurrentDB) + dbName = ast.NewCIStr(sessionVars.CurrentDB) } is := b.is @@ -4607,7 +4606,7 @@ func (b *PlanBuilder) buildDataSource(ctx context.Context, tn *ast.TableName, as Columns: make([]*model.ColumnInfo, 0, len(columns)), PartitionNames: tn.PartitionNames, TblCols: make([]*expression.Column, 0, len(columns)), - PreferPartitions: make(map[int][]pmodel.CIStr), + PreferPartitions: make(map[int][]ast.CIStr), IS: b.is, IsForUpdateRead: b.isForUpdateRead, }.Init(b.ctx, b.getSelectOffset()) @@ -4769,7 +4768,7 @@ func (b *PlanBuilder) timeRangeForSummaryTable() util.QueryTimeRange { return util.QueryTimeRange{From: from, To: to} } -func (b *PlanBuilder) buildMemTable(_ context.Context, dbName pmodel.CIStr, tableInfo *model.TableInfo) (base.LogicalPlan, error) { +func (b *PlanBuilder) buildMemTable(_ context.Context, dbName ast.CIStr, tableInfo *model.TableInfo) (base.LogicalPlan, error) { // We can use the `TableInfo.Columns` directly because the memory table has // a stable schema and there is no online DDL on the memory table. schema := expression.NewSchema(make([]*expression.Column, 0, len(tableInfo.Columns))...) @@ -4884,7 +4883,7 @@ func (b *PlanBuilder) buildMemTable(_ context.Context, dbName pmodel.CIStr, tabl } // checkRecursiveView checks whether this view is recursively defined. -func (b *PlanBuilder) checkRecursiveView(dbName pmodel.CIStr, tableName pmodel.CIStr) (func(), error) { +func (b *PlanBuilder) checkRecursiveView(dbName ast.CIStr, tableName ast.CIStr) (func(), error) { viewFullName := dbName.L + "." + tableName.L if b.buildingViewStack == nil { b.buildingViewStack = set.NewStringSet() @@ -4906,7 +4905,7 @@ func (b *PlanBuilder) checkRecursiveView(dbName pmodel.CIStr, tableName pmodel.C // qbNameMap4View and viewHints are used for the view's hint. // qbNameMap4View maps the query block name to the view table lists. // viewHints group the view hints based on the view's query block name. -func (b *PlanBuilder) BuildDataSourceFromView(ctx context.Context, dbName pmodel.CIStr, tableInfo *model.TableInfo, qbNameMap4View map[string][]ast.HintTable, viewHints map[string][]*ast.TableOptimizerHint) (base.LogicalPlan, error) { +func (b *PlanBuilder) BuildDataSourceFromView(ctx context.Context, dbName ast.CIStr, tableInfo *model.TableInfo, qbNameMap4View map[string][]ast.HintTable, viewHints map[string][]*ast.TableOptimizerHint) (base.LogicalPlan, error) { viewDepth := b.ctx.GetSessionVars().StmtCtx.ViewDepth b.ctx.GetSessionVars().StmtCtx.ViewDepth++ deferFunc, err := b.checkRecursiveView(dbName, tableInfo.Name) @@ -5010,7 +5009,7 @@ func (b *PlanBuilder) BuildDataSourceFromView(ctx context.Context, dbName pmodel !pm.RequestVerification(b.ctx.GetSessionVars().ActiveRoles, dbName.L, tableInfo.Name.L, "", mysql.SelectPriv) { return nil, plannererrors.ErrViewNoExplain } - if tableInfo.View.Security == pmodel.SecurityDefiner { + if tableInfo.View.Security == ast.SecurityDefiner { if pm != nil { for _, v := range b.visitInfo { if !pm.RequestVerificationWithUser(ctx, v.db, v.table, v.column, v.privilege, tableInfo.View.Definer) { @@ -5033,7 +5032,7 @@ func (b *PlanBuilder) BuildDataSourceFromView(ctx context.Context, dbName pmodel return b.buildProjUponView(ctx, dbName, tableInfo, selectLogicalPlan) } -func (b *PlanBuilder) buildProjUponView(_ context.Context, dbName pmodel.CIStr, tableInfo *model.TableInfo, selectLogicalPlan base.Plan) (base.LogicalPlan, error) { +func (b *PlanBuilder) buildProjUponView(_ context.Context, dbName ast.CIStr, tableInfo *model.TableInfo, selectLogicalPlan base.Plan) (base.LogicalPlan, error) { columnInfo := tableInfo.Cols() cols := selectLogicalPlan.Schema().Clone().Columns outputNamesOfUnderlyingSelect := selectLogicalPlan.OutputNames().Shallow() @@ -5603,7 +5602,7 @@ func CheckUpdateList(assignFlags []int, updt *Update, newTblID2Table map[int64]t tbl := newTblID2Table[content.TblID] flags := assignFlags[content.Start:content.End] var update, updatePK, updatePartitionCol bool - var partitionColumnNames []pmodel.CIStr + var partitionColumnNames []ast.CIStr if pt, ok := tbl.(table.PartitionedTable); ok && pt != nil { partitionColumnNames = pt.GetPartitionColumnNames() } @@ -5958,7 +5957,7 @@ func (b *PlanBuilder) buildDelete(ctx context.Context, ds *ast.DeleteStmt) (base if !foundMatch { if tn.Schema.L == "" { - name = pmodel.NewCIStr(b.ctx.GetSessionVars().CurrentDB).L + "." + tn.Name.L + name = ast.NewCIStr(b.ctx.GetSessionVars().CurrentDB).L + "." + tn.Name.L } else { name = tn.Schema.L + "." + tn.Name.L } @@ -6829,7 +6828,7 @@ func mergeWindowSpec(spec, ref *ast.WindowSpec) error { spec.OrderBy = ref.OrderBy } spec.PartitionBy = ref.PartitionBy - spec.Ref = pmodel.NewCIStr("") + spec.Ref = ast.NewCIStr("") return nil } @@ -6871,7 +6870,7 @@ func (u *updatableTableListResolver) Leave(inNode ast.Node) (ast.Node, bool) { if v.AsName.L != "" { newTableName := *s newTableName.Name = v.AsName - newTableName.Schema = pmodel.NewCIStr("") + newTableName.Schema = ast.NewCIStr("") u.updatableTableList = append(u.updatableTableList, &newTableName) if tnW := u.resolveCtx.GetTableName(s); tnW != nil { u.resolveCtx.AddTableName(&resolve.TableNameW{ @@ -6951,7 +6950,7 @@ func (e *tableListExtractor) Enter(n ast.Node) (_ ast.Node, skipChildren bool) { if x.AsName.L != "" && e.asName { newTableName := *s newTableName.Name = x.AsName - newTableName.Schema = pmodel.NewCIStr("") + newTableName.Schema = ast.NewCIStr("") e.tableNames = append(e.tableNames, &newTableName) if tnW := e.resolveCtx.GetTableName(s); tnW != nil { e.resolveCtx.AddTableName(&resolve.TableNameW{ @@ -6971,7 +6970,7 @@ func (e *tableListExtractor) Enter(n ast.Node) (_ ast.Node, skipChildren bool) { if x.AsName.L != "" && e.asName { newTableName := *innerList[0] newTableName.Name = x.AsName - newTableName.Schema = pmodel.NewCIStr("") + newTableName.Schema = ast.NewCIStr("") innerTableName = &newTableName if tnW := e.resolveCtx.GetTableName(innerList[0]); tnW != nil { e.resolveCtx.AddTableName(&resolve.TableNameW{ @@ -6990,7 +6989,7 @@ func (e *tableListExtractor) Enter(n ast.Node) (_ ast.Node, skipChildren bool) { case *ast.ShowStmt: if x.DBName != "" { - e.tableNames = append(e.tableNames, &ast.TableName{Schema: pmodel.NewCIStr(x.DBName)}) + e.tableNames = append(e.tableNames, &ast.TableName{Schema: ast.NewCIStr(x.DBName)}) } case *ast.CreateDatabaseStmt: e.tableNames = append(e.tableNames, &ast.TableName{Schema: x.Name}) @@ -7001,7 +7000,7 @@ func (e *tableListExtractor) Enter(n ast.Node) (_ ast.Node, skipChildren bool) { case *ast.FlashBackDatabaseStmt: e.tableNames = append(e.tableNames, &ast.TableName{Schema: x.DBName}) - e.tableNames = append(e.tableNames, &ast.TableName{Schema: pmodel.NewCIStr(x.NewName)}) + e.tableNames = append(e.tableNames, &ast.TableName{Schema: ast.NewCIStr(x.NewName)}) case *ast.FlashBackToTimestampStmt: if x.DBName.L != "" { e.tableNames = append(e.tableNames, &ast.TableName{Schema: x.DBName}) @@ -7010,15 +7009,15 @@ func (e *tableListExtractor) Enter(n ast.Node) (_ ast.Node, skipChildren bool) { if newName := x.NewName; newName != "" { e.tableNames = append(e.tableNames, &ast.TableName{ Schema: x.Table.Schema, - Name: pmodel.NewCIStr(newName)}) + Name: ast.NewCIStr(newName)}) } case *ast.GrantStmt: if x.ObjectType == ast.ObjectTypeTable || x.ObjectType == ast.ObjectTypeNone { if x.Level.Level == ast.GrantLevelDB || x.Level.Level == ast.GrantLevelTable { e.tableNames = append(e.tableNames, &ast.TableName{ - Schema: pmodel.NewCIStr(x.Level.DBName), - Name: pmodel.NewCIStr(x.Level.TableName), + Schema: ast.NewCIStr(x.Level.DBName), + Name: ast.NewCIStr(x.Level.TableName), }) } } @@ -7026,19 +7025,19 @@ func (e *tableListExtractor) Enter(n ast.Node) (_ ast.Node, skipChildren bool) { if x.ObjectType == ast.ObjectTypeTable || x.ObjectType == ast.ObjectTypeNone { if x.Level.Level == ast.GrantLevelDB || x.Level.Level == ast.GrantLevelTable { e.tableNames = append(e.tableNames, &ast.TableName{ - Schema: pmodel.NewCIStr(x.Level.DBName), - Name: pmodel.NewCIStr(x.Level.TableName), + Schema: ast.NewCIStr(x.Level.DBName), + Name: ast.NewCIStr(x.Level.TableName), }) } } case *ast.BRIEStmt: if x.Kind == ast.BRIEKindBackup || x.Kind == ast.BRIEKindRestore { for _, v := range x.Schemas { - e.tableNames = append(e.tableNames, &ast.TableName{Schema: pmodel.NewCIStr(v)}) + e.tableNames = append(e.tableNames, &ast.TableName{Schema: ast.NewCIStr(v)}) } } case *ast.UseStmt: - e.tableNames = append(e.tableNames, &ast.TableName{Schema: pmodel.NewCIStr(x.DBName)}) + e.tableNames = append(e.tableNames, &ast.TableName{Schema: ast.NewCIStr(x.DBName)}) case *ast.ExecuteStmt: if v, ok := x.PrepStmt.(*PlanCacheStmt); ok { e.tableNames = append(e.tableNames, innerExtract(v.PreparedAst.Stmt, v.ResolveCtx)...) @@ -7342,7 +7341,7 @@ func (b *PlanBuilder) adjustCTEPlanOutputName(p base.LogicalPlan, def *ast.Commo for _, name := range outPutNames { name.TblName = def.Name if name.DBName.String() == "" { - name.DBName = pmodel.NewCIStr(b.ctx.GetSessionVars().CurrentDB) + name.DBName = ast.NewCIStr(b.ctx.GetSessionVars().CurrentDB) } } if len(def.ColNameList) > 0 { diff --git a/pkg/planner/core/logical_plans_test.go b/pkg/planner/core/logical_plans_test.go index f76b32182d8d8..3660bafa6f9d5 100644 --- a/pkg/planner/core/logical_plans_test.go +++ b/pkg/planner/core/logical_plans_test.go @@ -31,7 +31,6 @@ import ( "github.com/pingcap/tidb/pkg/parser" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/format" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/planner/core/base" @@ -512,27 +511,27 @@ func TestTablePartition(t *testing.T) { definitions := []model.PartitionDefinition{ { ID: 41, - Name: pmodel.NewCIStr("p1"), + Name: ast.NewCIStr("p1"), LessThan: []string{"16"}, }, { ID: 42, - Name: pmodel.NewCIStr("p2"), + Name: ast.NewCIStr("p2"), LessThan: []string{"32"}, }, { ID: 43, - Name: pmodel.NewCIStr("p3"), + Name: ast.NewCIStr("p3"), LessThan: []string{"64"}, }, { ID: 44, - Name: pmodel.NewCIStr("p4"), + Name: ast.NewCIStr("p4"), LessThan: []string{"128"}, }, { ID: 45, - Name: pmodel.NewCIStr("p5"), + Name: ast.NewCIStr("p5"), LessThan: []string{"maxvalue"}, }, } diff --git a/pkg/planner/core/memtable_infoschema_extractor.go b/pkg/planner/core/memtable_infoschema_extractor.go index b4c320e60211f..fd7f0fa9e2ede 100644 --- a/pkg/planner/core/memtable_infoschema_extractor.go +++ b/pkg/planner/core/memtable_infoschema_extractor.go @@ -27,7 +27,7 @@ import ( "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/planner/core/base" @@ -115,11 +115,11 @@ func (e *InfoSchemaBaseExtractor) GetBase() *InfoSchemaBaseExtractor { // ListSchemas lists all schemas from predicate. If no schema is specified, it lists // all schemas in the storage. -func (e *InfoSchemaBaseExtractor) ListSchemas(is infoschema.InfoSchema) []pmodel.CIStr { +func (e *InfoSchemaBaseExtractor) ListSchemas(is infoschema.InfoSchema) []ast.CIStr { extractedSchemas, unspecified := e.listPredicateSchemas(is) if unspecified { ret := is.AllSchemaNames() - slices.SortFunc(ret, func(a, b pmodel.CIStr) int { + slices.SortFunc(ret, func(a, b ast.CIStr) int { return strings.Compare(a.L, b.L) }) return filterSchemaObjectByRegexp(e, e.extractableColumns.schema, ret, extractStrCIStr) @@ -131,7 +131,7 @@ func (e *InfoSchemaBaseExtractor) ListSchemas(is infoschema.InfoSchema) []pmodel // If no schema is specified in predicates, return `unspecified` as true. func (e *InfoSchemaBaseExtractor) listPredicateSchemas( is infoschema.InfoSchema, -) (schemas []pmodel.CIStr, unspecified bool) { +) (schemas []ast.CIStr, unspecified bool) { ec := e.extractableColumns schemas = e.getSchemaObjectNames(ec.schema) if len(schemas) == 0 { @@ -151,13 +151,13 @@ func (e *InfoSchemaBaseExtractor) listPredicateSchemas( func (e *InfoSchemaBaseExtractor) ListSchemasAndTables( ctx context.Context, is infoschema.InfoSchema, -) ([]pmodel.CIStr, []*model.TableInfo, error) { +) ([]ast.CIStr, []*model.TableInfo, error) { ec := e.extractableColumns - var tableNames []pmodel.CIStr + var tableNames []ast.CIStr if ec.table != "" { tableNames = e.getSchemaObjectNames(ec.table) } - var tableIDs []pmodel.CIStr + var tableIDs []ast.CIStr if ec.tableID != "" { tableIDs = e.getSchemaObjectNames(ec.tableID) if len(tableIDs) > 0 { @@ -610,8 +610,8 @@ func NewInfoSchemaSequenceExtractor() *InfoSchemaSequenceExtractor { // findTablesByID finds tables by table IDs and append them to table map. func findTablesByID( is infoschema.InfoSchema, - tableIDs []pmodel.CIStr, - tableNames []pmodel.CIStr, + tableIDs []ast.CIStr, + tableNames []ast.CIStr, tables map[int64]*model.TableInfo, ) { tblNameMap := make(map[string]struct{}, len(tableNames)) @@ -640,8 +640,8 @@ func findTablesByID( // findTablesByPartID finds tables by partition IDs and append them to table map. func findTablesByPartID( is infoschema.InfoSchema, - partIDs []pmodel.CIStr, - tableNames []pmodel.CIStr, + partIDs []ast.CIStr, + tableNames []ast.CIStr, tables map[int64]*model.TableInfo, ) { tblNameMap := make(map[string]struct{}, len(tableNames)) @@ -667,11 +667,11 @@ func findTablesByPartID( func findTableAndSchemaByName( ctx context.Context, is infoschema.InfoSchema, - schemas []pmodel.CIStr, - tableNames []pmodel.CIStr, -) ([]pmodel.CIStr, []*model.TableInfo, error) { + schemas []ast.CIStr, + tableNames []ast.CIStr, +) ([]ast.CIStr, []*model.TableInfo, error) { type schemaAndTable struct { - schema pmodel.CIStr + schema ast.CIStr table *model.TableInfo } tableMap := make(map[int64]schemaAndTable, len(tableNames)) @@ -692,7 +692,7 @@ func findTableAndSchemaByName( tableMap[tblInfo.ID] = schemaAndTable{s, tblInfo} } } - schemaSlice := make([]pmodel.CIStr, 0, len(tableMap)) + schemaSlice := make([]ast.CIStr, 0, len(tableMap)) tableSlice := make([]*model.TableInfo, 0, len(tableMap)) for _, st := range tableMap { schemaSlice = append(schemaSlice, st.schema) @@ -714,10 +714,10 @@ func listTablesForEachSchema( ctx context.Context, e *InfoSchemaBaseExtractor, is infoschema.InfoSchema, - schemas []pmodel.CIStr, -) ([]pmodel.CIStr, []*model.TableInfo, error) { + schemas []ast.CIStr, +) ([]ast.CIStr, []*model.TableInfo, error) { ec := e.extractableColumns - schemaSlice := make([]pmodel.CIStr, 0, 8) + schemaSlice := make([]ast.CIStr, 0, 8) tableSlice := make([]*model.TableInfo, 0, 8) for _, s := range schemas { tables, err := is.SchemaTableInfos(ctx, s) @@ -743,9 +743,9 @@ func findSchemasForTables( e *InfoSchemaBaseExtractor, is infoschema.InfoSchema, tableSlice []*model.TableInfo, -) ([]pmodel.CIStr, []*model.TableInfo, error) { +) ([]ast.CIStr, []*model.TableInfo, error) { schemas, unspecified := e.listPredicateSchemas(is) - schemaSlice := make([]pmodel.CIStr, 0, len(tableSlice)) + schemaSlice := make([]ast.CIStr, 0, len(tableSlice)) for i, tbl := range tableSlice { dbInfo, ok := is.SchemaByID(tbl.DBID) intest.Assert(ok) @@ -790,7 +790,7 @@ func findSchemasForTables( return schemaSlice, remains, nil } -func parseIDs(ids []pmodel.CIStr) []int64 { +func parseIDs(ids []ast.CIStr) []int64 { tableIDs := make([]int64, 0, len(ids)) for _, s := range ids { v, err := strconv.ParseInt(s.L, 10, 64) @@ -804,14 +804,14 @@ func parseIDs(ids []pmodel.CIStr) []int64 { } // getSchemaObjectNames gets the schema object names specified in predicate of given column name. -func (e *InfoSchemaBaseExtractor) getSchemaObjectNames(colName string) []pmodel.CIStr { +func (e *InfoSchemaBaseExtractor) getSchemaObjectNames(colName string) []ast.CIStr { predVals, ok := e.ColPredicates[colName] if ok && len(predVals) > 0 { - objNames := make([]pmodel.CIStr, 0, len(predVals)) + objNames := make([]ast.CIStr, 0, len(predVals)) predVals.IterateWith(func(n string) { - objNames = append(objNames, pmodel.NewCIStr(n)) + objNames = append(objNames, ast.NewCIStr(n)) }) - slices.SortFunc(objNames, func(a, b pmodel.CIStr) int { + slices.SortFunc(objNames, func(a, b ast.CIStr) int { return strings.Compare(a.L, b.L) }) return objNames @@ -819,7 +819,7 @@ func (e *InfoSchemaBaseExtractor) getSchemaObjectNames(colName string) []pmodel. return nil } -func extractStrCIStr(str pmodel.CIStr) string { +func extractStrCIStr(str ast.CIStr) string { return str.O } @@ -874,13 +874,13 @@ func NewInfoSchemaColumnsExtractor() *InfoSchemaColumnsExtractor { // TODO(tangenta): remove this after streaming interface is supported. func (e *InfoSchemaColumnsExtractor) ListTables( ctx context.Context, - s pmodel.CIStr, + s ast.CIStr, is infoschema.InfoSchema, ) ([]*model.TableInfo, error) { ec := e.extractableColumns base := e.GetBase() - schemas := []pmodel.CIStr{s} - var tableNames []pmodel.CIStr + schemas := []ast.CIStr{s} + var tableNames []ast.CIStr if ec.table != "" { tableNames = e.getSchemaObjectNames(ec.table) } diff --git a/pkg/planner/core/mock.go b/pkg/planner/core/mock.go index 122741adc8c6a..72fef5e690e6b 100644 --- a/pkg/planner/core/mock.go +++ b/pkg/planner/core/mock.go @@ -22,8 +22,8 @@ import ( "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/meta/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/auth" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/types" @@ -55,20 +55,20 @@ func MockSignedTable() *model.TableInfo { indices := []*model.IndexInfo{ { ID: 1, - Name: pmodel.NewCIStr("c_d_e"), + Name: ast.NewCIStr("c_d_e"), Columns: []*model.IndexColumn{ { - Name: pmodel.NewCIStr("c"), + Name: ast.NewCIStr("c"), Length: types.UnspecifiedLength, Offset: 2, }, { - Name: pmodel.NewCIStr("d"), + Name: ast.NewCIStr("d"), Length: types.UnspecifiedLength, Offset: 3, }, { - Name: pmodel.NewCIStr("e"), + Name: ast.NewCIStr("e"), Length: types.UnspecifiedLength, Offset: 4, }, @@ -78,10 +78,10 @@ func MockSignedTable() *model.TableInfo { }, { ID: 2, - Name: pmodel.NewCIStr("x"), + Name: ast.NewCIStr("x"), Columns: []*model.IndexColumn{ { - Name: pmodel.NewCIStr("e"), + Name: ast.NewCIStr("e"), Length: types.UnspecifiedLength, Offset: 4, }, @@ -91,10 +91,10 @@ func MockSignedTable() *model.TableInfo { }, { ID: 3, - Name: pmodel.NewCIStr("f"), + Name: ast.NewCIStr("f"), Columns: []*model.IndexColumn{ { - Name: pmodel.NewCIStr("f"), + Name: ast.NewCIStr("f"), Length: types.UnspecifiedLength, Offset: 8, }, @@ -104,10 +104,10 @@ func MockSignedTable() *model.TableInfo { }, { ID: 4, - Name: pmodel.NewCIStr("g"), + Name: ast.NewCIStr("g"), Columns: []*model.IndexColumn{ { - Name: pmodel.NewCIStr("g"), + Name: ast.NewCIStr("g"), Length: types.UnspecifiedLength, Offset: 9, }, @@ -116,15 +116,15 @@ func MockSignedTable() *model.TableInfo { }, { ID: 5, - Name: pmodel.NewCIStr("f_g"), + Name: ast.NewCIStr("f_g"), Columns: []*model.IndexColumn{ { - Name: pmodel.NewCIStr("f"), + Name: ast.NewCIStr("f"), Length: types.UnspecifiedLength, Offset: 8, }, { - Name: pmodel.NewCIStr("g"), + Name: ast.NewCIStr("g"), Length: types.UnspecifiedLength, Offset: 9, }, @@ -134,20 +134,20 @@ func MockSignedTable() *model.TableInfo { }, { ID: 6, - Name: pmodel.NewCIStr("c_d_e_str"), + Name: ast.NewCIStr("c_d_e_str"), Columns: []*model.IndexColumn{ { - Name: pmodel.NewCIStr("c_str"), + Name: ast.NewCIStr("c_str"), Length: types.UnspecifiedLength, Offset: 5, }, { - Name: pmodel.NewCIStr("d_str"), + Name: ast.NewCIStr("d_str"), Length: types.UnspecifiedLength, Offset: 6, }, { - Name: pmodel.NewCIStr("e_str"), + Name: ast.NewCIStr("e_str"), Length: types.UnspecifiedLength, Offset: 7, }, @@ -156,20 +156,20 @@ func MockSignedTable() *model.TableInfo { }, { ID: 7, - Name: pmodel.NewCIStr("e_d_c_str_prefix"), + Name: ast.NewCIStr("e_d_c_str_prefix"), Columns: []*model.IndexColumn{ { - Name: pmodel.NewCIStr("e_str"), + Name: ast.NewCIStr("e_str"), Length: types.UnspecifiedLength, Offset: 7, }, { - Name: pmodel.NewCIStr("d_str"), + Name: ast.NewCIStr("d_str"), Length: types.UnspecifiedLength, Offset: 6, }, { - Name: pmodel.NewCIStr("c_str"), + Name: ast.NewCIStr("c_str"), Length: 10, Offset: 5, }, @@ -180,84 +180,84 @@ func MockSignedTable() *model.TableInfo { pkColumn := &model.ColumnInfo{ State: model.StatePublic, Offset: 0, - Name: pmodel.NewCIStr("a"), + Name: ast.NewCIStr("a"), FieldType: newLongType(), ID: 1, } col0 := &model.ColumnInfo{ State: model.StatePublic, Offset: 1, - Name: pmodel.NewCIStr("b"), + Name: ast.NewCIStr("b"), FieldType: newLongType(), ID: 2, } col1 := &model.ColumnInfo{ State: model.StatePublic, Offset: 2, - Name: pmodel.NewCIStr("c"), + Name: ast.NewCIStr("c"), FieldType: newLongType(), ID: 3, } col2 := &model.ColumnInfo{ State: model.StatePublic, Offset: 3, - Name: pmodel.NewCIStr("d"), + Name: ast.NewCIStr("d"), FieldType: newLongType(), ID: 4, } col3 := &model.ColumnInfo{ State: model.StatePublic, Offset: 4, - Name: pmodel.NewCIStr("e"), + Name: ast.NewCIStr("e"), FieldType: newLongType(), ID: 5, } colStr1 := &model.ColumnInfo{ State: model.StatePublic, Offset: 5, - Name: pmodel.NewCIStr("c_str"), + Name: ast.NewCIStr("c_str"), FieldType: newStringType(), ID: 6, } colStr2 := &model.ColumnInfo{ State: model.StatePublic, Offset: 6, - Name: pmodel.NewCIStr("d_str"), + Name: ast.NewCIStr("d_str"), FieldType: newStringType(), ID: 7, } colStr3 := &model.ColumnInfo{ State: model.StatePublic, Offset: 7, - Name: pmodel.NewCIStr("e_str"), + Name: ast.NewCIStr("e_str"), FieldType: newStringType(), ID: 8, } col4 := &model.ColumnInfo{ State: model.StatePublic, Offset: 8, - Name: pmodel.NewCIStr("f"), + Name: ast.NewCIStr("f"), FieldType: newLongType(), ID: 9, } col5 := &model.ColumnInfo{ State: model.StatePublic, Offset: 9, - Name: pmodel.NewCIStr("g"), + Name: ast.NewCIStr("g"), FieldType: newLongType(), ID: 10, } col6 := &model.ColumnInfo{ State: model.StatePublic, Offset: 10, - Name: pmodel.NewCIStr("h"), + Name: ast.NewCIStr("h"), FieldType: newLongType(), ID: 11, } col7 := &model.ColumnInfo{ State: model.StatePublic, Offset: 11, - Name: pmodel.NewCIStr("i_date"), + Name: ast.NewCIStr("i_date"), FieldType: newDateType(), ID: 12, } @@ -273,7 +273,7 @@ func MockSignedTable() *model.TableInfo { ID: 1, Columns: []*model.ColumnInfo{pkColumn, col0, col1, col2, col3, colStr1, colStr2, colStr3, col4, col5, col6, col7}, Indices: indices, - Name: pmodel.NewCIStr("t"), + Name: ast.NewCIStr("t"), PKIsHandle: true, State: model.StatePublic, } @@ -287,10 +287,10 @@ func MockUnsignedTable() *model.TableInfo { // indeices: b, b_c indices := []*model.IndexInfo{ { - Name: pmodel.NewCIStr("b"), + Name: ast.NewCIStr("b"), Columns: []*model.IndexColumn{ { - Name: pmodel.NewCIStr("b"), + Name: ast.NewCIStr("b"), Length: types.UnspecifiedLength, Offset: 1, }, @@ -299,15 +299,15 @@ func MockUnsignedTable() *model.TableInfo { Unique: true, }, { - Name: pmodel.NewCIStr("b_c"), + Name: ast.NewCIStr("b_c"), Columns: []*model.IndexColumn{ { - Name: pmodel.NewCIStr("b"), + Name: ast.NewCIStr("b"), Length: types.UnspecifiedLength, Offset: 1, }, { - Name: pmodel.NewCIStr("c"), + Name: ast.NewCIStr("c"), Length: types.UnspecifiedLength, Offset: 2, }, @@ -318,21 +318,21 @@ func MockUnsignedTable() *model.TableInfo { pkColumn := &model.ColumnInfo{ State: model.StatePublic, Offset: 0, - Name: pmodel.NewCIStr("a"), + Name: ast.NewCIStr("a"), FieldType: newLongType(), ID: 1, } col0 := &model.ColumnInfo{ State: model.StatePublic, Offset: 1, - Name: pmodel.NewCIStr("b"), + Name: ast.NewCIStr("b"), FieldType: newLongType(), ID: 2, } col1 := &model.ColumnInfo{ State: model.StatePublic, Offset: 2, - Name: pmodel.NewCIStr("c"), + Name: ast.NewCIStr("c"), FieldType: newLongType(), ID: 3, } @@ -344,7 +344,7 @@ func MockUnsignedTable() *model.TableInfo { ID: 2, Columns: []*model.ColumnInfo{pkColumn, col0, col1}, Indices: indices, - Name: pmodel.NewCIStr("t2"), + Name: ast.NewCIStr("t2"), PKIsHandle: true, State: model.StatePublic, } @@ -357,14 +357,14 @@ func MockNoPKTable() *model.TableInfo { col0 := &model.ColumnInfo{ State: model.StatePublic, Offset: 1, - Name: pmodel.NewCIStr("a"), + Name: ast.NewCIStr("a"), FieldType: newLongType(), ID: 2, } col1 := &model.ColumnInfo{ State: model.StatePublic, Offset: 2, - Name: pmodel.NewCIStr("b"), + Name: ast.NewCIStr("b"), FieldType: newLongType(), ID: 3, } @@ -374,7 +374,7 @@ func MockNoPKTable() *model.TableInfo { table := &model.TableInfo{ ID: 3, Columns: []*model.ColumnInfo{col0, col1}, - Name: pmodel.NewCIStr("t3"), + Name: ast.NewCIStr("t3"), PKIsHandle: true, State: model.StatePublic, } @@ -387,25 +387,25 @@ func MockView() *model.TableInfo { col0 := &model.ColumnInfo{ State: model.StatePublic, Offset: 0, - Name: pmodel.NewCIStr("b"), + Name: ast.NewCIStr("b"), ID: 1, } col1 := &model.ColumnInfo{ State: model.StatePublic, Offset: 1, - Name: pmodel.NewCIStr("c"), + Name: ast.NewCIStr("c"), ID: 2, } col2 := &model.ColumnInfo{ State: model.StatePublic, Offset: 2, - Name: pmodel.NewCIStr("d"), + Name: ast.NewCIStr("d"), ID: 3, } - view := &model.ViewInfo{SelectStmt: selectStmt, Security: pmodel.SecurityDefiner, Definer: &auth.UserIdentity{Username: "root", Hostname: ""}, Cols: []pmodel.CIStr{col0.Name, col1.Name, col2.Name}} + view := &model.ViewInfo{SelectStmt: selectStmt, Security: ast.SecurityDefiner, Definer: &auth.UserIdentity{Username: "root", Hostname: ""}, Cols: []ast.CIStr{col0.Name, col1.Name, col2.Name}} table := &model.TableInfo{ ID: 4, - Name: pmodel.NewCIStr("v"), + Name: ast.NewCIStr("v"), Columns: []*model.ColumnInfo{col0, col1, col2}, View: view, State: model.StatePublic, @@ -442,12 +442,12 @@ func MockPartitionInfoSchema(definitions []model.PartitionDefinition) infoschema cols = append(cols, &model.ColumnInfo{ State: model.StatePublic, Offset: last.Offset + 1, - Name: pmodel.NewCIStr("ptn"), + Name: ast.NewCIStr("ptn"), FieldType: newLongType(), ID: last.ID + 1, }) partition := &model.PartitionInfo{ - Type: pmodel.PartitionTypeRange, + Type: ast.PartitionTypeRange, Expr: "ptn", Enable: true, Definitions: definitions, @@ -463,30 +463,30 @@ func MockRangePartitionTable() *model.TableInfo { definitions := []model.PartitionDefinition{ { ID: 41, - Name: pmodel.NewCIStr("p1"), + Name: ast.NewCIStr("p1"), LessThan: []string{"16"}, }, { ID: 42, - Name: pmodel.NewCIStr("p2"), + Name: ast.NewCIStr("p2"), LessThan: []string{"32"}, }, } tableInfo := MockSignedTable() tableInfo.ID = 5 - tableInfo.Name = pmodel.NewCIStr("pt1") + tableInfo.Name = ast.NewCIStr("pt1") cols := make([]*model.ColumnInfo, 0, len(tableInfo.Columns)) cols = append(cols, tableInfo.Columns...) last := tableInfo.Columns[len(tableInfo.Columns)-1] cols = append(cols, &model.ColumnInfo{ State: model.StatePublic, Offset: last.Offset + 1, - Name: pmodel.NewCIStr("ptn"), + Name: ast.NewCIStr("ptn"), FieldType: newLongType(), ID: last.ID + 1, }) partition := &model.PartitionInfo{ - Type: pmodel.PartitionTypeRange, + Type: ast.PartitionTypeRange, Expr: "ptn", Enable: true, Definitions: definitions, @@ -501,28 +501,28 @@ func MockHashPartitionTable() *model.TableInfo { definitions := []model.PartitionDefinition{ { ID: 51, - Name: pmodel.NewCIStr("p1"), + Name: ast.NewCIStr("p1"), }, { ID: 52, - Name: pmodel.NewCIStr("p2"), + Name: ast.NewCIStr("p2"), }, } tableInfo := MockSignedTable() tableInfo.ID = 6 - tableInfo.Name = pmodel.NewCIStr("pt2") + tableInfo.Name = ast.NewCIStr("pt2") cols := make([]*model.ColumnInfo, 0, len(tableInfo.Columns)) cols = append(cols, tableInfo.Columns...) last := tableInfo.Columns[len(tableInfo.Columns)-1] cols = append(cols, &model.ColumnInfo{ State: model.StatePublic, Offset: last.Offset + 1, - Name: pmodel.NewCIStr("ptn"), + Name: ast.NewCIStr("ptn"), FieldType: newLongType(), ID: last.ID + 1, }) partition := &model.PartitionInfo{ - Type: pmodel.PartitionTypeHash, + Type: ast.PartitionTypeHash, Expr: "ptn", Enable: true, Definitions: definitions, @@ -538,7 +538,7 @@ func MockListPartitionTable() *model.TableInfo { definitions := []model.PartitionDefinition{ { ID: 61, - Name: pmodel.NewCIStr("p1"), + Name: ast.NewCIStr("p1"), InValues: [][]string{ { "1", @@ -547,7 +547,7 @@ func MockListPartitionTable() *model.TableInfo { }, { ID: 62, - Name: pmodel.NewCIStr("p2"), + Name: ast.NewCIStr("p2"), InValues: [][]string{ { "2", @@ -557,19 +557,19 @@ func MockListPartitionTable() *model.TableInfo { } tableInfo := MockSignedTable() tableInfo.ID = 7 - tableInfo.Name = pmodel.NewCIStr("pt3") + tableInfo.Name = ast.NewCIStr("pt3") cols := make([]*model.ColumnInfo, 0, len(tableInfo.Columns)) cols = append(cols, tableInfo.Columns...) last := tableInfo.Columns[len(tableInfo.Columns)-1] cols = append(cols, &model.ColumnInfo{ State: model.StatePublic, Offset: last.Offset + 1, - Name: pmodel.NewCIStr("ptn"), + Name: ast.NewCIStr("ptn"), FieldType: newLongType(), ID: last.ID + 1, }) partition := &model.PartitionInfo{ - Type: pmodel.PartitionTypeList, + Type: ast.PartitionTypeList, Expr: "ptn", Enable: true, Definitions: definitions, @@ -585,27 +585,27 @@ func MockGlobalIndexHashPartitionTable() *model.TableInfo { definitions := []model.PartitionDefinition{ { ID: 51, - Name: pmodel.NewCIStr("p1"), + Name: ast.NewCIStr("p1"), }, { ID: 52, - Name: pmodel.NewCIStr("p2"), + Name: ast.NewCIStr("p2"), }, } tableInfo := MockSignedTable() - tableInfo.Name = pmodel.NewCIStr("pt2_global_index") + tableInfo.Name = ast.NewCIStr("pt2_global_index") cols := make([]*model.ColumnInfo, 0, len(tableInfo.Columns)) cols = append(cols, tableInfo.Columns...) last := tableInfo.Columns[len(tableInfo.Columns)-1] cols = append(cols, &model.ColumnInfo{ State: model.StatePublic, Offset: last.Offset + 1, - Name: pmodel.NewCIStr("ptn"), + Name: ast.NewCIStr("ptn"), FieldType: newLongType(), ID: last.ID + 1, }) partition := &model.PartitionInfo{ - Type: pmodel.PartitionTypeHash, + Type: ast.PartitionTypeHash, Expr: "ptn", Enable: true, Definitions: definitions, @@ -616,10 +616,10 @@ func MockGlobalIndexHashPartitionTable() *model.TableInfo { // add a global index `b_global` and `b_c_global` and normal index `b` and `b_c` tableInfo.Indices = append(tableInfo.Indices, []*model.IndexInfo{ { - Name: pmodel.NewCIStr("b"), + Name: ast.NewCIStr("b"), Columns: []*model.IndexColumn{ { - Name: pmodel.NewCIStr("b"), + Name: ast.NewCIStr("b"), Length: types.UnspecifiedLength, Offset: 1, }, @@ -627,10 +627,10 @@ func MockGlobalIndexHashPartitionTable() *model.TableInfo { State: model.StatePublic, }, { - Name: pmodel.NewCIStr("b_global"), + Name: ast.NewCIStr("b_global"), Columns: []*model.IndexColumn{ { - Name: pmodel.NewCIStr("b"), + Name: ast.NewCIStr("b"), Length: types.UnspecifiedLength, Offset: 1, }, @@ -640,15 +640,15 @@ func MockGlobalIndexHashPartitionTable() *model.TableInfo { Global: true, }, { - Name: pmodel.NewCIStr("b_c"), + Name: ast.NewCIStr("b_c"), Columns: []*model.IndexColumn{ { - Name: pmodel.NewCIStr("b"), + Name: ast.NewCIStr("b"), Length: types.UnspecifiedLength, Offset: 1, }, { - Name: pmodel.NewCIStr("c"), + Name: ast.NewCIStr("c"), Length: types.UnspecifiedLength, Offset: 2, }, @@ -656,15 +656,15 @@ func MockGlobalIndexHashPartitionTable() *model.TableInfo { State: model.StatePublic, }, { - Name: pmodel.NewCIStr("b_c_global"), + Name: ast.NewCIStr("b_c_global"), Columns: []*model.IndexColumn{ { - Name: pmodel.NewCIStr("b"), + Name: ast.NewCIStr("b"), Length: types.UnspecifiedLength, Offset: 1, }, { - Name: pmodel.NewCIStr("c"), + Name: ast.NewCIStr("c"), Length: types.UnspecifiedLength, Offset: 2, }, @@ -684,10 +684,10 @@ func MockStateNoneColumnTable() *model.TableInfo { // indeices: b indices := []*model.IndexInfo{ { - Name: pmodel.NewCIStr("b"), + Name: ast.NewCIStr("b"), Columns: []*model.IndexColumn{ { - Name: pmodel.NewCIStr("b"), + Name: ast.NewCIStr("b"), Length: types.UnspecifiedLength, Offset: 1, }, @@ -699,21 +699,21 @@ func MockStateNoneColumnTable() *model.TableInfo { pkColumn := &model.ColumnInfo{ State: model.StatePublic, Offset: 0, - Name: pmodel.NewCIStr("a"), + Name: ast.NewCIStr("a"), FieldType: newLongType(), ID: 1, } col0 := &model.ColumnInfo{ State: model.StatePublic, Offset: 1, - Name: pmodel.NewCIStr("b"), + Name: ast.NewCIStr("b"), FieldType: newLongType(), ID: 2, } col1 := &model.ColumnInfo{ State: model.StateNone, Offset: 2, - Name: pmodel.NewCIStr("c"), + Name: ast.NewCIStr("c"), FieldType: newLongType(), ID: 3, } @@ -724,7 +724,7 @@ func MockStateNoneColumnTable() *model.TableInfo { ID: 8, Columns: []*model.ColumnInfo{pkColumn, col0, col1}, Indices: indices, - Name: pmodel.NewCIStr("T_StateNoneColumn"), + Name: ast.NewCIStr("T_StateNoneColumn"), PKIsHandle: true, State: model.StatePublic, } diff --git a/pkg/planner/core/operator/logicalop/BUILD.bazel b/pkg/planner/core/operator/logicalop/BUILD.bazel index 381d64d540dfd..7c09c69e9436d 100644 --- a/pkg/planner/core/operator/logicalop/BUILD.bazel +++ b/pkg/planner/core/operator/logicalop/BUILD.bazel @@ -44,7 +44,6 @@ go_library( "//pkg/meta/model", "//pkg/parser/ast", "//pkg/parser/auth", - "//pkg/parser/model", "//pkg/parser/mysql", "//pkg/planner/cardinality", "//pkg/planner/cascades/base", diff --git a/pkg/planner/core/operator/logicalop/logical_cte.go b/pkg/planner/core/operator/logicalop/logical_cte.go index c2be53b743a12..6aad3703d083f 100644 --- a/pkg/planner/core/operator/logicalop/logical_cte.go +++ b/pkg/planner/core/operator/logicalop/logical_cte.go @@ -19,7 +19,7 @@ import ( "unsafe" "github.com/pingcap/tidb/pkg/expression" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/planner/cardinality" "github.com/pingcap/tidb/pkg/planner/core/base" ruleutil "github.com/pingcap/tidb/pkg/planner/core/rule/util" @@ -36,8 +36,8 @@ type LogicalCTE struct { LogicalSchemaProducer Cte *CTEClass - CteAsName model.CIStr - CteName model.CIStr + CteAsName ast.CIStr + CteName ast.CIStr SeedStat *property.StatsInfo OnlyUsedAsStorage bool diff --git a/pkg/planner/core/operator/logicalop/logical_datasource.go b/pkg/planner/core/operator/logicalop/logical_datasource.go index 1f796cb919760..25acc8ebd32c6 100644 --- a/pkg/planner/core/operator/logicalop/logical_datasource.go +++ b/pkg/planner/core/operator/logicalop/logical_datasource.go @@ -24,7 +24,6 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/ast" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/core/constraint" @@ -54,9 +53,9 @@ type DataSource struct { Table table.Table TableInfo *model.TableInfo `hash64-equals:"true"` Columns []*model.ColumnInfo - DBName pmodel.CIStr + DBName ast.CIStr - TableAsName *pmodel.CIStr `hash64-equals:"true"` + TableAsName *ast.CIStr `hash64-equals:"true"` // IndexMergeHints are the hint for indexmerge. IndexMergeHints []h.HintedIndex // PushedDownConds are the conditions that will be pushed down to coprocessor. @@ -74,7 +73,7 @@ type DataSource struct { // The data source may be a partition, rather than a real table. PartitionDefIdx *int PhysicalTableID int64 - PartitionNames []pmodel.CIStr + PartitionNames []ast.CIStr // handleCol represents the handle column for the datasource, either the // int primary key column or extra handle column. @@ -93,7 +92,7 @@ type DataSource struct { // PreferStoreType means the DataSource is enforced to which storage. PreferStoreType int `hash64-equals:"true"` // PreferPartitions store the map, the key represents store type, the value represents the partition name list. - PreferPartitions map[int][]pmodel.CIStr + PreferPartitions map[int][]ast.CIStr SampleInfo *tablesampler.TableSampleInfo IS infoschema.InfoSchema // IsForUpdateRead should be true in either of the following situations diff --git a/pkg/planner/core/operator/logicalop/logical_mem_table.go b/pkg/planner/core/operator/logicalop/logical_mem_table.go index 60715c3ae8221..8bd0d5902d42a 100644 --- a/pkg/planner/core/operator/logicalop/logical_mem_table.go +++ b/pkg/planner/core/operator/logicalop/logical_mem_table.go @@ -19,7 +19,7 @@ import ( "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/property" "github.com/pingcap/tidb/pkg/planner/util" @@ -43,7 +43,7 @@ type LogicalMemTable struct { LogicalSchemaProducer `hash64-equals:"true"` Extractor base.MemTablePredicateExtractor - DBName pmodel.CIStr `hash64-equals:"true"` + DBName ast.CIStr `hash64-equals:"true"` TableInfo *model.TableInfo `hash64-equals:"true"` Columns []*model.ColumnInfo // QueryTimeRange is used to specify the time range for metrics summary tables and inspection tables diff --git a/pkg/planner/core/operator/logicalop/logical_show.go b/pkg/planner/core/operator/logicalop/logical_show.go index d79b253ba1aba..03fcbc8b1e675 100644 --- a/pkg/planner/core/operator/logicalop/logical_show.go +++ b/pkg/planner/core/operator/logicalop/logical_show.go @@ -20,7 +20,6 @@ import ( "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/auth" - "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/core/resolve" "github.com/pingcap/tidb/pkg/planner/property" @@ -43,9 +42,9 @@ type ShowContents struct { Tp ast.ShowStmtType // Databases/Tables/Columns/.... DBName string Table *resolve.TableNameW // Used for showing columns. - Partition model.CIStr // Use for showing partition + Partition ast.CIStr // Use for showing partition Column *ast.ColumnName // Used for `desc table column`. - IndexName model.CIStr + IndexName ast.CIStr ResourceGroupName string // Used for showing resource group Flag int // Some flag parsed from sql, such as FULL. User *auth.UserIdentity // Used for show grants. diff --git a/pkg/planner/core/operator/logicalop/logicalop_test/BUILD.bazel b/pkg/planner/core/operator/logicalop/logicalop_test/BUILD.bazel index 85ab42abff2fa..8b24e7c700c8d 100644 --- a/pkg/planner/core/operator/logicalop/logicalop_test/BUILD.bazel +++ b/pkg/planner/core/operator/logicalop/logicalop_test/BUILD.bazel @@ -16,7 +16,6 @@ go_test( "//pkg/meta/model", "//pkg/parser", "//pkg/parser/ast", - "//pkg/parser/model", "//pkg/parser/mysql", "//pkg/planner", "//pkg/planner/cascades/base", diff --git a/pkg/planner/core/operator/logicalop/logicalop_test/hash64_equals_test.go b/pkg/planner/core/operator/logicalop/logicalop_test/hash64_equals_test.go index 160283443d590..9bea997a55608 100644 --- a/pkg/planner/core/operator/logicalop/logicalop_test/hash64_equals_test.go +++ b/pkg/planner/core/operator/logicalop/logicalop_test/hash64_equals_test.go @@ -21,7 +21,6 @@ import ( "github.com/pingcap/tidb/pkg/expression/aggregation" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/ast" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/cascades/base" "github.com/pingcap/tidb/pkg/planner/core/operator/logicalop" @@ -444,13 +443,13 @@ func TestLogicalMemTableHash64Equals(t *testing.T) { require.False(t, m1.Equals(m2)) m2.LogicalSchemaProducer.SetSchema(&expression.Schema{Columns: []*expression.Column{col1}}) - m2.DBName = pmodel.NewCIStr("d1") + m2.DBName = ast.NewCIStr("d1") hasher2.Reset() m2.Hash64(hasher2) require.NotEqual(t, hasher1.Sum64(), hasher2.Sum64()) require.False(t, m1.Equals(m2)) - m2.DBName = pmodel.NewCIStr("") + m2.DBName = ast.NewCIStr("") m2.TableInfo = &model.TableInfo{ID: 1} hasher2.Reset() m2.Hash64(hasher2) diff --git a/pkg/planner/core/optimizer.go b/pkg/planner/core/optimizer.go index 6f58ff3dea66b..1fcc5ba581706 100644 --- a/pkg/planner/core/optimizer.go +++ b/pkg/planner/core/optimizer.go @@ -36,7 +36,6 @@ import ( "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/auth" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/core/operator/logicalop" @@ -218,7 +217,7 @@ func VisitInfo4PrivCheck(ctx context.Context, is infoschema.InfoSchema, node ast func needCheckTmpTablePriv(ctx context.Context, is infoschema.InfoSchema, v visitInfo) bool { if v.db != "" && v.table != "" { // Other statements on local temporary tables except `CREATE` do not check any privileges. - tb, err := is.TableByName(ctx, pmodel.NewCIStr(v.db), pmodel.NewCIStr(v.table)) + tb, err := is.TableByName(ctx, ast.NewCIStr(v.db), ast.NewCIStr(v.table)) // If the table doesn't exist, we do not report errors to avoid leaking the existence of the table. if err == nil && tb.Meta().TempTableType == model.TempTableLocal { return false diff --git a/pkg/planner/core/partition_prune.go b/pkg/planner/core/partition_prune.go index 194c1fb2b2d4b..81f51a6cfcae5 100644 --- a/pkg/planner/core/partition_prune.go +++ b/pkg/planner/core/partition_prune.go @@ -17,7 +17,7 @@ package core import ( "github.com/pingcap/tidb/pkg/expression" tmodel "github.com/pingcap/tidb/pkg/meta/model" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/types" @@ -26,14 +26,14 @@ import ( // PartitionPruning finds all used partitions according to query conditions, it will // return nil if condition match none of partitions. The return value is a array of the // idx in the partition definitions array, use pi.Definitions[idx] to get the partition ID -func PartitionPruning(ctx base.PlanContext, tbl table.PartitionedTable, conds []expression.Expression, partitionNames []model.CIStr, +func PartitionPruning(ctx base.PlanContext, tbl table.PartitionedTable, conds []expression.Expression, partitionNames []ast.CIStr, columns []*expression.Column, names types.NameSlice) ([]int, error) { s := PartitionProcessor{} pi := tbl.Meta().Partition switch pi.Type { - case model.PartitionTypeHash, model.PartitionTypeKey: + case ast.PartitionTypeHash, ast.PartitionTypeKey: return s.pruneHashOrKeyPartition(ctx, tbl, partitionNames, conds, columns, names) - case model.PartitionTypeRange: + case ast.PartitionTypeRange: rangeOr, err := s.pruneRangePartition(ctx, pi, tbl, conds, columns, names) if err != nil { return nil, err @@ -41,13 +41,13 @@ func PartitionPruning(ctx base.PlanContext, tbl table.PartitionedTable, conds [] ret := s.convertToIntSlice(rangeOr, pi, partitionNames) ret = handleDroppingForRange(pi, partitionNames, ret) return ret, nil - case model.PartitionTypeList: + case ast.PartitionTypeList: return s.pruneListPartition(ctx, tbl, partitionNames, conds, columns) } return []int{FullRange}, nil } -func handleDroppingForRange(pi *tmodel.PartitionInfo, partitionNames []model.CIStr, usedPartitions []int) []int { +func handleDroppingForRange(pi *tmodel.PartitionInfo, partitionNames []ast.CIStr, usedPartitions []int) []int { if pi.CanHaveOverlappingDroppingPartition() { if len(usedPartitions) == 1 && usedPartitions[0] == FullRange { usedPartitions = make([]int, 0, len(pi.Definitions)) diff --git a/pkg/planner/core/partition_pruning_test.go b/pkg/planner/core/partition_pruning_test.go index 67a21859123c4..c971c1c62d997 100644 --- a/pkg/planner/core/partition_pruning_test.go +++ b/pkg/planner/core/partition_pruning_test.go @@ -23,7 +23,6 @@ import ( "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/parser" "github.com/pingcap/tidb/pkg/parser/ast" - "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/testkit/ddlhelper" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/mock" @@ -187,7 +186,7 @@ func prepareBenchCtx(createTable string, partitionExpr string) *testCtx { if err != nil { return nil } - columns, names, err := expression.ColumnInfos2ColumnsAndNames(sctx, model.NewCIStr("t"), tblInfo.Name, tblInfo.Cols(), tblInfo) + columns, names, err := expression.ColumnInfos2ColumnsAndNames(sctx, ast.NewCIStr("t"), tblInfo.Name, tblInfo.Cols(), tblInfo) if err != nil { return nil } @@ -215,7 +214,7 @@ func prepareTestCtx(t *testing.T, createTable string, partitionExpr string) *tes sctx := mock.NewContext() tblInfo, err := ddlhelper.BuildTableInfoFromASTForTest(stmt.(*ast.CreateTableStmt)) require.NoError(t, err) - columns, names, err := expression.ColumnInfos2ColumnsAndNames(sctx, model.NewCIStr("t"), tblInfo.Name, tblInfo.Cols(), tblInfo) + columns, names, err := expression.ColumnInfos2ColumnsAndNames(sctx, ast.NewCIStr("t"), tblInfo.Name, tblInfo.Cols(), tblInfo) require.NoError(t, err) schema := expression.NewSchema(columns...) diff --git a/pkg/planner/core/physical_plans.go b/pkg/planner/core/physical_plans.go index 40feb3d27d629..f71723ca910bd 100644 --- a/pkg/planner/core/physical_plans.go +++ b/pkg/planner/core/physical_plans.go @@ -26,7 +26,6 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/ast" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/cardinality" "github.com/pingcap/tidb/pkg/planner/core/base" @@ -166,7 +165,7 @@ func (p *PhysicalTableReader) LoadTableStats(ctx sessionctx.Context) { // PhysPlanPartInfo indicates partition helper info in physical plan. type PhysPlanPartInfo struct { PruningConds []expression.Expression - PartitionNames []pmodel.CIStr + PartitionNames []ast.CIStr Columns []*expression.Column ColumnNames types.NameSlice } @@ -737,9 +736,9 @@ type PhysicalIndexScan struct { IdxColLens []int Ranges []*ranger.Range `plan-cache-clone:"shallow"` Columns []*model.ColumnInfo `plan-cache-clone:"shallow"` - DBName pmodel.CIStr `plan-cache-clone:"shallow"` + DBName ast.CIStr `plan-cache-clone:"shallow"` - TableAsName *pmodel.CIStr `plan-cache-clone:"shallow"` + TableAsName *ast.CIStr `plan-cache-clone:"shallow"` // dataSourceSchema is the original schema of DataSource. The schema of index scan in KV and index reader in TiDB // will be different. The schema of index scan will decode all columns of index but the TiDB only need some of them. @@ -878,7 +877,7 @@ func AddExtraPhysTblIDColumn(sctx base.PlanContext, columns []*model.ColumnInfo, type PhysicalMemTable struct { physicalSchemaProducer - DBName pmodel.CIStr + DBName ast.CIStr Table *model.TableInfo Columns []*model.ColumnInfo Extractor base.MemTablePredicateExtractor @@ -910,10 +909,10 @@ type PhysicalTableScan struct { Table *model.TableInfo `plan-cache-clone:"shallow"` Columns []*model.ColumnInfo `plan-cache-clone:"shallow"` - DBName pmodel.CIStr `plan-cache-clone:"shallow"` + DBName ast.CIStr `plan-cache-clone:"shallow"` Ranges []*ranger.Range `plan-cache-clone:"shallow"` - TableAsName *pmodel.CIStr `plan-cache-clone:"shallow"` + TableAsName *ast.CIStr `plan-cache-clone:"shallow"` physicalTableID int64 @@ -2700,8 +2699,8 @@ type PhysicalCTE struct { SeedPlan base.PhysicalPlan RecurPlan base.PhysicalPlan CTE *logicalop.CTEClass - cteAsName pmodel.CIStr - cteName pmodel.CIStr + cteAsName ast.CIStr + cteName ast.CIStr readerReceiver *PhysicalExchangeReceiver storageSender *PhysicalExchangeSender diff --git a/pkg/planner/core/plan_cache_utils.go b/pkg/planner/core/plan_cache_utils.go index bf927cfacdb5e..47a465aaa726c 100644 --- a/pkg/planner/core/plan_cache_utils.go +++ b/pkg/planner/core/plan_cache_utils.go @@ -37,7 +37,6 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser" "github.com/pingcap/tidb/pkg/parser/ast" - "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/core/resolve" @@ -190,7 +189,7 @@ func GeneratePlanCacheStmtWithAST(ctx context.Context, sctx sessionctx.Context, } // Collect information for metadata lock. - dbName := make([]model.CIStr, 0, len(vars.StmtCtx.MDLRelatedTableIDs)) + dbName := make([]ast.CIStr, 0, len(vars.StmtCtx.MDLRelatedTableIDs)) tbls := make([]table.Table, 0, len(vars.StmtCtx.MDLRelatedTableIDs)) relateVersion := make(map[int64]uint64, len(vars.StmtCtx.MDLRelatedTableIDs)) for id := range vars.StmtCtx.MDLRelatedTableIDs { @@ -666,7 +665,7 @@ type PlanCacheStmt struct { StmtText string // dbName and tbls are used to add metadata lock. - dbName []model.CIStr + dbName []ast.CIStr tbls []table.Table } diff --git a/pkg/planner/core/plan_cacheable_checker_test.go b/pkg/planner/core/plan_cacheable_checker_test.go index 7f6ae8931e52d..f7380e9245d4e 100644 --- a/pkg/planner/core/plan_cacheable_checker_test.go +++ b/pkg/planner/core/plan_cacheable_checker_test.go @@ -24,7 +24,6 @@ import ( "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/parser" "github.com/pingcap/tidb/pkg/parser/ast" - "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/sessiontxn" @@ -123,7 +122,7 @@ func TestCacheable(t *testing.T) { tk.MustExec("create table t1(a int, b int) partition by range(a) ( partition p0 values less than (6), partition p1 values less than (11) )") tk.MustExec("create table t2(a int, b int) partition by hash(a) partitions 11") tk.MustExec("create table t3(a int, b int)") - tbl := &ast.TableName{Schema: model.NewCIStr("test"), Name: model.NewCIStr("t3")} + tbl := &ast.TableName{Schema: ast.NewCIStr("test"), Name: ast.NewCIStr("t3")} is := tk.Session().GetInfoSchema().(infoschema.InfoSchema) // test non-SelectStmt/-InsertStmt/-DeleteStmt/-UpdateStmt/-SetOprStmt var stmt ast.Node = &ast.ShowStmt{} @@ -155,11 +154,11 @@ func TestCacheable(t *testing.T) { require.True(t, core.Cacheable(stmt, is)) for funcName := range expression.UnCacheableFunctions { - whereExpr.FnName = model.NewCIStr(funcName) + whereExpr.FnName = ast.NewCIStr(funcName) require.False(t, core.Cacheable(stmt, is)) } - whereExpr.FnName = model.NewCIStr(ast.Rand) + whereExpr.FnName = ast.NewCIStr(ast.Rand) require.True(t, core.Cacheable(stmt, is)) stmt = &ast.DeleteStmt{ @@ -198,7 +197,7 @@ func TestCacheable(t *testing.T) { require.True(t, c) stmt.(*ast.DeleteStmt).TableHints = append(stmt.(*ast.DeleteStmt).TableHints, &ast.TableOptimizerHint{ - HintName: model.NewCIStr(hint.HintIgnorePlanCache), + HintName: ast.NewCIStr(hint.HintIgnorePlanCache), }) require.False(t, core.Cacheable(stmt, is)) @@ -211,11 +210,11 @@ func TestCacheable(t *testing.T) { require.True(t, core.Cacheable(stmt, is)) for funcName := range expression.UnCacheableFunctions { - whereExpr.FnName = model.NewCIStr(funcName) + whereExpr.FnName = ast.NewCIStr(funcName) require.False(t, core.Cacheable(stmt, is)) } - whereExpr.FnName = model.NewCIStr(ast.Rand) + whereExpr.FnName = ast.NewCIStr(ast.Rand) require.True(t, core.Cacheable(stmt, is)) stmt = &ast.UpdateStmt{ @@ -254,7 +253,7 @@ func TestCacheable(t *testing.T) { require.True(t, c) stmt.(*ast.UpdateStmt).TableHints = append(stmt.(*ast.UpdateStmt).TableHints, &ast.TableOptimizerHint{ - HintName: model.NewCIStr(hint.HintIgnorePlanCache), + HintName: ast.NewCIStr(hint.HintIgnorePlanCache), }) require.False(t, core.Cacheable(stmt, is)) @@ -266,11 +265,11 @@ func TestCacheable(t *testing.T) { require.True(t, core.Cacheable(stmt, is)) for funcName := range expression.UnCacheableFunctions { - whereExpr.FnName = model.NewCIStr(funcName) + whereExpr.FnName = ast.NewCIStr(funcName) require.False(t, core.Cacheable(stmt, is)) } - whereExpr.FnName = model.NewCIStr(ast.Rand) + whereExpr.FnName = ast.NewCIStr(ast.Rand) require.True(t, core.Cacheable(stmt, is)) stmt = &ast.SelectStmt{ @@ -319,7 +318,7 @@ func TestCacheable(t *testing.T) { require.True(t, core.Cacheable(stmt, is)) stmt.(*ast.SelectStmt).TableHints = append(stmt.(*ast.SelectStmt).TableHints, &ast.TableOptimizerHint{ - HintName: model.NewCIStr(hint.HintIgnorePlanCache), + HintName: ast.NewCIStr(hint.HintIgnorePlanCache), }) require.False(t, core.Cacheable(stmt, is)) @@ -328,8 +327,8 @@ func TestCacheable(t *testing.T) { // Partition table can not be cached. join := &ast.Join{ - Left: &ast.TableName{Schema: model.NewCIStr("test"), Name: model.NewCIStr("t1")}, - Right: &ast.TableName{Schema: model.NewCIStr("test"), Name: model.NewCIStr("t2")}, + Left: &ast.TableName{Schema: ast.NewCIStr("test"), Name: ast.NewCIStr("t1")}, + Right: &ast.TableName{Schema: ast.NewCIStr("test"), Name: ast.NewCIStr("t2")}, } stmt = &ast.SelectStmt{ From: &ast.TableRefsClause{ @@ -339,7 +338,7 @@ func TestCacheable(t *testing.T) { require.False(t, core.Cacheable(stmt, is)) join = &ast.Join{ - Left: &ast.TableName{Schema: model.NewCIStr("test"), Name: model.NewCIStr("t3")}, + Left: &ast.TableName{Schema: ast.NewCIStr("test"), Name: ast.NewCIStr("t3")}, } stmt = &ast.SelectStmt{ From: &ast.TableRefsClause{ diff --git a/pkg/planner/core/plan_cost_ver1_test.go b/pkg/planner/core/plan_cost_ver1_test.go index 496a916e1f963..25bdfab91c841 100644 --- a/pkg/planner/core/plan_cost_ver1_test.go +++ b/pkg/planner/core/plan_cost_ver1_test.go @@ -22,7 +22,7 @@ import ( "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/testkit" "github.com/stretchr/testify/require" ) @@ -109,7 +109,7 @@ func TestScanOnSmallTable(t *testing.T) { // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - db, exists := is.SchemaByName(pmodel.NewCIStr("test")) + db, exists := is.SchemaByName(ast.NewCIStr("test")) require.True(t, exists) tblInfos, err := is.SchemaTableInfos(context.Background(), db.Name) require.NoError(t, err) diff --git a/pkg/planner/core/plan_test.go b/pkg/planner/core/plan_test.go index 0fca55f71a337..26f80737c3c09 100644 --- a/pkg/planner/core/plan_test.go +++ b/pkg/planner/core/plan_test.go @@ -28,7 +28,6 @@ import ( "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/ast" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/planner/core/base" @@ -371,7 +370,7 @@ func TestExplainFormatHintRecoverableForTiFlashReplica(t *testing.T) { tk.MustExec("create table t(a int)") // Create virtual `tiflash` replica info. is := dom.InfoSchema() - tblInfo, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tblInfo, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tblInfo.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ Count: 1, diff --git a/pkg/planner/core/planbuilder.go b/pkg/planner/core/planbuilder.go index 184f3ea803e7d..782971cec0b71 100644 --- a/pkg/planner/core/planbuilder.go +++ b/pkg/planner/core/planbuilder.go @@ -37,7 +37,6 @@ import ( "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/auth" "github.com/pingcap/tidb/pkg/parser/charset" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/opcode" "github.com/pingcap/tidb/pkg/parser/terror" @@ -1076,7 +1075,7 @@ func (*PlanBuilder) detectSelectWindow(sel *ast.SelectStmt) bool { return false } -func getPathByIndexName(paths []*util.AccessPath, idxName pmodel.CIStr, tblInfo *model.TableInfo) *util.AccessPath { +func getPathByIndexName(paths []*util.AccessPath, idxName ast.CIStr, tblInfo *model.TableInfo) *util.AccessPath { var indexPrefixPath *util.AccessPath prefixMatches := 0 for _, path := range paths { @@ -1104,7 +1103,7 @@ func getPathByIndexName(paths []*util.AccessPath, idxName pmodel.CIStr, tblInfo return nil } -func isPrimaryIndex(indexName pmodel.CIStr) bool { +func isPrimaryIndex(indexName ast.CIStr) bool { return indexName.L == "primary" } @@ -1138,7 +1137,7 @@ func isForUpdateReadSelectLock(lock *ast.SelectLockInfo) bool { lock.LockType == ast.SelectLockForUpdateWaitN } -func getPossibleAccessPaths(ctx base.PlanContext, tableHints *hint.PlanHints, indexHints []*ast.IndexHint, tbl table.Table, dbName, tblName pmodel.CIStr, check bool, hasFlagPartitionProcessor bool) ([]*util.AccessPath, error) { +func getPossibleAccessPaths(ctx base.PlanContext, tableHints *hint.PlanHints, indexHints []*ast.IndexHint, tbl table.Table, dbName, tblName ast.CIStr, check bool, hasFlagPartitionProcessor bool) ([]*util.AccessPath, error) { tblInfo := tbl.Meta() publicPaths := make([]*util.AccessPath, 0, len(tblInfo.Indices)+2) tp := kv.TiKV @@ -1343,7 +1342,7 @@ func getPossibleAccessPaths(ctx base.PlanContext, tableHints *hint.PlanHints, in return available, nil } -func filterPathByIsolationRead(ctx base.PlanContext, paths []*util.AccessPath, tblName pmodel.CIStr, dbName pmodel.CIStr) ([]*util.AccessPath, error) { +func filterPathByIsolationRead(ctx base.PlanContext, paths []*util.AccessPath, tblName ast.CIStr, dbName ast.CIStr) ([]*util.AccessPath, error) { // TODO: filter paths with isolation read locations. if util2.IsSysDB(dbName.L) { return paths, nil @@ -1595,7 +1594,7 @@ func (b *PlanBuilder) buildAdmin(ctx context.Context, as *ast.AdminStmt) (base.P return ret, nil } -func (b *PlanBuilder) buildPhysicalIndexLookUpReader(_ context.Context, dbName pmodel.CIStr, tbl table.Table, idx *model.IndexInfo) (base.Plan, error) { +func (b *PlanBuilder) buildPhysicalIndexLookUpReader(_ context.Context, dbName ast.CIStr, tbl table.Table, idx *model.IndexInfo) (base.Plan, error) { tblInfo := tbl.Meta() physicalID, isPartition := getPhysicalID(tbl, idx.Global) fullExprCols, _, err := expression.TableInfo2SchemaAndNames(b.ctx.GetExprCtx(), dbName, tblInfo) @@ -1755,7 +1754,7 @@ func tryGetPkHandleCol(tblInfo *model.TableInfo, allColSchema *expression.Schema return nil, nil, false } -func (b *PlanBuilder) buildPhysicalIndexLookUpReaders(ctx context.Context, dbName pmodel.CIStr, tbl table.Table, indices []table.Index) ([]base.Plan, []*model.IndexInfo, error) { +func (b *PlanBuilder) buildPhysicalIndexLookUpReaders(ctx context.Context, dbName ast.CIStr, tbl table.Table, indices []table.Index) ([]base.Plan, []*model.IndexInfo, error) { tblInfo := tbl.Meta() // get index information indexInfos := make([]*model.IndexInfo, 0, len(tblInfo.Indices)) @@ -1897,7 +1896,7 @@ func (b *PlanBuilder) buildCheckIndexSchema(tn *ast.TableName, indexName string) ID: col.ID}) } names = append(names, &types.FieldName{ - ColName: pmodel.NewCIStr("extra_handle"), + ColName: ast.NewCIStr("extra_handle"), TblName: tn.Name, DBName: tn.Schema, }) @@ -1986,7 +1985,7 @@ func BuildHandleColsForAnalyze(ctx base.PlanContext, tblInfo *model.TableInfo, a } // GetPhysicalIDsAndPartitionNames returns physical IDs and names of these partitions. -func GetPhysicalIDsAndPartitionNames(tblInfo *model.TableInfo, partitionNames []pmodel.CIStr) ([]int64, []string, error) { +func GetPhysicalIDsAndPartitionNames(tblInfo *model.TableInfo, partitionNames []ast.CIStr) ([]int64, []string, error) { pi := tblInfo.GetPartitionInfo() if pi == nil { if len(partitionNames) != 0 { @@ -2122,7 +2121,7 @@ func (b *PlanBuilder) getPredicateColumns(tbl *resolve.TableNameW, cols *calcOnc return cols.data, nil } -func getAnalyzeColumnList(specifiedColumns []pmodel.CIStr, tbl *resolve.TableNameW) ([]*model.ColumnInfo, error) { +func getAnalyzeColumnList(specifiedColumns []ast.CIStr, tbl *resolve.TableNameW) ([]*model.ColumnInfo, error) { colList := make([]*model.ColumnInfo, 0, len(specifiedColumns)) for _, colName := range specifiedColumns { colInfo := model.FindColumnInfo(tbl.TableInfo.Columns, colName.L) @@ -2139,23 +2138,23 @@ func getAnalyzeColumnList(specifiedColumns []pmodel.CIStr, tbl *resolve.TableNam // be record in mysql.analyze_options(only for the case of analyze table t columns c1, .., cn). func (b *PlanBuilder) getFullAnalyzeColumnsInfo( tbl *resolve.TableNameW, - columnChoice pmodel.ColumnChoice, + columnChoice ast.ColumnChoice, specifiedCols []*model.ColumnInfo, predicateCols, mustAnalyzedCols *calcOnceMap, mustAllColumns bool, warning bool, ) ([]*model.ColumnInfo, []*model.ColumnInfo, error) { - if mustAllColumns && warning && (columnChoice == pmodel.PredicateColumns || columnChoice == pmodel.ColumnList) { + if mustAllColumns && warning && (columnChoice == ast.PredicateColumns || columnChoice == ast.ColumnList) { b.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.NewNoStackErrorf("Table %s.%s has version 1 statistics so all the columns must be analyzed to overwrite the current statistics", tbl.Schema.L, tbl.Name.L)) } switch columnChoice { - case pmodel.DefaultChoice: + case ast.DefaultChoice: columnOptions := variable.AnalyzeColumnOptions.Load() switch columnOptions { - case pmodel.AllColumns.String(): + case ast.AllColumns.String(): return tbl.TableInfo.Columns, nil, nil - case pmodel.PredicateColumns.String(): + case ast.PredicateColumns.String(): columns, err := b.getColumnsBasedOnPredicateColumns( tbl, predicateCols, @@ -2171,9 +2170,9 @@ func (b *PlanBuilder) getFullAnalyzeColumnsInfo( logutil.BgLogger().Warn("Unknown default column choice, analyze all columns", zap.String("choice", columnOptions)) return tbl.TableInfo.Columns, nil, nil } - case pmodel.AllColumns: + case ast.AllColumns: return tbl.TableInfo.Columns, nil, nil - case pmodel.PredicateColumns: + case ast.PredicateColumns: columns, err := b.getColumnsBasedOnPredicateColumns( tbl, predicateCols, @@ -2184,7 +2183,7 @@ func (b *PlanBuilder) getFullAnalyzeColumnsInfo( return nil, nil, err } return columns, nil, nil - case pmodel.ColumnList: + case ast.ColumnList: colSet := getColumnSetFromSpecifiedCols(specifiedCols) mustAnalyzed, err := b.getMustAnalyzedColumns(tbl, mustAnalyzedCols) if err != nil { @@ -2555,7 +2554,7 @@ func (b *PlanBuilder) genV2AnalyzeOptions( isAnalyzeTable bool, physicalIDs []int64, astOpts map[ast.AnalyzeOptionType]uint64, - astColChoice pmodel.ColumnChoice, + astColChoice ast.ColumnChoice, astColList []*model.ColumnInfo, predicateCols, mustAnalyzedCols *calcOnceMap, mustAllColumns bool, @@ -2571,9 +2570,9 @@ func (b *PlanBuilder) genV2AnalyzeOptions( // Because the plan is generated for each partition individually, each partition uses its own statistics; // In dynamic mode, there is no partitioning, and a global plan is generated for the whole table, so a global statistic is needed; dynamicPrune := variable.PartitionPruneMode(b.ctx.GetSessionVars().PartitionPruneMode.Load()) == variable.Dynamic - if !isAnalyzeTable && dynamicPrune && (len(astOpts) > 0 || astColChoice != pmodel.DefaultChoice) { + if !isAnalyzeTable && dynamicPrune && (len(astOpts) > 0 || astColChoice != ast.DefaultChoice) { astOpts = make(map[ast.AnalyzeOptionType]uint64, 0) - astColChoice = pmodel.DefaultChoice + astColChoice = ast.DefaultChoice astColList = make([]*model.ColumnInfo, 0) b.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.NewNoStackError("Ignore columns and options when analyze partition in dynamic mode")) } @@ -2658,16 +2657,16 @@ func (b *PlanBuilder) genV2AnalyzeOptions( } // getSavedAnalyzeOpts gets the analyze options which are saved in mysql.analyze_options. -func (b *PlanBuilder) getSavedAnalyzeOpts(physicalID int64, tblInfo *model.TableInfo) (map[ast.AnalyzeOptionType]uint64, pmodel.ColumnChoice, []*model.ColumnInfo, error) { +func (b *PlanBuilder) getSavedAnalyzeOpts(physicalID int64, tblInfo *model.TableInfo) (map[ast.AnalyzeOptionType]uint64, ast.ColumnChoice, []*model.ColumnInfo, error) { analyzeOptions := map[ast.AnalyzeOptionType]uint64{} exec := b.ctx.GetRestrictedSQLExecutor() ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnStats) rows, _, err := exec.ExecRestrictedSQL(ctx, nil, "select sample_num,sample_rate,buckets,topn,column_choice,column_ids from mysql.analyze_options where table_id = %?", physicalID) if err != nil { - return nil, pmodel.DefaultChoice, nil, err + return nil, ast.DefaultChoice, nil, err } if len(rows) <= 0 { - return analyzeOptions, pmodel.DefaultChoice, nil, nil + return analyzeOptions, ast.DefaultChoice, nil, nil } row := rows[0] @@ -2690,7 +2689,7 @@ func (b *PlanBuilder) getSavedAnalyzeOpts(physicalID int64, tblInfo *model.Table colType := row.GetEnum(4) switch colType.Name { case "ALL": - return analyzeOptions, pmodel.AllColumns, tblInfo.Columns, nil + return analyzeOptions, ast.AllColumns, tblInfo.Columns, nil case "LIST": colIDStrs := strings.Split(row.GetString(5), ",") colList := make([]*model.ColumnInfo, 0, len(colIDStrs)) @@ -2701,11 +2700,11 @@ func (b *PlanBuilder) getSavedAnalyzeOpts(physicalID int64, tblInfo *model.Table colList = append(colList, colInfo) } } - return analyzeOptions, pmodel.ColumnList, colList, nil + return analyzeOptions, ast.ColumnList, colList, nil case "PREDICATE": - return analyzeOptions, pmodel.PredicateColumns, nil, nil + return analyzeOptions, ast.PredicateColumns, nil, nil default: - return analyzeOptions, pmodel.DefaultChoice, nil, nil + return analyzeOptions, ast.DefaultChoice, nil, nil } } @@ -2723,8 +2722,8 @@ func mergeAnalyzeOptions(stmtOpts map[ast.AnalyzeOptionType]uint64, savedOpts ma // pickColumnList picks the column list to be analyzed. // If the column list is specified in the statement, we will use it. -func pickColumnList(astColChoice pmodel.ColumnChoice, astColList []*model.ColumnInfo, tblSavedColChoice pmodel.ColumnChoice, tblSavedColList []*model.ColumnInfo) (pmodel.ColumnChoice, []*model.ColumnInfo) { - if astColChoice != pmodel.DefaultChoice { +func pickColumnList(astColChoice ast.ColumnChoice, astColList []*model.ColumnInfo, tblSavedColChoice ast.ColumnChoice, tblSavedColList []*model.ColumnInfo) (ast.ColumnChoice, []*model.ColumnInfo) { + if astColChoice != ast.DefaultChoice { return astColChoice, astColList } return tblSavedColChoice, tblSavedColList @@ -2761,10 +2760,10 @@ func (b *PlanBuilder) buildAnalyzeTable(as *ast.AnalyzeTableStmt, opts map[ast.A } // Version 1 analyze. - if as.ColumnChoice == pmodel.PredicateColumns { + if as.ColumnChoice == ast.PredicateColumns { return nil, errors.Errorf("Only the version 2 of analyze supports analyzing predicate columns") } - if as.ColumnChoice == pmodel.ColumnList { + if as.ColumnChoice == ast.ColumnList { return nil, errors.Errorf("Only the version 2 of analyze supports analyzing the specified columns") } for _, idx := range idxInfo { @@ -3357,7 +3356,7 @@ func buildColumnWithName(tableName, name string, tp byte, size int) (*expression fieldType.SetFlag(flag) return &expression.Column{ RetType: fieldType, - }, &types.FieldName{DBName: util2.InformationSchemaName, TblName: pmodel.NewCIStr(tableName), ColName: pmodel.NewCIStr(name)} + }, &types.FieldName{DBName: util2.InformationSchemaName, TblName: ast.NewCIStr(tableName), ColName: ast.NewCIStr(name)} } type columnsWithNames struct { @@ -4414,7 +4413,7 @@ func (b *PlanBuilder) buildLoadData(ctx context.Context, ld *ast.LoadDataStmt) ( db := b.ctx.GetSessionVars().CurrentDB return nil, infoschema.ErrTableNotExists.FastGenByArgs(db, tableInfo.Name.O) } - schema, names, err := expression.TableInfo2SchemaAndNames(b.ctx.GetExprCtx(), pmodel.NewCIStr(""), tableInfo) + schema, names, err := expression.TableInfo2SchemaAndNames(b.ctx.GetExprCtx(), ast.NewCIStr(""), tableInfo) if err != nil { return nil, err } @@ -4631,7 +4630,7 @@ func (b *PlanBuilder) buildImportInto(ctx context.Context, ld *ast.ImportIntoStm db := b.ctx.GetSessionVars().CurrentDB return nil, infoschema.ErrTableNotExists.FastGenByArgs(db, tableInfo.Name.O) } - schema, names, err := expression.TableInfo2SchemaAndNames(b.ctx.GetExprCtx(), pmodel.NewCIStr(""), tableInfo) + schema, names, err := expression.TableInfo2SchemaAndNames(b.ctx.GetExprCtx(), ast.NewCIStr(""), tableInfo) if err != nil { return nil, err } @@ -4978,7 +4977,7 @@ func (b *PlanBuilder) buildDDL(ctx context.Context, node ast.DDLNode) (base.Plan switch v := node.(type) { case *ast.AlterDatabaseStmt: if v.AlterDefaultDatabase { - v.Name = pmodel.NewCIStr(b.ctx.GetSessionVars().CurrentDB) + v.Name = ast.NewCIStr(b.ctx.GetSessionVars().CurrentDB) } if v.Name.O == "" { return nil, plannererrors.ErrNoDB @@ -5146,7 +5145,7 @@ func (b *PlanBuilder) buildDDL(ctx context.Context, node ast.DDLNode) (base.Plan names := plan.OutputNames() if v.Cols == nil { adjustOverlongViewColname(plan.(base.LogicalPlan)) - v.Cols = make([]pmodel.CIStr, len(schema.Columns)) + v.Cols = make([]ast.CIStr, len(schema.Columns)) for i, name := range names { v.Cols[i] = name.ColName } @@ -5725,7 +5724,7 @@ func convert2OutputSchemasAndNames(names []string, ftypes []byte, flags []uint) outputNames = make([]*types.FieldName, 0, len(names)) for i := range names { col := &expression.Column{} - outputNames = append(outputNames, &types.FieldName{ColName: pmodel.NewCIStr(names[i])}) + outputNames = append(outputNames, &types.FieldName{ColName: ast.NewCIStr(names[i])}) // User varchar as the default return column type. tp := mysql.TypeVarchar if len(ftypes) != 0 && ftypes[i] != mysql.TypeUnspecified { @@ -5818,7 +5817,7 @@ func adjustOverlongViewColname(plan base.LogicalPlan) { outputNames := plan.OutputNames() for i := range outputNames { if outputName := outputNames[i].ColName.L; len(outputName) > mysql.MaxColumnNameLength { - outputNames[i].ColName = pmodel.NewCIStr(fmt.Sprintf("name_exp_%d", i+1)) + outputNames[i].ColName = ast.NewCIStr(fmt.Sprintf("name_exp_%d", i+1)) } } } diff --git a/pkg/planner/core/planbuilder_test.go b/pkg/planner/core/planbuilder_test.go index 39801cd2da88a..8e61505232dd6 100644 --- a/pkg/planner/core/planbuilder_test.go +++ b/pkg/planner/core/planbuilder_test.go @@ -33,7 +33,6 @@ import ( "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser" "github.com/pingcap/tidb/pkg/parser/ast" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/core/resolve" @@ -96,24 +95,24 @@ func TestGetPathByIndexName(t *testing.T) { accessPath := []*util.AccessPath{ {IsIntHandlePath: true}, - {Index: &model.IndexInfo{Name: pmodel.NewCIStr("idx")}}, + {Index: &model.IndexInfo{Name: ast.NewCIStr("idx")}}, genTiFlashPath(tblInfo), } - path := getPathByIndexName(accessPath, pmodel.NewCIStr("idx"), tblInfo) + path := getPathByIndexName(accessPath, ast.NewCIStr("idx"), tblInfo) require.NotNil(t, path) require.Equal(t, accessPath[1], path) // "id" is a prefix of "idx" - path = getPathByIndexName(accessPath, pmodel.NewCIStr("id"), tblInfo) + path = getPathByIndexName(accessPath, ast.NewCIStr("id"), tblInfo) require.NotNil(t, path) require.Equal(t, accessPath[1], path) - path = getPathByIndexName(accessPath, pmodel.NewCIStr("primary"), tblInfo) + path = getPathByIndexName(accessPath, ast.NewCIStr("primary"), tblInfo) require.NotNil(t, path) require.Equal(t, accessPath[0], path) - path = getPathByIndexName(accessPath, pmodel.NewCIStr("not exists"), tblInfo) + path = getPathByIndexName(accessPath, ast.NewCIStr("not exists"), tblInfo) require.Nil(t, path) tblInfo = &model.TableInfo{ @@ -121,7 +120,7 @@ func TestGetPathByIndexName(t *testing.T) { PKIsHandle: false, } - path = getPathByIndexName(accessPath, pmodel.NewCIStr("primary"), tblInfo) + path = getPathByIndexName(accessPath, ast.NewCIStr("primary"), tblInfo) require.Nil(t, path) } @@ -686,23 +685,23 @@ func TestGetFullAnalyzeColumnsInfo(t *testing.T) { // Create a new TableName instance. tableName := &ast.TableName{ - Schema: pmodel.NewCIStr("test"), - Name: pmodel.NewCIStr("my_table"), + Schema: ast.NewCIStr("test"), + Name: ast.NewCIStr("my_table"), } columns := []*model.ColumnInfo{ { ID: 1, - Name: pmodel.NewCIStr("id"), + Name: ast.NewCIStr("id"), FieldType: *types.NewFieldType(mysql.TypeLonglong), }, { ID: 2, - Name: pmodel.NewCIStr("name"), + Name: ast.NewCIStr("name"), FieldType: *types.NewFieldType(mysql.TypeString), }, { ID: 3, - Name: pmodel.NewCIStr("age"), + Name: ast.NewCIStr("age"), FieldType: *types.NewFieldType(mysql.TypeLonglong), }, } @@ -714,7 +713,7 @@ func TestGetFullAnalyzeColumnsInfo(t *testing.T) { } // Test case 1: AllColumns. - cols, _, err := pb.getFullAnalyzeColumnsInfo(tblNameW, pmodel.AllColumns, nil, nil, nil, false, false) + cols, _, err := pb.getFullAnalyzeColumnsInfo(tblNameW, ast.AllColumns, nil, nil, nil, false, false) require.NoError(t, err) require.Equal(t, columns, cols) @@ -726,7 +725,7 @@ func TestGetFullAnalyzeColumnsInfo(t *testing.T) { // Test case 3: ColumnList. specifiedCols := []*model.ColumnInfo{columns[0], columns[2]} mustAnalyzedCols.data[3] = struct{}{} - cols, _, err = pb.getFullAnalyzeColumnsInfo(tblNameW, pmodel.ColumnList, specifiedCols, nil, mustAnalyzedCols, false, false) + cols, _, err = pb.getFullAnalyzeColumnsInfo(tblNameW, ast.ColumnList, specifiedCols, nil, mustAnalyzedCols, false, false) require.NoError(t, err) require.Equal(t, specifiedCols, cols) } @@ -740,12 +739,12 @@ func TestRequireInsertAndSelectPriv(t *testing.T) { tables := []*ast.TableName{ { - Schema: pmodel.NewCIStr("test"), - Name: pmodel.NewCIStr("t1"), + Schema: ast.NewCIStr("test"), + Name: ast.NewCIStr("t1"), }, { - Schema: pmodel.NewCIStr("test"), - Name: pmodel.NewCIStr("t2"), + Schema: ast.NewCIStr("test"), + Name: ast.NewCIStr("t2"), }, } diff --git a/pkg/planner/core/point_get_plan.go b/pkg/planner/core/point_get_plan.go index c18d9fb978ade..95d1c77d690b8 100644 --- a/pkg/planner/core/point_get_plan.go +++ b/pkg/planner/core/point_get_plan.go @@ -29,7 +29,6 @@ import ( "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/charset" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/opcode" "github.com/pingcap/tidb/pkg/parser/terror" @@ -82,7 +81,7 @@ type PointGetPlan struct { // Please see comments in PhysicalPlan for details. probeParents []base.PhysicalPlan `plan-cache-clone:"shallow"` // explicit partition selection - PartitionNames []pmodel.CIStr `plan-cache-clone:"shallow"` + PartitionNames []ast.CIStr `plan-cache-clone:"shallow"` dbName string schema *expression.Schema `plan-cache-clone:"shallow"` @@ -421,7 +420,7 @@ type BatchPointGetPlan struct { // Please see comments in PhysicalPlan for details. probeParents []base.PhysicalPlan // explicit partition selection - PartitionNames []pmodel.CIStr `plan-cache-clone:"shallow"` + PartitionNames []ast.CIStr `plan-cache-clone:"shallow"` ctx base.PlanContext dbName string @@ -642,7 +641,7 @@ func (p *BatchPointGetPlan) LoadTableStats(ctx sessionctx.Context) { loadTableStats(ctx, p.TblInfo, p.TblInfo.ID) } -func isInExplicitPartitions(pi *model.PartitionInfo, idx int, names []pmodel.CIStr) bool { +func isInExplicitPartitions(pi *model.PartitionInfo, idx int, names []ast.CIStr) bool { if len(names) == 0 { return true } @@ -1450,7 +1449,7 @@ func indexIsAvailableByHints(idxInfo *model.IndexInfo, idxHints []*ast.IndexHint if len(idxHints) == 0 { return true } - match := func(name pmodel.CIStr) bool { + match := func(name ast.CIStr) bool { if idxInfo == nil { return name.L == "primary" } @@ -1519,9 +1518,9 @@ func checkFastPlanPrivilege(ctx base.PlanContext, dbName, tableName string, chec } func buildSchemaFromFields( - dbName pmodel.CIStr, + dbName ast.CIStr, tbl *model.TableInfo, - tblName pmodel.CIStr, + tblName ast.CIStr, fields []*ast.SelectField, ) ( *expression.Schema, @@ -1627,7 +1626,7 @@ func tryExtractRowChecksumColumn(field *ast.SelectField, idx int) (*types.FieldN // getSingleTableNameAndAlias return the ast node of queried table name and the alias string. // `tblName` is `nil` if there are multiple tables in the query. // `tblAlias` will be the real table name if there is no table alias in the query. -func getSingleTableNameAndAlias(tableRefs *ast.TableRefsClause) (tblName *ast.TableName, tblAlias pmodel.CIStr) { +func getSingleTableNameAndAlias(tableRefs *ast.TableRefsClause) (tblName *ast.TableName, tblAlias ast.CIStr) { if tableRefs == nil || tableRefs.TableRefs == nil || tableRefs.TableRefs.Right != nil { return nil, tblAlias } @@ -1647,7 +1646,7 @@ func getSingleTableNameAndAlias(tableRefs *ast.TableRefsClause) (tblName *ast.Ta } // getNameValuePairs extracts `column = constant/paramMarker` conditions from expr as name value pairs. -func getNameValuePairs(ctx expression.BuildContext, tbl *model.TableInfo, tblName pmodel.CIStr, nvPairs []nameValuePair, expr ast.ExprNode) ( +func getNameValuePairs(ctx expression.BuildContext, tbl *model.TableInfo, tblName ast.CIStr, nvPairs []nameValuePair, expr ast.ExprNode) ( pairs []nameValuePair, isTableDual bool) { evalCtx := ctx.GetEvalCtx() binOp, ok := expr.(*ast.BinaryOperationExpr) @@ -2135,7 +2134,7 @@ func buildHandleCols(ctx base.PlanContext, dbName string, tbl *model.TableInfo, tableAliasName = pointget.OutputNames()[0].TblName } newOutputNames = append(newOutputNames, &types.FieldName{ - DBName: pmodel.NewCIStr(dbName), + DBName: ast.NewCIStr(dbName), TblName: tableAliasName, OrigTblName: tbl.Name, ColName: model.ExtraHandleName, @@ -2146,12 +2145,12 @@ func buildHandleCols(ctx base.PlanContext, dbName string, tbl *model.TableInfo, // TODO: Remove this, by enabling all types of partitioning // and update/add tests -func getHashOrKeyPartitionColumnName(ctx base.PlanContext, tbl *model.TableInfo) *pmodel.CIStr { +func getHashOrKeyPartitionColumnName(ctx base.PlanContext, tbl *model.TableInfo) *ast.CIStr { pi := tbl.GetPartitionInfo() if pi == nil { return nil } - if pi.Type != pmodel.PartitionTypeHash && pi.Type != pmodel.PartitionTypeKey { + if pi.Type != ast.PartitionTypeHash && pi.Type != ast.PartitionTypeKey { return nil } is := ctx.GetInfoSchema().(infoschema.InfoSchema) @@ -2161,7 +2160,7 @@ func getHashOrKeyPartitionColumnName(ctx base.PlanContext, tbl *model.TableInfo) } // PartitionExpr don't need columns and names for hash partition. partitionExpr := table.(partitionTable).PartitionExpr() - if pi.Type == pmodel.PartitionTypeKey { + if pi.Type == ast.PartitionTypeKey { // used to judge whether the key partition contains only one field if len(pi.Columns) != 1 { return nil diff --git a/pkg/planner/core/preprocess.go b/pkg/planner/core/preprocess.go index a1a870f7ead5b..3758482044c89 100644 --- a/pkg/planner/core/preprocess.go +++ b/pkg/planner/core/preprocess.go @@ -32,7 +32,6 @@ import ( "github.com/pingcap/tidb/pkg/parser/auth" "github.com/pingcap/tidb/pkg/parser/charset" "github.com/pingcap/tidb/pkg/parser/format" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/planner/core/base" @@ -516,7 +515,7 @@ func (p *preprocessor) tableByName(tn *ast.TableName) (table.Table, error) { if currentDB == "" { return nil, errors.Trace(plannererrors.ErrNoDB) } - sName := pmodel.NewCIStr(currentDB) + sName := ast.NewCIStr(currentDB) is := p.ensureInfoSchema() // for 'SHOW CREATE VIEW/SEQUENCE ...' statement, ignore local temporary tables. @@ -874,7 +873,7 @@ func (p *preprocessor) checkAdminCheckTableGrammar(stmt *ast.AdminStmt) { func (p *preprocessor) checkCreateTableGrammar(stmt *ast.CreateTableStmt) { if stmt.ReferTable != nil { - schema := pmodel.NewCIStr(p.sctx.GetSessionVars().CurrentDB) + schema := ast.NewCIStr(p.sctx.GetSessionVars().CurrentDB) if stmt.ReferTable.Schema.String() != "" { schema = stmt.ReferTable.Schema } @@ -1050,7 +1049,7 @@ func (p *preprocessor) checkDropTableGrammar(stmt *ast.DropTableStmt) { } func (p *preprocessor) checkDropTemporaryTableGrammar(stmt *ast.DropTableStmt) { - currentDB := pmodel.NewCIStr(p.sctx.GetSessionVars().CurrentDB) + currentDB := ast.NewCIStr(p.sctx.GetSessionVars().CurrentDB) for _, t := range stmt.Tables { if util.IsInCorrectIdentifierName(t.Name.String()) { p.err = dbterror.ErrWrongTableName.GenWithStackByArgs(t.Name.String()) @@ -1115,7 +1114,7 @@ func isTableAliasDuplicate(node ast.ResultSetNode, tableAliases map[string]any) if tabName.L == "" { if tableNode, ok := ts.Source.(*ast.TableName); ok { if tableNode.Schema.L != "" { - tabName = pmodel.NewCIStr(fmt.Sprintf("%s.%s", tableNode.Schema.L, tableNode.Name.L)) + tabName = ast.NewCIStr(fmt.Sprintf("%s.%s", tableNode.Schema.L, tableNode.Name.L)) } else { tabName = tableNode.Name } @@ -1602,7 +1601,7 @@ func (p *preprocessor) handleTableName(tn *ast.TableName) { return } - tn.Schema = pmodel.NewCIStr(currentDB) + tn.Schema = ast.NewCIStr(currentDB) } if p.flag&inCreateOrDropTable > 0 { @@ -1640,7 +1639,7 @@ func (p *preprocessor) handleTableName(tn *ast.TableName) { } if !p.skipLockMDL() { - table, err = tryLockMDLAndUpdateSchemaIfNecessary(p.ctx, p.sctx.GetPlanCtx(), pmodel.NewCIStr(tn.Schema.L), table, p.ensureInfoSchema()) + table, err = tryLockMDLAndUpdateSchemaIfNecessary(p.ctx, p.sctx.GetPlanCtx(), ast.NewCIStr(tn.Schema.L), table, p.ensureInfoSchema()) if err != nil { p.err = err return @@ -1701,7 +1700,7 @@ func (p *preprocessor) resolveShowStmt(node *ast.ShowStmt) { node.DBName = p.sctx.GetSessionVars().CurrentDB } } else if node.Table != nil && node.Table.Schema.L == "" { - node.Table.Schema = pmodel.NewCIStr(node.DBName) + node.Table.Schema = ast.NewCIStr(node.DBName) } if node.User != nil && node.User.CurrentUser { // Fill the Username and Hostname with the current user. @@ -1746,7 +1745,7 @@ func (p *preprocessor) resolveAlterTableStmt(node *ast.AlterTableStmt) { if spec.Tp == ast.AlterTableAddConstraint && spec.Constraint.Refer != nil { table := spec.Constraint.Refer.Table if table.Schema.L == "" && node.Table.Schema.L != "" { - table.Schema = pmodel.NewCIStr(node.Table.Schema.L) + table.Schema = ast.NewCIStr(node.Table.Schema.L) } if spec.Constraint.Tp == ast.ConstraintForeignKey { // when foreign_key_checks is off, should ignore err when refer table is not exists. @@ -1861,7 +1860,7 @@ func (p *preprocessor) hasAutoConvertWarning(colDef *ast.ColumnDef) bool { return false } -func tryLockMDLAndUpdateSchemaIfNecessary(ctx context.Context, sctx base.PlanContext, dbName pmodel.CIStr, tbl table.Table, is infoschema.InfoSchema) (retTbl table.Table, err error) { +func tryLockMDLAndUpdateSchemaIfNecessary(ctx context.Context, sctx base.PlanContext, dbName ast.CIStr, tbl table.Table, is infoschema.InfoSchema) (retTbl table.Table, err error) { skipLock := false shouldLockMDL := false var lockedID int64 @@ -2025,7 +2024,7 @@ type aliasChecker struct{} func (*aliasChecker) Enter(in ast.Node) (ast.Node, bool) { if deleteStmt, ok := in.(*ast.DeleteStmt); ok { // 1. check the tableRefs of deleteStmt to find the alias - var aliases []*pmodel.CIStr + var aliases []*ast.CIStr if deleteStmt.TableRefs != nil && deleteStmt.TableRefs.TableRefs != nil { tableRefs := deleteStmt.TableRefs.TableRefs if val := getTableRefsAlias(tableRefs.Left); val != nil { @@ -2054,7 +2053,7 @@ func (*aliasChecker) Enter(in ast.Node) (ast.Node, bool) { return in, false } -func getTableRefsAlias(tableRefs ast.ResultSetNode) *pmodel.CIStr { +func getTableRefsAlias(tableRefs ast.ResultSetNode) *ast.CIStr { switch v := tableRefs.(type) { case *ast.Join: if v.Left != nil { diff --git a/pkg/planner/core/resolve/BUILD.bazel b/pkg/planner/core/resolve/BUILD.bazel index 2c1beb5266085..b83fbc3603ad2 100644 --- a/pkg/planner/core/resolve/BUILD.bazel +++ b/pkg/planner/core/resolve/BUILD.bazel @@ -11,6 +11,5 @@ go_library( deps = [ "//pkg/meta/model", "//pkg/parser/ast", - "//pkg/parser/model", ], ) diff --git a/pkg/planner/core/resolve/result.go b/pkg/planner/core/resolve/result.go index b85d7a6590c94..e0461ff6bb59e 100644 --- a/pkg/planner/core/resolve/result.go +++ b/pkg/planner/core/resolve/result.go @@ -16,7 +16,7 @@ package resolve import ( "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" ) // ResultField represents a result field which can be a column from a table, @@ -28,13 +28,13 @@ import ( // easily evaluated. type ResultField struct { Column *model.ColumnInfo - ColumnAsName pmodel.CIStr + ColumnAsName ast.CIStr // EmptyOrgName indicates whether this field has an empty org_name. A field has an empty org name, if it's an // expression. It's not sure whether it's safe to use empty string in `.Column.Name`, so a new field is added to // indicate whether it's empty. EmptyOrgName bool Table *model.TableInfo - TableAsName pmodel.CIStr - DBName pmodel.CIStr + TableAsName ast.CIStr + DBName ast.CIStr } diff --git a/pkg/planner/core/rule_generate_column_substitute_test.go b/pkg/planner/core/rule_generate_column_substitute_test.go index 6808197fc82da..04a6abf668eae 100644 --- a/pkg/planner/core/rule_generate_column_substitute_test.go +++ b/pkg/planner/core/rule_generate_column_substitute_test.go @@ -20,7 +20,7 @@ import ( "testing" "github.com/pingcap/tidb/pkg/domain" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/core/operator/logicalop" @@ -212,7 +212,7 @@ func BenchmarkSubstituteExpression(b *testing.B) { tk.MustExec("drop table if exists tai") tk.MustExec("create table tai(a varchar(256), b varchar(256), c int as (a+1), d int as (b+1))") is := domain.GetDomain(tk.Session()).InfoSchema() - _, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("tai")) + _, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("tai")) require.NoError(b, err) condition := "(tai.a='%s' AND tai.b='%s') OR" + "(tai.a='%s' AND tai.b='%s') OR" + diff --git a/pkg/planner/core/rule_join_reorder_dp_test.go b/pkg/planner/core/rule_join_reorder_dp_test.go index f89c34a8654cb..7603b1eb64a73 100644 --- a/pkg/planner/core/rule_join_reorder_dp_test.go +++ b/pkg/planner/core/rule_join_reorder_dp_test.go @@ -21,7 +21,6 @@ import ( "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/parser/ast" - "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/core/operator/logicalop" @@ -138,7 +137,7 @@ func makeStatsMapForTPCHQ5() map[int]*property.StatsInfo { func newDataSource(ctx base.PlanContext, name string, count int) base.LogicalPlan { ds := logicalop.DataSource{}.Init(ctx, 0) - tan := model.NewCIStr(name) + tan := ast.NewCIStr(name) ds.TableAsName = &tan ds.SetSchema(expression.NewSchema()) ds.Schema().Append(&expression.Column{ diff --git a/pkg/planner/core/rule_partition_processor.go b/pkg/planner/core/rule_partition_processor.go index 5066d314f7ba2..c5e8d8aa9923b 100644 --- a/pkg/planner/core/rule_partition_processor.go +++ b/pkg/planner/core/rule_partition_processor.go @@ -28,7 +28,6 @@ import ( "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/ast" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/core/operator/logicalop" @@ -163,7 +162,7 @@ func getPartColumnsForHashPartition(hashExpr expression.Expression) ([]*expressi } func (s *PartitionProcessor) getUsedHashPartitions(ctx base.PlanContext, - tbl table.Table, partitionNames []pmodel.CIStr, columns []*expression.Column, + tbl table.Table, partitionNames []ast.CIStr, columns []*expression.Column, conds []expression.Expression, names types.NameSlice) ([]int, error) { pi := tbl.Meta().Partition hashExpr, err := generateHashPartitionExpr(ctx, pi, columns, names) @@ -280,7 +279,7 @@ func (s *PartitionProcessor) getUsedHashPartitions(ctx base.PlanContext, } func (s *PartitionProcessor) getUsedKeyPartitions(ctx base.PlanContext, - tbl table.Table, partitionNames []pmodel.CIStr, columns []*expression.Column, + tbl table.Table, partitionNames []ast.CIStr, columns []*expression.Column, conds []expression.Expression, _ types.NameSlice) ([]int, error) { pi := tbl.Meta().Partition partExpr := tbl.(partitionTable).PartitionExpr() @@ -388,9 +387,9 @@ func (s *PartitionProcessor) getUsedKeyPartitions(ctx base.PlanContext, // getUsedPartitions is used to get used partitions for hash or key partition tables func (s *PartitionProcessor) getUsedPartitions(ctx base.PlanContext, tbl table.Table, - partitionNames []pmodel.CIStr, columns []*expression.Column, conds []expression.Expression, - names types.NameSlice, partType pmodel.PartitionType) ([]int, error) { - if partType == pmodel.PartitionTypeHash { + partitionNames []ast.CIStr, columns []*expression.Column, conds []expression.Expression, + names types.NameSlice, partType ast.PartitionType) ([]int, error) { + if partType == ast.PartitionTypeHash { return s.getUsedHashPartitions(ctx, tbl, partitionNames, columns, conds, names) } return s.getUsedKeyPartitions(ctx, tbl, partitionNames, columns, conds, names) @@ -399,7 +398,7 @@ func (s *PartitionProcessor) getUsedPartitions(ctx base.PlanContext, tbl table.T // findUsedPartitions is used to get used partitions for hash or key partition tables. // The first returning is the used partition index set pruned by `conds`. func (s *PartitionProcessor) findUsedPartitions(ctx base.PlanContext, - tbl table.Table, partitionNames []pmodel.CIStr, conds []expression.Expression, + tbl table.Table, partitionNames []ast.CIStr, conds []expression.Expression, columns []*expression.Column, names types.NameSlice) ([]int, error) { pi := tbl.Meta().Partition used, err := s.getUsedPartitions(ctx, tbl, partitionNames, columns, conds, names, pi.Type) @@ -421,7 +420,7 @@ func (s *PartitionProcessor) findUsedPartitions(ctx base.PlanContext, return ret, nil } -func (s *PartitionProcessor) convertToIntSlice(or partitionRangeOR, pi *model.PartitionInfo, partitionNames []pmodel.CIStr) []int { +func (s *PartitionProcessor) convertToIntSlice(or partitionRangeOR, pi *model.PartitionInfo, partitionNames []ast.CIStr) []int { if len(or) == 1 && or[0].start == 0 && or[0].end == len(pi.Definitions) { if len(partitionNames) == 0 { if len(pi.Definitions) == 1 { @@ -455,7 +454,7 @@ func convertToRangeOr(used []int, pi *model.PartitionInfo) partitionRangeOR { } // pruneHashOrKeyPartition is used to prune hash or key partition tables -func (s *PartitionProcessor) pruneHashOrKeyPartition(ctx base.PlanContext, tbl table.Table, partitionNames []pmodel.CIStr, +func (s *PartitionProcessor) pruneHashOrKeyPartition(ctx base.PlanContext, tbl table.Table, partitionNames []ast.CIStr, conds []expression.Expression, columns []*expression.Column, names types.NameSlice) ([]int, error) { used, err := s.findUsedPartitions(ctx, tbl, partitionNames, conds, columns, names) if err != nil { @@ -536,12 +535,12 @@ type listPartitionPruner struct { *PartitionProcessor ctx base.PlanContext pi *model.PartitionInfo - partitionNames []pmodel.CIStr + partitionNames []ast.CIStr fullRange map[int]struct{} listPrune *tables.ForListPruning } -func newListPartitionPruner(ctx base.PlanContext, tbl table.Table, partitionNames []pmodel.CIStr, s *PartitionProcessor, pruneList *tables.ForListPruning, columns []*expression.Column) *listPartitionPruner { +func newListPartitionPruner(ctx base.PlanContext, tbl table.Table, partitionNames []ast.CIStr, s *PartitionProcessor, pruneList *tables.ForListPruning, columns []*expression.Column) *listPartitionPruner { pruneList = pruneList.Clone() for i := range pruneList.PruneExprCols { for j := range columns { @@ -822,7 +821,7 @@ func (l *listPartitionPruner) findUsedListPartitions(conds []expression.Expressi return used, nil } -func (s *PartitionProcessor) findUsedListPartitions(ctx base.PlanContext, tbl table.Table, partitionNames []pmodel.CIStr, +func (s *PartitionProcessor) findUsedListPartitions(ctx base.PlanContext, tbl table.Table, partitionNames []ast.CIStr, conds []expression.Expression, columns []*expression.Column) ([]int, error) { pi := tbl.Meta().Partition partExpr := tbl.(partitionTable).PartitionExpr() @@ -865,7 +864,7 @@ func (s *PartitionProcessor) findUsedListPartitions(ctx base.PlanContext, tbl ta return ret, nil } -func (s *PartitionProcessor) pruneListPartition(ctx base.PlanContext, tbl table.Table, partitionNames []pmodel.CIStr, +func (s *PartitionProcessor) pruneListPartition(ctx base.PlanContext, tbl table.Table, partitionNames []ast.CIStr, conds []expression.Expression, columns []*expression.Column) ([]int, error) { used, err := s.findUsedListPartitions(ctx, tbl, partitionNames, conds, columns) if err != nil { @@ -891,11 +890,11 @@ func (s *PartitionProcessor) prune(ds *logicalop.DataSource, opt *optimizetrace. // a = 1 OR a = 2 => for p1 only "a = 1" and for p2 only "a = 2" // since a cannot be 2 in p1 and a cannot be 1 in p2 switch pi.Type { - case pmodel.PartitionTypeRange: + case ast.PartitionTypeRange: return s.processRangePartition(ds, pi, opt) - case pmodel.PartitionTypeHash, pmodel.PartitionTypeKey: + case ast.PartitionTypeHash, ast.PartitionTypeKey: return s.processHashOrKeyPartition(ds, pi, opt) - case pmodel.PartitionTypeList: + case ast.PartitionTypeList: return s.processListPartition(ds, pi, opt) } @@ -903,7 +902,7 @@ func (s *PartitionProcessor) prune(ds *logicalop.DataSource, opt *optimizetrace. } // findByName checks whether object name exists in list. -func (*PartitionProcessor) findByName(partitionNames []pmodel.CIStr, partitionName string) bool { +func (*PartitionProcessor) findByName(partitionNames []ast.CIStr, partitionName string) bool { for _, s := range partitionNames { if s.L == partitionName { return true @@ -1765,7 +1764,7 @@ func (*PartitionProcessor) resolveAccessPaths(ds *logicalop.DataSource) error { return nil } -func (s *PartitionProcessor) resolveOptimizeHint(ds *logicalop.DataSource, partitionName pmodel.CIStr) error { +func (s *PartitionProcessor) resolveOptimizeHint(ds *logicalop.DataSource, partitionName ast.CIStr) error { // index hint if len(ds.IndexHints) > 0 { newIndexHint := make([]h.HintedIndex, 0, len(ds.IndexHints)) @@ -1831,7 +1830,7 @@ func (s *PartitionProcessor) resolveOptimizeHint(ds *logicalop.DataSource, parti return s.resolveAccessPaths(ds) } -func checkTableHintsApplicableForPartition(partitions []pmodel.CIStr, partitionSet set.StringSet) []string { +func checkTableHintsApplicableForPartition(partitions []ast.CIStr, partitionSet set.StringSet) []string { var unknownPartitions []string for _, p := range partitions { if !partitionSet.Exist(p.L) { diff --git a/pkg/planner/core/runtime_filter_generator_test.go b/pkg/planner/core/runtime_filter_generator_test.go index 4b62e22fb4fc2..146cf9cd53674 100644 --- a/pkg/planner/core/runtime_filter_generator_test.go +++ b/pkg/planner/core/runtime_filter_generator_test.go @@ -23,7 +23,7 @@ import ( "github.com/pingcap/tidb/pkg/config" "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/testkit/testdata" @@ -53,13 +53,13 @@ func TestRuntimeFilterGenerator(t *testing.T) { // set tiflash replica dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() - tblInfo, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t1")) + tblInfo, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t1")) require.NoError(t, err) tblInfo.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ Count: 1, Available: true, } - tblInfo, err = is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t2")) + tblInfo, err = is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t2")) require.NoError(t, err) tblInfo.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ Count: 1, diff --git a/pkg/planner/core/util_test.go b/pkg/planner/core/util_test.go index 8fae0096f9951..49b80730cd9d5 100644 --- a/pkg/planner/core/util_test.go +++ b/pkg/planner/core/util_test.go @@ -22,7 +22,6 @@ import ( "github.com/pingcap/tidb/pkg/parser" "github.com/pingcap/tidb/pkg/parser/ast" - "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/planner/core/resolve" "github.com/stretchr/testify/require" ) @@ -53,31 +52,31 @@ func TestExtractTableList(t *testing.T) { { sql: "WITH t AS (SELECT * FROM t2) SELECT * FROM t, t1, mysql.user WHERE t1.a = mysql.user.username", expect: []*ast.TableName{ - {Name: model.NewCIStr("t")}, - {Name: model.NewCIStr("t1")}, - {Name: model.NewCIStr("t2")}, - {Name: model.NewCIStr("user"), Schema: model.NewCIStr("mysql")}, + {Name: ast.NewCIStr("t")}, + {Name: ast.NewCIStr("t1")}, + {Name: ast.NewCIStr("t2")}, + {Name: ast.NewCIStr("user"), Schema: ast.NewCIStr("mysql")}, }, }, { sql: "SELECT (SELECT a,b,c FROM t1) AS t WHERE t.a = 1", expect: []*ast.TableName{ - {Name: model.NewCIStr("t1")}, + {Name: ast.NewCIStr("t1")}, }, }, { sql: "SELECT * FROM t, v AS w", expect: []*ast.TableName{ - {Name: model.NewCIStr("t")}, - {Name: model.NewCIStr("v")}, + {Name: ast.NewCIStr("t")}, + {Name: ast.NewCIStr("v")}, }, }, { sql: "SELECT * FROM t, v AS w", asName: true, expect: []*ast.TableName{ - {Name: model.NewCIStr("t")}, - {Name: model.NewCIStr("w")}, + {Name: ast.NewCIStr("t")}, + {Name: ast.NewCIStr("w")}, }, }, { @@ -100,206 +99,206 @@ func TestExtractTableList(t *testing.T) { ORDER BY avg_score DESC`, expect: []*ast.TableName{ - {Name: model.NewCIStr("scores")}, - {Name: model.NewCIStr("students")}, + {Name: ast.NewCIStr("scores")}, + {Name: ast.NewCIStr("students")}, }, }, { sql: "DELETE FROM x.y z WHERE z.a > 0", expect: []*ast.TableName{ - {Name: model.NewCIStr("y"), Schema: model.NewCIStr("x")}, + {Name: ast.NewCIStr("y"), Schema: ast.NewCIStr("x")}, }, }, { sql: "WITH t AS (SELECT * FROM v) DELETE FROM x.y z WHERE z.a > t.c", expect: []*ast.TableName{ - {Name: model.NewCIStr("y"), Schema: model.NewCIStr("x")}, - {Name: model.NewCIStr("v")}, + {Name: ast.NewCIStr("y"), Schema: ast.NewCIStr("x")}, + {Name: ast.NewCIStr("v")}, }, }, { sql: "DELETE FROM `t1` AS `t2` USE INDEX (`fld1`) WHERE `t2`.`fld`=2", expect: []*ast.TableName{ - {Name: model.NewCIStr("t1")}, + {Name: ast.NewCIStr("t1")}, }, }, { sql: "DELETE FROM `t1` AS `t2` USE INDEX (`fld1`) WHERE `t2`.`fld`=2", asName: true, expect: []*ast.TableName{ - {Name: model.NewCIStr("t2")}, + {Name: ast.NewCIStr("t2")}, }, }, { sql: "UPDATE t1 USE INDEX(idx_a) JOIN t2 SET t1.price=t2.price WHERE t1.id=t2.id;", expect: []*ast.TableName{ - {Name: model.NewCIStr("t1")}, - {Name: model.NewCIStr("t2")}, + {Name: ast.NewCIStr("t1")}, + {Name: ast.NewCIStr("t2")}, }, }, { sql: "INSERT INTO t (a,b,c) SELECT x,y,z FROM t1;", expect: []*ast.TableName{ - {Name: model.NewCIStr("t")}, - {Name: model.NewCIStr("t1")}, + {Name: ast.NewCIStr("t")}, + {Name: ast.NewCIStr("t1")}, }, }, { sql: "WITH t AS (SELECT * FROM v) SELECT a FROM t UNION SELECT b FROM t1", expect: []*ast.TableName{ - {Name: model.NewCIStr("v")}, - {Name: model.NewCIStr("t")}, - {Name: model.NewCIStr("t1")}, + {Name: ast.NewCIStr("v")}, + {Name: ast.NewCIStr("t")}, + {Name: ast.NewCIStr("t1")}, }, }, { sql: "LOAD DATA INFILE '/a.csv' FORMAT 'sql file' INTO TABLE `t`", expect: []*ast.TableName{ - {Name: model.NewCIStr("t")}, + {Name: ast.NewCIStr("t")}, }, }, { sql: "batch on c limit 10 delete from t where t.c = 10", expect: []*ast.TableName{ - {Name: model.NewCIStr("t")}, + {Name: ast.NewCIStr("t")}, }, }, { sql: "split table t1 between () and () regions 10", expect: []*ast.TableName{ - {Name: model.NewCIStr("t1")}, + {Name: ast.NewCIStr("t1")}, }, }, { sql: "show create table t", expect: []*ast.TableName{ - {Name: model.NewCIStr("t")}, + {Name: ast.NewCIStr("t")}, }, }, { sql: "show create database test", expect: []*ast.TableName{ - {Schema: model.NewCIStr("test")}, + {Schema: ast.NewCIStr("test")}, }, }, { sql: "create database test", expect: []*ast.TableName{ - {Schema: model.NewCIStr("test")}, + {Schema: ast.NewCIStr("test")}, }, }, { sql: "FLASHBACK DATABASE t1 TO t2", expect: []*ast.TableName{ - {Schema: model.NewCIStr("t1")}, - {Schema: model.NewCIStr("t2")}, + {Schema: ast.NewCIStr("t1")}, + {Schema: ast.NewCIStr("t2")}, }, }, { sql: "flashback table t,t1,test.t2 to timestamp '2021-05-26 16:45:26'", expect: []*ast.TableName{ - {Name: model.NewCIStr("t")}, - {Name: model.NewCIStr("t1")}, - {Name: model.NewCIStr("t2"), Schema: model.NewCIStr("test")}, + {Name: ast.NewCIStr("t")}, + {Name: ast.NewCIStr("t1")}, + {Name: ast.NewCIStr("t2"), Schema: ast.NewCIStr("test")}, }, }, { sql: "flashback database test to timestamp '2021-05-26 16:45:26'", expect: []*ast.TableName{ - {Schema: model.NewCIStr("test")}, + {Schema: ast.NewCIStr("test")}, }, }, { sql: "flashback table t TO t1", expect: []*ast.TableName{ - {Name: model.NewCIStr("t")}, - {Name: model.NewCIStr("t1")}, + {Name: ast.NewCIStr("t")}, + {Name: ast.NewCIStr("t1")}, }, }, { sql: "create table t", expect: []*ast.TableName{ - {Name: model.NewCIStr("t")}, + {Name: ast.NewCIStr("t")}, }, }, { sql: "RENAME TABLE t TO t1, test.t2 TO test.t3", expect: []*ast.TableName{ - {Name: model.NewCIStr("t")}, - {Name: model.NewCIStr("t1")}, - {Name: model.NewCIStr("t2"), Schema: model.NewCIStr("test")}, - {Name: model.NewCIStr("t3"), Schema: model.NewCIStr("test")}, + {Name: ast.NewCIStr("t")}, + {Name: ast.NewCIStr("t1")}, + {Name: ast.NewCIStr("t2"), Schema: ast.NewCIStr("test")}, + {Name: ast.NewCIStr("t3"), Schema: ast.NewCIStr("test")}, }, }, { sql: "drop table test.t, t1", expect: []*ast.TableName{ - {Name: model.NewCIStr("t1")}, - {Name: model.NewCIStr("t"), Schema: model.NewCIStr("test")}, + {Name: ast.NewCIStr("t1")}, + {Name: ast.NewCIStr("t"), Schema: ast.NewCIStr("test")}, }, }, { sql: "create view v as (select * from t)", expect: []*ast.TableName{ - {Name: model.NewCIStr("v")}, - {Name: model.NewCIStr("t")}, + {Name: ast.NewCIStr("v")}, + {Name: ast.NewCIStr("t")}, }, }, { sql: "create sequence if not exists seq no cycle", expect: []*ast.TableName{ - {Name: model.NewCIStr("seq")}, + {Name: ast.NewCIStr("seq")}, }, }, { sql: "CREATE INDEX idx ON t ( a ) VISIBLE INVISIBLE", expect: []*ast.TableName{ - {Name: model.NewCIStr("t")}, + {Name: ast.NewCIStr("t")}, }, }, { sql: "LOCK TABLE t1 WRITE, t2 READ", expect: []*ast.TableName{ - {Name: model.NewCIStr("t1")}, - {Name: model.NewCIStr("t2")}, + {Name: ast.NewCIStr("t1")}, + {Name: ast.NewCIStr("t2")}, }, }, { sql: "grant select on test.* to u1", expect: []*ast.TableName{ - {Schema: model.NewCIStr("test")}, + {Schema: ast.NewCIStr("test")}, }, }, { sql: "BACKUP TABLE a.b,c.d,e TO 'noop://'", expect: []*ast.TableName{ - {Name: model.NewCIStr("b"), Schema: model.NewCIStr("a")}, - {Name: model.NewCIStr("d"), Schema: model.NewCIStr("c")}, - {Name: model.NewCIStr("e")}, + {Name: ast.NewCIStr("b"), Schema: ast.NewCIStr("a")}, + {Name: ast.NewCIStr("d"), Schema: ast.NewCIStr("c")}, + {Name: ast.NewCIStr("e")}, }, }, { sql: "TRACE SELECT (SELECT a,b,c FROM t1) AS t WHERE t.a = 1", expect: []*ast.TableName{ - {Name: model.NewCIStr("t1")}, + {Name: ast.NewCIStr("t1")}, }, }, { sql: "EXPLAIN SELECT (SELECT a,b,c FROM t1) AS t WHERE t.a = 1", expect: []*ast.TableName{ - {Name: model.NewCIStr("t1")}, + {Name: ast.NewCIStr("t1")}, }, }, { sql: "PLAN REPLAYER DUMP EXPLAIN SELECT (SELECT a,b,c FROM t1) AS t WHERE t.a = 1", expect: []*ast.TableName{ - {Name: model.NewCIStr("t1")}, + {Name: ast.NewCIStr("t1")}, }, }, { sql: "ALTER TABLE t COMPACT", expect: []*ast.TableName{ - {Name: model.NewCIStr("t")}, + {Name: ast.NewCIStr("t")}, }, }, } diff --git a/pkg/planner/indexadvisor/BUILD.bazel b/pkg/planner/indexadvisor/BUILD.bazel index 7dc47aac1d653..a8f5a50057f30 100644 --- a/pkg/planner/indexadvisor/BUILD.bazel +++ b/pkg/planner/indexadvisor/BUILD.bazel @@ -19,7 +19,6 @@ go_library( "//pkg/meta/model", "//pkg/parser", "//pkg/parser/ast", - "//pkg/parser/model", "//pkg/parser/mysql", "//pkg/parser/opcode", "//pkg/planner/util/fixcontrol", diff --git a/pkg/planner/indexadvisor/optimizer.go b/pkg/planner/indexadvisor/optimizer.go index c24036e7da645..db2fd01714f89 100644 --- a/pkg/planner/indexadvisor/optimizer.go +++ b/pkg/planner/indexadvisor/optimizer.go @@ -23,7 +23,6 @@ import ( "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/ast" - model2 "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/planner/util/fixcontrol" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/types" @@ -75,7 +74,7 @@ func (opt *optimizerImpl) is() infoschema.InfoSchema { // IndexNameExist returns whether the specified index name exists in the specified table. func (opt *optimizerImpl) IndexNameExist(schema, table, indexName string) (bool, error) { - tbl, err := opt.is().TableByName(context.Background(), model2.NewCIStr(schema), model2.NewCIStr(table)) + tbl, err := opt.is().TableByName(context.Background(), ast.NewCIStr(schema), ast.NewCIStr(table)) if err != nil { return false, err } @@ -89,7 +88,7 @@ func (opt *optimizerImpl) IndexNameExist(schema, table, indexName string) (bool, // TableColumns returns the columns of the specified table. func (opt *optimizerImpl) TableColumns(schema, table string) ([]Column, error) { - tbl, err := opt.is().TableByName(context.Background(), model2.NewCIStr(schema), model2.NewCIStr(table)) + tbl, err := opt.is().TableByName(context.Background(), ast.NewCIStr(schema), ast.NewCIStr(table)) if err != nil { return nil, err } @@ -114,7 +113,7 @@ func (opt *optimizerImpl) PossibleColumns(schema, colName string) ([]Column, err } cols := make([]Column, 0) - tbls, err := opt.is().SchemaTableInfos(context.Background(), model2.NewCIStr(schema)) + tbls, err := opt.is().SchemaTableInfos(context.Background(), ast.NewCIStr(schema)) if err != nil { return nil, err } @@ -134,7 +133,7 @@ func (opt *optimizerImpl) PossibleColumns(schema, colName string) ([]Column, err // PrefixContainIndex returns whether the specified index is a prefix of an existing index. func (opt *optimizerImpl) PrefixContainIndex(idx Index) (bool, error) { - tbl, err := opt.is().TableByName(context.Background(), model2.NewCIStr(idx.SchemaName), model2.NewCIStr(idx.TableName)) + tbl, err := opt.is().TableByName(context.Background(), ast.NewCIStr(idx.SchemaName), ast.NewCIStr(idx.TableName)) if err != nil { return false, err } @@ -158,7 +157,7 @@ func (opt *optimizerImpl) PrefixContainIndex(idx Index) (bool, error) { // ColumnType returns the column type of the specified column. func (opt *optimizerImpl) ColumnType(c Column) (*types.FieldType, error) { - tbl, err := opt.is().TableByName(context.Background(), model2.NewCIStr(c.SchemaName), model2.NewCIStr(c.TableName)) + tbl, err := opt.is().TableByName(context.Background(), ast.NewCIStr(c.SchemaName), ast.NewCIStr(c.TableName)) if err != nil { return nil, err } @@ -172,7 +171,7 @@ func (opt *optimizerImpl) ColumnType(c Column) (*types.FieldType, error) { func (opt *optimizerImpl) addHypoIndex(hypoIndexes ...Index) error { for _, h := range hypoIndexes { - tInfo, err := opt.is().TableByName(context.Background(), model2.NewCIStr(h.SchemaName), model2.NewCIStr(h.TableName)) + tInfo, err := opt.is().TableByName(context.Background(), ast.NewCIStr(h.SchemaName), ast.NewCIStr(h.TableName)) if err != nil { return err } @@ -190,16 +189,16 @@ func (opt *optimizerImpl) addHypoIndex(hypoIndexes ...Index) error { return fmt.Errorf("column %v not found in table %v.%v", col.ColumnName, h.SchemaName, h.TableName) } cols = append(cols, &model.IndexColumn{ - Name: model2.NewCIStr(col.ColumnName), + Name: ast.NewCIStr(col.ColumnName), Offset: colOffset, Length: types.UnspecifiedLength, }) } idxInfo := &model.IndexInfo{ - Name: model2.NewCIStr(h.IndexName), + Name: ast.NewCIStr(h.IndexName), Columns: cols, State: model.StatePublic, - Tp: model2.IndexTypeHypo, + Tp: ast.IndexTypeHypo, } if opt.sctx.GetSessionVars().HypoIndexes == nil { @@ -246,7 +245,7 @@ func (opt *optimizerImpl) QueryPlanCost(sql string, hypoIndexes ...Index) (cost // EstIndexSize return the estimated index size of the specified table and columns func (opt *optimizerImpl) EstIndexSize(db, table string, cols ...string) (indexSize float64, err error) { - tbl, err := opt.is().TableByName(context.Background(), model2.NewCIStr(db), model2.NewCIStr(table)) + tbl, err := opt.is().TableByName(context.Background(), ast.NewCIStr(db), ast.NewCIStr(table)) if err != nil { return 0, err } diff --git a/pkg/planner/optimize.go b/pkg/planner/optimize.go index ec4232b4d0e6d..526e23c14905e 100644 --- a/pkg/planner/optimize.go +++ b/pkg/planner/optimize.go @@ -28,7 +28,6 @@ import ( "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser/ast" - "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/core/resolve" @@ -557,8 +556,8 @@ func setVarHintChecker(varName, hint string) (ok bool, warning error) { return true, warning } -func hypoIndexChecker(ctx context.Context, is infoschema.InfoSchema) func(db, tbl, col model.CIStr) (colOffset int, err error) { - return func(db, tbl, col model.CIStr) (colOffset int, err error) { +func hypoIndexChecker(ctx context.Context, is infoschema.InfoSchema) func(db, tbl, col ast.CIStr) (colOffset int, err error) { + return func(db, tbl, col ast.CIStr) (colOffset int, err error) { t, err := is.TableByName(ctx, db, tbl) if err != nil { return 0, errors.NewNoStackErrorf("table '%v.%v' doesn't exist", db, tbl) diff --git a/pkg/planner/util/BUILD.bazel b/pkg/planner/util/BUILD.bazel index 5212f0e28b71b..3735a0fc8e494 100644 --- a/pkg/planner/util/BUILD.bazel +++ b/pkg/planner/util/BUILD.bazel @@ -19,7 +19,6 @@ go_library( "//pkg/kv", "//pkg/meta/model", "//pkg/parser/ast", - "//pkg/parser/model", "//pkg/parser/mysql", "//pkg/planner/cascades/base", "//pkg/planner/core/base", diff --git a/pkg/planner/util/misc.go b/pkg/planner/util/misc.go index 623219d151e28..08787d3193a0d 100644 --- a/pkg/planner/util/misc.go +++ b/pkg/planner/util/misc.go @@ -27,7 +27,6 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/ast" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/planner/core/base" "github.com/pingcap/tidb/pkg/planner/property" "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" @@ -118,11 +117,11 @@ func CloneFieldNames(names []*types.FieldName) []*types.FieldName { } // CloneCIStrs uses ast.CIStr.Clone to clone a slice of ast.CIStr. -func CloneCIStrs(strs []pmodel.CIStr) []pmodel.CIStr { +func CloneCIStrs(strs []ast.CIStr) []ast.CIStr { if strs == nil { return nil } - cloned := make([]pmodel.CIStr, 0, len(strs)) + cloned := make([]ast.CIStr, 0, len(strs)) cloned = append(cloned, strs...) return cloned } @@ -362,7 +361,7 @@ func ExtractTableAlias(p base.Plan, parentOffset int) *h.HintedTable { } dbName := firstName.DBName if dbName.L == "" { - dbName = pmodel.NewCIStr(p.SCtx().GetSessionVars().CurrentDB) + dbName = ast.NewCIStr(p.SCtx().GetSessionVars().CurrentDB) } return &h.HintedTable{DBName: dbName, TblName: firstName.TblName, SelectOffset: qbOffset} } diff --git a/pkg/resourcegroup/runaway/BUILD.bazel b/pkg/resourcegroup/runaway/BUILD.bazel index 4d0fc71efbce8..b333ed7aa0d12 100644 --- a/pkg/resourcegroup/runaway/BUILD.bazel +++ b/pkg/resourcegroup/runaway/BUILD.bazel @@ -15,7 +15,7 @@ go_library( "//pkg/infoschema", "//pkg/kv", "//pkg/metrics", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/parser/terror", "//pkg/sessionctx", "//pkg/ttl/cache", diff --git a/pkg/resourcegroup/runaway/record.go b/pkg/resourcegroup/runaway/record.go index 3286e0326034c..fa4160f286a5f 100644 --- a/pkg/resourcegroup/runaway/record.go +++ b/pkg/resourcegroup/runaway/record.go @@ -25,7 +25,7 @@ import ( "github.com/pingcap/failpoint" rmpb "github.com/pingcap/kvproto/pkg/resource_manager" "github.com/pingcap/tidb/pkg/kv" - "github.com/pingcap/tidb/pkg/parser/model" + "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/ttl/cache" @@ -218,7 +218,7 @@ func (rm *Manager) deleteExpiredRows(expiredDuration time.Duration) { tableName = "tidb_runaway_queries" colName = "start_time" ) - var systemSchemaCIStr = model.NewCIStr("mysql") + var systemSchemaCIStr = ast.NewCIStr("mysql") if !rm.ddl.OwnerManager().IsOwner() { return @@ -227,7 +227,7 @@ func (rm *Manager) deleteExpiredRows(expiredDuration time.Duration) { expiredDuration = time.Second * 1 }) expiredTime := time.Now().Add(-expiredDuration) - tbCIStr := model.NewCIStr(tableName) + tbCIStr := ast.NewCIStr(tableName) tbl, err := rm.infoCache.GetLatest().TableByName(context.Background(), systemSchemaCIStr, tbCIStr) if err != nil { logutil.BgLogger().Error("delete system table failed", zap.String("table", tableName), zap.Error(err)) @@ -239,7 +239,7 @@ func (rm *Manager) deleteExpiredRows(expiredDuration time.Duration) { logutil.BgLogger().Error("time column is not public in table", zap.String("table", tableName), zap.String("column", colName)) return } - tb, err := cache.NewBasePhysicalTable(systemSchemaCIStr, tbInfo, model.NewCIStr(""), col) + tb, err := cache.NewBasePhysicalTable(systemSchemaCIStr, tbInfo, ast.NewCIStr(""), col) if err != nil { logutil.BgLogger().Error("delete system table failed", zap.String("table", tableName), zap.Error(err)) return diff --git a/pkg/resourcegroup/tests/BUILD.bazel b/pkg/resourcegroup/tests/BUILD.bazel index ebc61d5cba148..962a84d8623d7 100644 --- a/pkg/resourcegroup/tests/BUILD.bazel +++ b/pkg/resourcegroup/tests/BUILD.bazel @@ -13,8 +13,8 @@ go_test( "//pkg/domain/infosync", "//pkg/errno", "//pkg/meta/model", + "//pkg/parser/ast", "//pkg/parser/auth", - "//pkg/parser/model", "//pkg/server", "//pkg/sessionctx", "//pkg/testkit", diff --git a/pkg/resourcegroup/tests/resource_group_test.go b/pkg/resourcegroup/tests/resource_group_test.go index 578c86fcc1af9..f50f1bdacfba1 100644 --- a/pkg/resourcegroup/tests/resource_group_test.go +++ b/pkg/resourcegroup/tests/resource_group_test.go @@ -29,8 +29,8 @@ import ( "github.com/pingcap/tidb/pkg/domain/infosync" mysql "github.com/pingcap/tidb/pkg/errno" "github.com/pingcap/tidb/pkg/meta/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/auth" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/server" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/testkit" @@ -105,8 +105,8 @@ func TestResourceGroupBasic(t *testing.T) { re.Equal(uint64(2000), g.RURate) re.Equal(int64(-1), g.BurstLimit) re.Equal(uint64(time.Second*15/time.Millisecond), g.Runaway.ExecElapsedTimeMs) - re.Equal(pmodel.RunawayActionDryRun, g.Runaway.Action) - re.Equal(pmodel.WatchSimilar, g.Runaway.WatchType) + re.Equal(ast.RunawayActionDryRun, g.Runaway.Action) + re.Equal(ast.WatchSimilar, g.Runaway.WatchType) re.Equal(int64(time.Minute*10/time.Millisecond), g.Runaway.WatchDurationMs) tk.MustExec("alter resource group x QUERY_LIMIT=(EXEC_ELAPSED='20s' ACTION DRYRUN WATCH SIMILAR) BURSTABLE=FALSE") @@ -114,8 +114,8 @@ func TestResourceGroupBasic(t *testing.T) { re.Equal(uint64(2000), g.RURate) re.Equal(int64(2000), g.BurstLimit) re.Equal(uint64(time.Second*20/time.Millisecond), g.Runaway.ExecElapsedTimeMs) - re.Equal(pmodel.RunawayActionDryRun, g.Runaway.Action) - re.Equal(pmodel.WatchSimilar, g.Runaway.WatchType) + re.Equal(ast.RunawayActionDryRun, g.Runaway.Action) + re.Equal(ast.WatchSimilar, g.Runaway.WatchType) re.Equal(int64(0), g.Runaway.WatchDurationMs) tk.MustQuery("select * from information_schema.resource_groups where name = 'x'").Check(testkit.Rows("x 2000 MEDIUM NO EXEC_ELAPSED='20s', ACTION=DRYRUN, WATCH=SIMILAR DURATION=UNLIMITED ")) @@ -125,9 +125,9 @@ func TestResourceGroupBasic(t *testing.T) { re.Equal(uint64(math.MaxInt32), g.RURate) re.Equal(int64(-1), g.BurstLimit) re.Equal(uint64(time.Second*15/time.Millisecond), g.Runaway.ExecElapsedTimeMs) - re.Equal(pmodel.RunawayActionSwitchGroup, g.Runaway.Action) + re.Equal(ast.RunawayActionSwitchGroup, g.Runaway.Action) re.Equal("y", g.Runaway.SwitchGroupName) - re.Equal(pmodel.WatchSimilar, g.Runaway.WatchType) + re.Equal(ast.WatchSimilar, g.Runaway.WatchType) re.Equal(int64(time.Minute*10/time.Millisecond), g.Runaway.WatchDurationMs) tk.MustQuery("select * from information_schema.resource_groups where name = 'x'").Check(testkit.Rows("x UNLIMITED MEDIUM YES EXEC_ELAPSED='15s', ACTION=SWITCH_GROUP(y), WATCH=SIMILAR DURATION='10m0s' ")) @@ -181,7 +181,7 @@ func TestResourceGroupBasic(t *testing.T) { re.Equal(uint64(5000), groupInfo.RURate) re.Equal(int64(-1), groupInfo.BurstLimit) re.Equal(uint64(time.Second*15/time.Millisecond), groupInfo.Runaway.ExecElapsedTimeMs) - re.Equal(pmodel.RunawayActionKill, groupInfo.Runaway.Action) + re.Equal(ast.RunawayActionKill, groupInfo.Runaway.Action) re.Equal(int64(0), groupInfo.Runaway.WatchDurationMs) } g = testResourceGroupNameFromIS(t, tk.Session(), "y") @@ -299,7 +299,7 @@ func testResourceGroupNameFromIS(t *testing.T, ctx sessionctx.Context, name stri // Make sure the table schema is the new schema. err := dom.Reload() require.NoError(t, err) - g, _ := dom.InfoSchema().ResourceGroupByName(pmodel.NewCIStr(name)) + g, _ := dom.InfoSchema().ResourceGroupByName(ast.NewCIStr(name)) return g } @@ -608,7 +608,7 @@ func TestNewResourceGroupFromOptions(t *testing.T) { input: &model.ResourceGroupSettings{ Runaway: &model.ResourceGroupRunawaySettings{ ExecElapsedTimeMs: 1000, - Action: pmodel.RunawayActionSwitchGroup, + Action: ast.RunawayActionSwitchGroup, SwitchGroupName: "", }, }, diff --git a/pkg/server/BUILD.bazel b/pkg/server/BUILD.bazel index 5d543393cdb6c..4d54f8db8eafd 100644 --- a/pkg/server/BUILD.bazel +++ b/pkg/server/BUILD.bazel @@ -165,9 +165,9 @@ go_test( "//pkg/meta/model", "//pkg/metrics", "//pkg/param", + "//pkg/parser/ast", "//pkg/parser/auth", "//pkg/parser/charset", - "//pkg/parser/model", "//pkg/parser/mysql", "//pkg/parser/terror", "//pkg/planner/core/resolve", diff --git a/pkg/server/driver_tidb_test.go b/pkg/server/driver_tidb_test.go index f916da009076e..40b4a585037bd 100644 --- a/pkg/server/driver_tidb_test.go +++ b/pkg/server/driver_tidb_test.go @@ -18,8 +18,8 @@ import ( "testing" "github.com/pingcap/tidb/pkg/meta/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/charset" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/core/resolve" "github.com/pingcap/tidb/pkg/server/internal/column" @@ -48,15 +48,15 @@ func TestConvertColumnInfo(t *testing.T) { ftb.SetType(mysql.TypeBit).SetFlag(mysql.UnsignedFlag).SetFlen(1).SetCharset(charset.CharsetUTF8).SetCollate(charset.CollationUTF8) resultField := resolve.ResultField{ Column: &model.ColumnInfo{ - Name: pmodel.NewCIStr("a"), + Name: ast.NewCIStr("a"), ID: 0, Offset: 0, FieldType: ftb.Build(), Comment: "column a is the first column in table dual", }, - ColumnAsName: pmodel.NewCIStr("a"), - TableAsName: pmodel.NewCIStr("dual"), - DBName: pmodel.NewCIStr("test"), + ColumnAsName: ast.NewCIStr("a"), + TableAsName: ast.NewCIStr("dual"), + DBName: ast.NewCIStr("test"), } colInfo := column.ConvertColumnInfo(&resultField) require.Equal(t, createColumnByTypeAndLen(mysql.TypeBit, 1), colInfo) @@ -66,15 +66,15 @@ func TestConvertColumnInfo(t *testing.T) { ftpb.SetType(mysql.TypeTiny).SetFlag(mysql.UnsignedFlag).SetFlen(1).SetCharset(charset.CharsetUTF8).SetCollate(charset.CollationUTF8) resultField = resolve.ResultField{ Column: &model.ColumnInfo{ - Name: pmodel.NewCIStr("a"), + Name: ast.NewCIStr("a"), ID: 0, Offset: 0, FieldType: ftpb.Build(), Comment: "column a is the first column in table dual", }, - ColumnAsName: pmodel.NewCIStr("a"), - TableAsName: pmodel.NewCIStr("dual"), - DBName: pmodel.NewCIStr("test"), + ColumnAsName: ast.NewCIStr("a"), + TableAsName: ast.NewCIStr("dual"), + DBName: ast.NewCIStr("test"), } colInfo = column.ConvertColumnInfo(&resultField) require.Equal(t, createColumnByTypeAndLen(mysql.TypeTiny, 1), colInfo) @@ -83,15 +83,15 @@ func TestConvertColumnInfo(t *testing.T) { ftpb1.SetType(mysql.TypeYear).SetFlag(mysql.ZerofillFlag).SetFlen(4).SetCharset(charset.CharsetBin).SetCollate(charset.CollationBin) resultField = resolve.ResultField{ Column: &model.ColumnInfo{ - Name: pmodel.NewCIStr("a"), + Name: ast.NewCIStr("a"), ID: 0, Offset: 0, FieldType: ftpb1.Build(), Comment: "column a is the first column in table dual", }, - ColumnAsName: pmodel.NewCIStr("a"), - TableAsName: pmodel.NewCIStr("dual"), - DBName: pmodel.NewCIStr("test"), + ColumnAsName: ast.NewCIStr("a"), + TableAsName: ast.NewCIStr("dual"), + DBName: ast.NewCIStr("test"), } colInfo = column.ConvertColumnInfo(&resultField) require.Equal(t, uint32(4), colInfo.ColumnLength) diff --git a/pkg/server/handler/BUILD.bazel b/pkg/server/handler/BUILD.bazel index 691aa81b23077..e6f9ffac0bf2b 100644 --- a/pkg/server/handler/BUILD.bazel +++ b/pkg/server/handler/BUILD.bazel @@ -14,7 +14,7 @@ go_library( "//pkg/infoschema", "//pkg/kv", "//pkg/meta/model", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/parser/terror", "//pkg/session", "//pkg/sessionctx/stmtctx", diff --git a/pkg/server/handler/optimizor/BUILD.bazel b/pkg/server/handler/optimizor/BUILD.bazel index 6bd6840e5669c..1c30409cf01ee 100644 --- a/pkg/server/handler/optimizor/BUILD.bazel +++ b/pkg/server/handler/optimizor/BUILD.bazel @@ -14,7 +14,7 @@ go_library( "//pkg/domain/infosync", "//pkg/infoschema", "//pkg/meta/model", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/parser/mysql", "//pkg/server/handler", "//pkg/sessionctx/variable", @@ -50,7 +50,7 @@ go_test( "//pkg/domain", "//pkg/kv", "//pkg/metrics", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/server", "//pkg/server/internal/testserverclient", "//pkg/server/internal/testutil", diff --git a/pkg/server/handler/optimizor/plan_replayer.go b/pkg/server/handler/optimizor/plan_replayer.go index 4fda19bbcc5c1..ed04611c539fc 100644 --- a/pkg/server/handler/optimizor/plan_replayer.go +++ b/pkg/server/handler/optimizor/plan_replayer.go @@ -35,7 +35,7 @@ import ( "github.com/pingcap/tidb/pkg/domain/infosync" "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/server/handler" "github.com/pingcap/tidb/pkg/statistics/handle" util2 "github.com/pingcap/tidb/pkg/statistics/util" @@ -308,7 +308,7 @@ func loadSchemaMeta(z *zip.Reader, is infoschema.InfoSchema) (map[int64]*tblInfo s := strings.Split(row, ";") databaseName := s[0] tableName := s[1] - t, err := is.TableByName(context.Background(), pmodel.NewCIStr(databaseName), pmodel.NewCIStr(tableName)) + t, err := is.TableByName(context.Background(), ast.NewCIStr(databaseName), ast.NewCIStr(tableName)) if err != nil { return nil, err } diff --git a/pkg/server/handler/optimizor/plan_replayer_test.go b/pkg/server/handler/optimizor/plan_replayer_test.go index dd053ea48f469..928edcbaf0e73 100644 --- a/pkg/server/handler/optimizor/plan_replayer_test.go +++ b/pkg/server/handler/optimizor/plan_replayer_test.go @@ -35,7 +35,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/kv" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/server" "github.com/pingcap/tidb/pkg/server/internal/testserverclient" "github.com/pingcap/tidb/pkg/server/internal/testutil" @@ -587,7 +587,7 @@ func TestDumpPlanReplayerAPIWithHistoryStats(t *testing.T) { tk.MustExec("use test") tk.MustExec("create table t(a int, b int, c int, index ia(a))") is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tblInfo := tbl.Meta() diff --git a/pkg/server/handler/optimizor/statistics_handler.go b/pkg/server/handler/optimizor/statistics_handler.go index 8a0fe4264f4ef..56fce1180c241 100644 --- a/pkg/server/handler/optimizor/statistics_handler.go +++ b/pkg/server/handler/optimizor/statistics_handler.go @@ -23,7 +23,7 @@ import ( "github.com/gorilla/mux" "github.com/pingcap/tidb/pkg/domain" - "github.com/pingcap/tidb/pkg/parser/model" + "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" @@ -67,7 +67,7 @@ func (sh StatsHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) { return } } - tbl, err := is.TableByName(context.Background(), model.NewCIStr(params[handler.DBName]), model.NewCIStr(params[handler.TableName])) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr(params[handler.DBName]), ast.NewCIStr(params[handler.TableName])) if err != nil { handler.WriteError(w, err) } else { @@ -121,7 +121,7 @@ func (sh StatsHistoryHandler) ServeHTTP(w http.ResponseWriter, req *http.Request if err != nil { logutil.BgLogger().Info("fail to get snapshot TableInfo in historical stats API, switch to use latest infoschema", zap.Error(err)) is := sh.do.InfoSchema() - tbl, err = is.TableByName(context.Background(), model.NewCIStr(params[handler.DBName]), model.NewCIStr(params[handler.TableName])) + tbl, err = is.TableByName(context.Background(), ast.NewCIStr(params[handler.DBName]), ast.NewCIStr(params[handler.TableName])) if err != nil { handler.WriteError(w, err) return @@ -142,7 +142,7 @@ func getSnapshotTableInfo(dom *domain.Domain, snapshot uint64, dbName, tblName s if err != nil { return nil, err } - return is.TableByName(context.Background(), model.NewCIStr(dbName), model.NewCIStr(tblName)) + return is.TableByName(context.Background(), ast.NewCIStr(dbName), ast.NewCIStr(tblName)) } // StatsPriorityQueueHandler is the handler for dumping the stats priority queue snapshot. diff --git a/pkg/server/handler/optimizor/statistics_handler_test.go b/pkg/server/handler/optimizor/statistics_handler_test.go index 1da7e245e2382..d2cfad94814aa 100644 --- a/pkg/server/handler/optimizor/statistics_handler_test.go +++ b/pkg/server/handler/optimizor/statistics_handler_test.go @@ -26,7 +26,7 @@ import ( "github.com/go-sql-driver/mysql" "github.com/gorilla/mux" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" server2 "github.com/pingcap/tidb/pkg/server" "github.com/pingcap/tidb/pkg/server/handler/optimizor" "github.com/pingcap/tidb/pkg/server/internal/testserverclient" @@ -70,7 +70,7 @@ func TestDumpStatsAPI(t *testing.T) { statsHandler := optimizor.NewStatsHandler(dom) prepareData(t, client, statsHandler) - tableInfo, err := dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("tidb"), model.NewCIStr("test")) + tableInfo, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("tidb"), ast.NewCIStr("test")) require.NoError(t, err) err = dom.GetHistoricalStatsWorker().DumpHistoricalStats(tableInfo.Meta().ID, dom.StatsHandle()) require.NoError(t, err) diff --git a/pkg/server/handler/tests/BUILD.bazel b/pkg/server/handler/tests/BUILD.bazel index b09d490f19320..015f338779a74 100644 --- a/pkg/server/handler/tests/BUILD.bazel +++ b/pkg/server/handler/tests/BUILD.bazel @@ -22,7 +22,7 @@ go_test( "//pkg/meta", "//pkg/meta/model", "//pkg/metrics", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/parser/mysql", "//pkg/planner/core", "//pkg/server", diff --git a/pkg/server/handler/tests/http_handler_serial_test.go b/pkg/server/handler/tests/http_handler_serial_test.go index 8fccfa8e4f527..0944533fc4ba9 100644 --- a/pkg/server/handler/tests/http_handler_serial_test.go +++ b/pkg/server/handler/tests/http_handler_serial_test.go @@ -36,7 +36,7 @@ import ( "github.com/pingcap/tidb/pkg/config" ddlutil "github.com/pingcap/tidb/pkg/ddl/util" "github.com/pingcap/tidb/pkg/kv" - "github.com/pingcap/tidb/pkg/parser/model" + "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/server/handler/tikvhandler" @@ -328,7 +328,7 @@ func TestTiFlashReplica(t *testing.T) { require.NoError(t, resp.Body.Close()) require.Equal(t, "[schema:1146]Table which ID = 184 does not exist.", string(body)) - tbl, err := ts.domain.InfoSchema().TableByName(context.Background(), model.NewCIStr("tidb"), model.NewCIStr("test")) + tbl, err := ts.domain.InfoSchema().TableByName(context.Background(), ast.NewCIStr("tidb"), ast.NewCIStr("test")) require.NoError(t, err) req := fmt.Sprintf(`{"id":%d,"region_count":3,"flash_region_count":3}`, tbl.Meta().ID) resp, err = ts.PostStatus("/tiflash/replica-deprecated", "application/json", bytes.NewBuffer([]byte(req))) diff --git a/pkg/server/handler/tests/http_handler_test.go b/pkg/server/handler/tests/http_handler_test.go index 7fe06ef783602..be1efd69a3849 100644 --- a/pkg/server/handler/tests/http_handler_test.go +++ b/pkg/server/handler/tests/http_handler_test.go @@ -49,7 +49,7 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/core" server2 "github.com/pingcap/tidb/pkg/server" @@ -1138,7 +1138,7 @@ func TestWriteDBTablesData(t *testing.T) { // No table in a schema. info := infoschema.MockInfoSchema([]*model.TableInfo{}) rc := httptest.NewRecorder() - tbs, err := info.SchemaTableInfos(context.Background(), pmodel.NewCIStr("test")) + tbs, err := info.SchemaTableInfos(context.Background(), ast.NewCIStr("test")) require.NoError(t, err) require.Equal(t, 0, len(tbs)) tikvhandler.WriteDBTablesData(rc, tbs) @@ -1151,7 +1151,7 @@ func TestWriteDBTablesData(t *testing.T) { // One table in a schema. info = infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable()}) rc = httptest.NewRecorder() - tbs, err = info.SchemaTableInfos(context.Background(), pmodel.NewCIStr("test")) + tbs, err = info.SchemaTableInfos(context.Background(), ast.NewCIStr("test")) require.NoError(t, err) require.Equal(t, 1, len(tbs)) tikvhandler.WriteDBTablesData(rc, tbs) @@ -1165,7 +1165,7 @@ func TestWriteDBTablesData(t *testing.T) { // Two tables in a schema. info = infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) rc = httptest.NewRecorder() - tbs, err = info.SchemaTableInfos(context.Background(), pmodel.NewCIStr("test")) + tbs, err = info.SchemaTableInfos(context.Background(), ast.NewCIStr("test")) require.NoError(t, err) require.Equal(t, 2, len(tbs)) tikvhandler.WriteDBTablesData(rc, tbs) diff --git a/pkg/server/handler/tikv_handler.go b/pkg/server/handler/tikv_handler.go index 7cd75c81a044f..048abd25fd91b 100644 --- a/pkg/server/handler/tikv_handler.go +++ b/pkg/server/handler/tikv_handler.go @@ -30,7 +30,7 @@ import ( "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/session" "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" derr "github.com/pingcap/tidb/pkg/store/driver/error" @@ -192,7 +192,7 @@ func (t *TikvHandlerTool) GetTable(dbName, tableName string) (table.PhysicalTabl return nil, errors.Trace(err) } tableName, partitionName := ExtractTableAndPartitionName(tableName) - tableVal, err := schema.TableByName(context.Background(), pmodel.NewCIStr(dbName), pmodel.NewCIStr(tableName)) + tableVal, err := schema.TableByName(context.Background(), ast.NewCIStr(dbName), ast.NewCIStr(tableName)) if err != nil { return nil, errors.Trace(err) } diff --git a/pkg/server/handler/tikvhandler/BUILD.bazel b/pkg/server/handler/tikvhandler/BUILD.bazel index 5fac3f0e22e4d..08dc460b24a10 100644 --- a/pkg/server/handler/tikvhandler/BUILD.bazel +++ b/pkg/server/handler/tikvhandler/BUILD.bazel @@ -16,7 +16,7 @@ go_library( "//pkg/kv", "//pkg/meta", "//pkg/meta/model", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/parser/terror", "//pkg/server/handler", "//pkg/session", diff --git a/pkg/server/handler/tikvhandler/tikv_handler.go b/pkg/server/handler/tikvhandler/tikv_handler.go index 5f68499f90193..7707d9f562df7 100644 --- a/pkg/server/handler/tikvhandler/tikv_handler.go +++ b/pkg/server/handler/tikvhandler/tikv_handler.go @@ -44,7 +44,7 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/server/handler" "github.com/pingcap/tidb/pkg/session" @@ -765,7 +765,7 @@ type SchemaTableStorage struct { DataFree int64 `json:"data_free"` } -func getSchemaTablesStorageInfo(h *SchemaStorageHandler, schema *pmodel.CIStr, table *pmodel.CIStr) (messages []*SchemaTableStorage, err error) { +func getSchemaTablesStorageInfo(h *SchemaStorageHandler, schema *ast.CIStr, table *ast.CIStr) (messages []*SchemaTableStorage, err error) { var s sessiontypes.Session if s, err = session.CreateSession(h.Store); err != nil { return @@ -836,13 +836,13 @@ func (h SchemaStorageHandler) ServeHTTP(w http.ResponseWriter, req *http.Request params := mux.Vars(req) var ( - dbName *pmodel.CIStr - tableName *pmodel.CIStr + dbName *ast.CIStr + tableName *ast.CIStr isSingle bool ) if reqDbName, ok := params[handler.DBName]; ok { - cDBName := pmodel.NewCIStr(reqDbName) + cDBName := ast.NewCIStr(reqDbName) // all table schemas in a specified database schemaInfo, exists := schema.SchemaByName(cDBName) if !exists { @@ -853,7 +853,7 @@ func (h SchemaStorageHandler) ServeHTTP(w http.ResponseWriter, req *http.Request if reqTableName, ok := params[handler.TableName]; ok { // table schema of a specified table name - cTableName := pmodel.NewCIStr(reqTableName) + cTableName := ast.NewCIStr(reqTableName) data, e := schema.TableByName(context.Background(), cDBName, cTableName) if e != nil { handler.WriteError(w, e) @@ -952,10 +952,10 @@ func (h SchemaHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) { params := mux.Vars(req) if dbName, ok := params[handler.DBName]; ok { - cDBName := pmodel.NewCIStr(dbName) + cDBName := ast.NewCIStr(dbName) if tableName, ok := params[handler.TableName]; ok { // table schema of a specified table name - cTableName := pmodel.NewCIStr(tableName) + cTableName := ast.NewCIStr(tableName) data, err := schema.TableByName(context.Background(), cDBName, cTableName) if err != nil { handler.WriteError(w, err) @@ -1051,7 +1051,7 @@ func (h *TableHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) { } tableName, partitionName := handler.ExtractTableAndPartitionName(tableName) - tableVal, err := schema.TableByName(context.Background(), pmodel.NewCIStr(dbName), pmodel.NewCIStr(tableName)) + tableVal, err := schema.TableByName(context.Background(), ast.NewCIStr(dbName), ast.NewCIStr(tableName)) if err != nil { handler.WriteError(w, err) return diff --git a/pkg/session/BUILD.bazel b/pkg/session/BUILD.bazel index f2da24851b1d8..f8844b4717e78 100644 --- a/pkg/session/BUILD.bazel +++ b/pkg/session/BUILD.bazel @@ -54,7 +54,6 @@ go_library( "//pkg/parser/auth", "//pkg/parser/charset", "//pkg/parser/format", - "//pkg/parser/model", "//pkg/parser/mysql", "//pkg/parser/opcode", "//pkg/parser/terror", diff --git a/pkg/session/bootstrap.go b/pkg/session/bootstrap.go index 61a207c347667..68aaea4b2a3bc 100644 --- a/pkg/session/bootstrap.go +++ b/pkg/session/bootstrap.go @@ -43,8 +43,8 @@ import ( "github.com/pingcap/tidb/pkg/meta" "github.com/pingcap/tidb/pkg/owner" "github.com/pingcap/tidb/pkg/parser" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/auth" - "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" sessiontypes "github.com/pingcap/tidb/pkg/session/types" @@ -3228,7 +3228,7 @@ 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, model.AllColumns.String()) + initGlobalVariableIfNotExists(s, variable.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. @@ -3670,7 +3670,7 @@ func rebuildAllPartitionValueMapAndSorted(ctx context.Context, s *session) { for _, db := range dbs { for _, t := range db.TableInfos { pi := t.GetPartitionInfo() - if pi == nil || pi.Type != model.PartitionTypeList { + if pi == nil || pi.Type != ast.PartitionTypeList { continue } tbl, ok := is.TableByID(ctx, t.ID) diff --git a/pkg/session/nontransactional.go b/pkg/session/nontransactional.go index 6283d36a5b42c..8272d9d4e4806 100644 --- a/pkg/session/nontransactional.go +++ b/pkg/session/nontransactional.go @@ -28,7 +28,6 @@ import ( "github.com/pingcap/tidb/pkg/parser" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/format" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/opcode" "github.com/pingcap/tidb/pkg/planner/core" @@ -616,7 +615,7 @@ func selectShardColumn(stmt *ast.NonTransactionalDMLStmt, se sessiontypes.Sessio // the specified table must be in the join tableInJoin := false - var chosenTableName pmodel.CIStr + var chosenTableName ast.CIStr for _, tableSource := range tableSources { tableSourceName := tableSource.Source.(*ast.TableName) tableSourceFinalTableName := tableSource.AsName // precedence: alias name, then table name @@ -688,7 +687,7 @@ func collectTableSourcesInJoin(node ast.ResultSetNode, tableSources []*ast.Table // it attempts to auto-select a shard column from handle if not specified, and fills back the corresponding info in the stmt, // making it transparent to following steps func selectShardColumnFromTheOnlyTable(stmt *ast.NonTransactionalDMLStmt, tableName *ast.TableName, - tableAsName pmodel.CIStr, tbl table.Table) ( + tableAsName ast.CIStr, tbl table.Table) ( indexed bool, shardColumnInfo *model.ColumnInfo, err error) { if stmt.ShardColumn == nil { return selectShardColumnAutomatically(stmt, tbl, tableName, tableAsName) @@ -733,7 +732,7 @@ func selectShardColumnByGivenName(shardColumnName string, tbl table.Table) ( } func selectShardColumnAutomatically(stmt *ast.NonTransactionalDMLStmt, tbl table.Table, - tableName *ast.TableName, tableAsName pmodel.CIStr) (bool, *model.ColumnInfo, error) { + tableName *ast.TableName, tableAsName ast.CIStr) (bool, *model.ColumnInfo, error) { // auto-detect shard column var shardColumnInfo *model.ColumnInfo tableInfo := tbl.Meta() @@ -767,7 +766,7 @@ func selectShardColumnAutomatically(stmt *ast.NonTransactionalDMLStmt, tbl table stmt.ShardColumn = &ast.ColumnName{ Schema: tableName.Schema, Table: outputTableName, // so that table alias works - Name: pmodel.NewCIStr(shardColumnName), + Name: ast.NewCIStr(shardColumnName), } return true, shardColumnInfo, nil } @@ -784,7 +783,7 @@ func buildDryRunResults(dryRunOption int, results []string, maxChunkSize int) (s Column: &model.ColumnInfo{ FieldType: *types.NewFieldType(mysql.TypeString), }, - ColumnAsName: pmodel.NewCIStr(fieldName), + ColumnAsName: ast.NewCIStr(fieldName), }} rows := make([][]any, 0, len(results)) for _, result := range results { @@ -812,13 +811,13 @@ func buildExecuteResults(ctx context.Context, jobs []job, maxChunkSize int, reda Column: &model.ColumnInfo{ FieldType: *types.NewFieldType(mysql.TypeLong), }, - ColumnAsName: pmodel.NewCIStr("number of jobs"), + ColumnAsName: ast.NewCIStr("number of jobs"), }, { Column: &model.ColumnInfo{ FieldType: *types.NewFieldType(mysql.TypeString), }, - ColumnAsName: pmodel.NewCIStr("job status"), + ColumnAsName: ast.NewCIStr("job status"), }, } rows := make([][]any, 1) diff --git a/pkg/session/schematest/BUILD.bazel b/pkg/session/schematest/BUILD.bazel index 4bbebb9cca5e6..8a6b85c2ac110 100644 --- a/pkg/session/schematest/BUILD.bazel +++ b/pkg/session/schematest/BUILD.bazel @@ -13,7 +13,7 @@ go_test( "//pkg/config", "//pkg/domain", "//pkg/kv", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/server", "//pkg/session", "//pkg/sessionctx/variable", diff --git a/pkg/session/schematest/schema_test.go b/pkg/session/schematest/schema_test.go index 86220db845a31..383edf8c7cc7f 100644 --- a/pkg/session/schematest/schema_test.go +++ b/pkg/session/schematest/schema_test.go @@ -23,7 +23,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/kv" - "github.com/pingcap/tidb/pkg/parser/model" + "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/variable" @@ -114,7 +114,7 @@ func TestTableReaderChunk(t *testing.T) { for i := 0; i < 100; i++ { tk.MustExec(fmt.Sprintf("insert chk values (%d)", i)) } - tbl, err := domain.GetDomain(tk.Session()).InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("chk")) + tbl, err := domain.GetDomain(tk.Session()).InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("chk")) require.NoError(t, err) tableStart := tablecodec.GenTableRecordPrefix(tbl.Meta().ID) cluster.SplitKeys(tableStart, tableStart.PrefixNext(), 10) @@ -319,7 +319,7 @@ func TestIndexLookUpReaderChunk(t *testing.T) { for i := 0; i < 100; i++ { tk.MustExec(fmt.Sprintf("insert chk values (%d, %d)", i, i)) } - tbl, err := domain.GetDomain(tk.Session()).InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("chk")) + tbl, err := domain.GetDomain(tk.Session()).InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("chk")) require.NoError(t, err) indexStart := tablecodec.EncodeTableIndexPrefix(tbl.Meta().ID, tbl.Indices()[0].Meta().ID) cluster.SplitKeys(indexStart, indexStart.PrefixNext(), 10) diff --git a/pkg/session/session.go b/pkg/session/session.go index ecf79663aa04d..1cdf1bb96a8d8 100644 --- a/pkg/session/session.go +++ b/pkg/session/session.go @@ -72,7 +72,6 @@ import ( "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/auth" "github.com/pingcap/tidb/pkg/parser/charset" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/planner" @@ -238,7 +237,7 @@ var parserPool = &sync.Pool{New: func() any { return parser.New() }} func (s *session) AddTableLock(locks []model.TableLockTpInfo) { for _, l := range locks { // read only lock is session unrelated, skip it when adding lock to session. - if l.Tp != pmodel.TableLockReadOnly { + if l.Tp != ast.TableLockReadOnly { s.lockedTables[l.TableID] = l } } @@ -259,10 +258,10 @@ func (s *session) ReleaseTableLockByTableIDs(tableIDs []int64) { } // CheckTableLocked checks the table lock. -func (s *session) CheckTableLocked(tblID int64) (bool, pmodel.TableLockType) { +func (s *session) CheckTableLocked(tblID int64) (bool, ast.TableLockType) { lt, ok := s.lockedTables[tblID] if !ok { - return false, pmodel.TableLockNone + return false, ast.TableLockNone } return true, lt.Tp } @@ -429,8 +428,8 @@ func (s *session) UpdateColStatsUsage(colStatsUsage iter.Seq[model.TableItemID]) // FieldList returns fields list of a table. func (s *session) FieldList(tableName string) ([]*resolve.ResultField, error) { is := s.GetInfoSchema().(infoschema.InfoSchema) - dbName := pmodel.NewCIStr(s.GetSessionVars().CurrentDB) - tName := pmodel.NewCIStr(tableName) + dbName := ast.NewCIStr(s.GetSessionVars().CurrentDB) + tName := ast.NewCIStr(tableName) pm := privilege.GetPrivilegeManager(s) if pm != nil && s.sessionVars.User != nil { if !pm.RequestVerification(s.sessionVars.ActiveRoles, dbName.O, tName.O, "", mysql.AllPrivMask) { @@ -2103,7 +2102,7 @@ func (s *session) ExecuteStmt(ctx context.Context, stmtNode ast.StmtNode) (sqlex // infoschema there. if sessVars.StmtCtx.ResourceGroupName != sessVars.ResourceGroupName { // if target resource group doesn't exist, fallback to the origin resource group. - if _, ok := domain.GetDomain(s).InfoSchema().ResourceGroupByName(pmodel.NewCIStr(sessVars.StmtCtx.ResourceGroupName)); !ok { + if _, ok := domain.GetDomain(s).InfoSchema().ResourceGroupByName(ast.NewCIStr(sessVars.StmtCtx.ResourceGroupName)); !ok { logutil.Logger(ctx).Warn("Unknown resource group from hint", zap.String("name", sessVars.StmtCtx.ResourceGroupName)) sessVars.StmtCtx.ResourceGroupName = sessVars.ResourceGroupName if txn, err := s.Txn(false); err == nil && txn != nil && txn.Valid() { @@ -4093,7 +4092,7 @@ func logStmt(execStmt *executor.ExecStmt, s *session) { } case *ast.CreateIndexStmt: isCrucial = true - if stmt.IndexOption != nil && stmt.IndexOption.Tp == pmodel.IndexTypeHypo { + if stmt.IndexOption != nil && stmt.IndexOption.Tp == ast.IndexTypeHypo { isCrucial = false } case *ast.CreateUserStmt, *ast.DropUserStmt, *ast.AlterUserStmt, *ast.SetPwdStmt, *ast.GrantStmt, @@ -4565,7 +4564,7 @@ func (s *session) usePipelinedDmlOrWarn(ctx context.Context) bool { } for _, t := range stmtCtx.Tables { // get table schema from current infoschema - tbl, err := is.TableByName(ctx, pmodel.NewCIStr(t.DB), pmodel.NewCIStr(t.Table)) + tbl, err := is.TableByName(ctx, ast.NewCIStr(t.DB), ast.NewCIStr(t.Table)) if err != nil { stmtCtx.AppendWarning(errors.New("Pipelined DML failed to get table schema. Fallback to standard mode")) return false diff --git a/pkg/session/test/BUILD.bazel b/pkg/session/test/BUILD.bazel index b96ef309e5cd4..cd560cb769d00 100644 --- a/pkg/session/test/BUILD.bazel +++ b/pkg/session/test/BUILD.bazel @@ -17,7 +17,6 @@ go_test( "//pkg/kv", "//pkg/parser/ast", "//pkg/parser/format", - "//pkg/parser/model", "//pkg/parser/mysql", "//pkg/parser/terror", "//pkg/planner/core", diff --git a/pkg/session/test/common/BUILD.bazel b/pkg/session/test/common/BUILD.bazel index ab6fd207120ef..d7f06efc986ed 100644 --- a/pkg/session/test/common/BUILD.bazel +++ b/pkg/session/test/common/BUILD.bazel @@ -12,7 +12,7 @@ go_test( deps = [ "//pkg/config", "//pkg/expression", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/parser/mysql", "//pkg/sessionctx", "//pkg/table/tables", diff --git a/pkg/session/test/common/common_test.go b/pkg/session/test/common/common_test.go index 353206e73a5b5..1dd2a1589ef52 100644 --- a/pkg/session/test/common/common_test.go +++ b/pkg/session/test/common/common_test.go @@ -19,7 +19,7 @@ import ( "testing" "github.com/pingcap/tidb/pkg/expression" - "github.com/pingcap/tidb/pkg/parser/model" + "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/table/tables" @@ -107,7 +107,7 @@ func TestIndexColumnLength(t *testing.T) { tk.MustExec("create index idx_c2 on t(c2(6));") is := dom.InfoSchema() - tab, err2 := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tab, err2 := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err2) idxC1Cols := tables.FindIndexByColName(tab, "c1").Meta().Columns diff --git a/pkg/session/test/session_test.go b/pkg/session/test/session_test.go index b86e9d1d99e45..01631ed0308b2 100644 --- a/pkg/session/test/session_test.go +++ b/pkg/session/test/session_test.go @@ -33,7 +33,6 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/format" - "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/session" @@ -347,7 +346,7 @@ func TestDoDDLJobQuit(t *testing.T) { }) // this DDL call will enter deadloop before this fix - err = dom.DDLExecutor().CreateSchema(se, &ast.CreateDatabaseStmt{Name: model.NewCIStr("testschema")}) + err = dom.DDLExecutor().CreateSchema(se, &ast.CreateDatabaseStmt{Name: ast.NewCIStr("testschema")}) require.Equal(t, "context canceled", err.Error()) } diff --git a/pkg/sessionctx/variable/BUILD.bazel b/pkg/sessionctx/variable/BUILD.bazel index 5f326b6e4ecb6..de5ab75eaa0cf 100644 --- a/pkg/sessionctx/variable/BUILD.bazel +++ b/pkg/sessionctx/variable/BUILD.bazel @@ -30,7 +30,6 @@ go_library( "//pkg/parser/ast", "//pkg/parser/auth", "//pkg/parser/charset", - "//pkg/parser/model", "//pkg/parser/mysql", "//pkg/parser/types", "//pkg/planner/util/fixcontrol", diff --git a/pkg/sessionctx/variable/sysvar.go b/pkg/sessionctx/variable/sysvar.go index 08dc15ae20324..c75b278a8a5da 100644 --- a/pkg/sessionctx/variable/sysvar.go +++ b/pkg/sessionctx/variable/sysvar.go @@ -36,7 +36,6 @@ import ( "github.com/pingcap/tidb/pkg/parser" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/charset" - "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/util/fixcontrol" "github.com/pingcap/tidb/pkg/privilege/privileges/ldap" @@ -1082,12 +1081,12 @@ var defaultSysVars = []*SysVar{ }, Validation: func(s *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { choice := strings.ToUpper(normalizedValue) - if choice != model.AllColumns.String() && choice != model.PredicateColumns.String() { + if choice != ast.AllColumns.String() && choice != ast.PredicateColumns.String() { return "", errors.Errorf( "invalid value for %s, it should be either '%s' or '%s'", TiDBAnalyzeColumnOptions, - model.AllColumns.String(), - model.PredicateColumns.String(), + ast.AllColumns.String(), + ast.PredicateColumns.String(), ) } return normalizedValue, nil diff --git a/pkg/sessiontxn/BUILD.bazel b/pkg/sessiontxn/BUILD.bazel index 63d9d885051be..83643edc74700 100644 --- a/pkg/sessiontxn/BUILD.bazel +++ b/pkg/sessiontxn/BUILD.bazel @@ -37,7 +37,6 @@ go_test( "//pkg/kv", "//pkg/meta/model", "//pkg/parser/ast", - "//pkg/parser/model", "//pkg/sessionctx", "//pkg/sessiontxn/internal", "//pkg/sessiontxn/isolation", diff --git a/pkg/sessiontxn/txn_manager_test.go b/pkg/sessiontxn/txn_manager_test.go index 068f563fa75b2..b2282dca76043 100644 --- a/pkg/sessiontxn/txn_manager_test.go +++ b/pkg/sessiontxn/txn_manager_test.go @@ -24,7 +24,6 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/ast" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessiontxn" "github.com/pingcap/tidb/pkg/sessiontxn/internal" @@ -449,7 +448,7 @@ func TestSnapshotInterceptor(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("create temporary table test.tmp1 (id int primary key)") - tbl, err := tk.Session().GetDomainInfoSchema().(infoschema.InfoSchema).TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("tmp1")) + tbl, err := tk.Session().GetDomainInfoSchema().(infoschema.InfoSchema).TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("tmp1")) require.NoError(t, err) require.Equal(t, model.TempTableLocal, tbl.Meta().TempTableType) tblID := tbl.Meta().ID diff --git a/pkg/statistics/BUILD.bazel b/pkg/statistics/BUILD.bazel index 29fa59bd61302..5b388ddc2cc5f 100644 --- a/pkg/statistics/BUILD.bazel +++ b/pkg/statistics/BUILD.bazel @@ -86,7 +86,7 @@ go_test( deps = [ "//pkg/config", "//pkg/meta/model", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/parser/mysql", "//pkg/planner/core/resolve", "//pkg/sessionctx", diff --git a/pkg/statistics/handle/autoanalyze/BUILD.bazel b/pkg/statistics/handle/autoanalyze/BUILD.bazel index ae7b9f57d2fb7..d6ae3b4ddd9f4 100644 --- a/pkg/statistics/handle/autoanalyze/BUILD.bazel +++ b/pkg/statistics/handle/autoanalyze/BUILD.bazel @@ -10,7 +10,7 @@ go_library( "//pkg/domain/infosync", "//pkg/infoschema", "//pkg/meta/model", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/parser/terror", "//pkg/sessionctx", "//pkg/sessionctx/sysproctrack", @@ -44,7 +44,7 @@ go_test( ":autoanalyze", "//pkg/domain", "//pkg/domain/infosync", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/parser/mysql", "//pkg/sessionctx", "//pkg/sessionctx/variable", diff --git a/pkg/statistics/handle/autoanalyze/autoanalyze.go b/pkg/statistics/handle/autoanalyze/autoanalyze.go index d752ffc143094..47504b5907a23 100644 --- a/pkg/statistics/handle/autoanalyze/autoanalyze.go +++ b/pkg/statistics/handle/autoanalyze/autoanalyze.go @@ -29,7 +29,7 @@ import ( "github.com/pingcap/tidb/pkg/domain/infosync" "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "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/sysproctrack" @@ -435,7 +435,7 @@ func RandomPickOneTableAndTryAutoAnalyze( continue } - tbls, err := is.SchemaTableInfos(context.Background(), pmodel.NewCIStr(db)) + tbls, err := is.SchemaTableInfos(context.Background(), ast.NewCIStr(db)) terror.Log(err) // We shuffle dbs and tbls so that the order of iterating tables is random. If the order is fixed and the auto // analyze job of one table fails for some reason, it may always analyze the same table and fail again and again diff --git a/pkg/statistics/handle/autoanalyze/autoanalyze_test.go b/pkg/statistics/handle/autoanalyze/autoanalyze_test.go index 90fb7dd911982..3ce37d1f0f64e 100644 --- a/pkg/statistics/handle/autoanalyze/autoanalyze_test.go +++ b/pkg/statistics/handle/autoanalyze/autoanalyze_test.go @@ -25,7 +25,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/domain/infosync" - "github.com/pingcap/tidb/pkg/parser/model" + "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" @@ -209,7 +209,7 @@ func TestAutoAnalyzeOnChangeAnalyzeVer(t *testing.T) { // Auto analyze when global ver is 1. h.HandleAutoAnalyze() require.NoError(t, h.Update(context.Background(), is)) - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) statsTbl1 := h.GetTableStats(tbl.Meta()) // Check that all the version of t's stats are 1. @@ -248,7 +248,7 @@ func TestAutoAnalyzeOnChangeAnalyzeVer(t *testing.T) { require.NoError(t, err) require.NoError(t, h.DumpStatsDeltaToKV(true)) is = do.InfoSchema() - tbl2, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("tt")) + tbl2, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("tt")) require.NoError(t, err) require.NoError(t, h.Update(context.Background(), is)) h.HandleAutoAnalyze() @@ -275,7 +275,7 @@ func TestTableAnalyzed(t *testing.T) { testKit.MustExec("insert into t values (1)") is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo := tbl.Meta() h := dom.StatsHandle() @@ -665,7 +665,7 @@ func TestAutoAnalyzeWithVectorIndex(t *testing.T) { tk.MustExec("insert into t values(1, '[1, 2]', '[1, 3, 4]', '[1, 4, 5, 6]')") tk.MustExec("SET GLOBAL tidb_enable_auto_analyze_priority_queue=off") tk.MustExec("analyze table t all columns") - tbl, err := dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo := tbl.Meta() statsTbl := h.GetTableStats(tableInfo) diff --git a/pkg/statistics/handle/autoanalyze/exec/BUILD.bazel b/pkg/statistics/handle/autoanalyze/exec/BUILD.bazel index a38903f35db5a..0951755570787 100644 --- a/pkg/statistics/handle/autoanalyze/exec/BUILD.bazel +++ b/pkg/statistics/handle/autoanalyze/exec/BUILD.bazel @@ -31,7 +31,7 @@ go_test( flaky = True, deps = [ ":exec", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/sessionctx", "//pkg/testkit", "//pkg/util", diff --git a/pkg/statistics/handle/autoanalyze/exec/exec_test.go b/pkg/statistics/handle/autoanalyze/exec/exec_test.go index 55ce730ac8c11..9e43cbf9e4a0c 100644 --- a/pkg/statistics/handle/autoanalyze/exec/exec_test.go +++ b/pkg/statistics/handle/autoanalyze/exec/exec_test.go @@ -20,7 +20,7 @@ import ( "testing" "time" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/statistics/handle/autoanalyze/exec" "github.com/pingcap/tidb/pkg/testkit" @@ -49,7 +49,7 @@ func TestExecAutoAnalyzes(t *testing.T) { // Check the result of analyze. is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tblStats := handle.GetTableStats(tbl.Meta()) require.Equal(t, int64(3), tblStats.RealtimeCount) diff --git a/pkg/statistics/handle/autoanalyze/priorityqueue/BUILD.bazel b/pkg/statistics/handle/autoanalyze/priorityqueue/BUILD.bazel index 993bc7491fabc..6661096287a1c 100644 --- a/pkg/statistics/handle/autoanalyze/priorityqueue/BUILD.bazel +++ b/pkg/statistics/handle/autoanalyze/priorityqueue/BUILD.bazel @@ -63,7 +63,7 @@ go_test( "//pkg/domain", "//pkg/domain/infosync", "//pkg/meta/model", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/session", "//pkg/sessionctx", "//pkg/sessionctx/sysproctrack", diff --git a/pkg/statistics/handle/autoanalyze/priorityqueue/analysis_job_factory_test.go b/pkg/statistics/handle/autoanalyze/priorityqueue/analysis_job_factory_test.go index 99d442a26d9e5..6df538ad2a923 100644 --- a/pkg/statistics/handle/autoanalyze/priorityqueue/analysis_job_factory_test.go +++ b/pkg/statistics/handle/autoanalyze/priorityqueue/analysis_job_factory_test.go @@ -20,7 +20,7 @@ import ( "time" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/statistics" "github.com/pingcap/tidb/pkg/statistics/handle/autoanalyze/priorityqueue" "github.com/stretchr/testify/require" @@ -134,7 +134,7 @@ func TestCheckIndexesNeedAnalyze(t *testing.T) { Indices: []*model.IndexInfo{ { ID: 1, - Name: pmodel.NewCIStr("index1"), + Name: ast.NewCIStr("index1"), State: model.StatePublic, }, }, @@ -148,12 +148,12 @@ func TestCheckIndexesNeedAnalyze(t *testing.T) { Indices: []*model.IndexInfo{ { ID: 1, - Name: pmodel.NewCIStr("index1"), + Name: ast.NewCIStr("index1"), State: model.StatePublic, }, { ID: 2, - Name: pmodel.NewCIStr("vec_index1"), + Name: ast.NewCIStr("vec_index1"), State: model.StatePublic, VectorInfo: &model.VectorIndexInfo{}, }, @@ -229,11 +229,11 @@ func TestCalculateIndicatorsForPartitions(t *testing.T) { defs: []model.PartitionDefinition{ { ID: 1, - Name: pmodel.NewCIStr("p0"), + Name: ast.NewCIStr("p0"), }, { ID: 2, - Name: pmodel.NewCIStr("p1"), + Name: ast.NewCIStr("p1"), }, }, autoAnalyzeRatio: 0.5, @@ -291,11 +291,11 @@ func TestCalculateIndicatorsForPartitions(t *testing.T) { defs: []model.PartitionDefinition{ { ID: 1, - Name: pmodel.NewCIStr("p0"), + Name: ast.NewCIStr("p0"), }, { ID: 2, - Name: pmodel.NewCIStr("p1"), + Name: ast.NewCIStr("p1"), }, }, autoAnalyzeRatio: 0.5, @@ -353,11 +353,11 @@ func TestCalculateIndicatorsForPartitions(t *testing.T) { defs: []model.PartitionDefinition{ { ID: 1, - Name: pmodel.NewCIStr("p0"), + Name: ast.NewCIStr("p0"), }, { ID: 2, - Name: pmodel.NewCIStr("p1"), + Name: ast.NewCIStr("p1"), }, }, autoAnalyzeRatio: 0.5, @@ -392,17 +392,17 @@ func TestCheckNewlyAddedIndexesNeedAnalyzeForPartitionedTable(t *testing.T) { Indices: []*model.IndexInfo{ { ID: 1, - Name: pmodel.NewCIStr("index1"), + Name: ast.NewCIStr("index1"), State: model.StatePublic, }, { ID: 2, - Name: pmodel.NewCIStr("index2"), + Name: ast.NewCIStr("index2"), State: model.StatePublic, }, { ID: 3, - Name: pmodel.NewCIStr("index3"), + Name: ast.NewCIStr("index3"), State: model.StatePublic, VectorInfo: &model.VectorIndexInfo{}, }, diff --git a/pkg/statistics/handle/autoanalyze/priorityqueue/dynamic_partitioned_table_analysis_job_test.go b/pkg/statistics/handle/autoanalyze/priorityqueue/dynamic_partitioned_table_analysis_job_test.go index d4851de502b68..95f519184ce36 100644 --- a/pkg/statistics/handle/autoanalyze/priorityqueue/dynamic_partitioned_table_analysis_job_test.go +++ b/pkg/statistics/handle/autoanalyze/priorityqueue/dynamic_partitioned_table_analysis_job_test.go @@ -19,7 +19,7 @@ import ( "strings" "testing" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/session" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/statistics/handle/autoanalyze/priorityqueue" @@ -46,7 +46,7 @@ func TestAnalyzeDynamicPartitionedTable(t *testing.T) { handle := dom.StatsHandle() // Check the result of analyze. is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) pid := tbl.Meta().GetPartitionInfo().Definitions[0].ID tblStats := handle.GetPartitionStats(tbl.Meta(), pid) @@ -55,7 +55,7 @@ func TestAnalyzeDynamicPartitionedTable(t *testing.T) { job.Analyze(handle, dom.SysProcTracker()) // Check the result of analyze. is = dom.InfoSchema() - tbl, err = is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err = is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) pid = tbl.Meta().GetPartitionInfo().Definitions[0].ID tblStats = handle.GetPartitionStats(tbl.Meta(), pid) @@ -70,7 +70,7 @@ func TestAnalyzeDynamicPartitionedTableIndexes(t *testing.T) { tk.MustExec("create table t (a int, b int, index idx(a), index idx1(b)) partition by range (a) (partition p0 values less than (2), partition p1 values less than (4))") tk.MustExec("insert into t values (1, 1), (2, 2), (3, 3)") - tableInfo, err := dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tableInfo, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) partitionInfo := tableInfo.Meta().GetPartitionInfo() require.NotNil(t, partitionInfo) @@ -88,7 +88,7 @@ func TestAnalyzeDynamicPartitionedTableIndexes(t *testing.T) { handle := dom.StatsHandle() // Check the result of analyze index. is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) pid := tbl.Meta().GetPartitionInfo().Definitions[0].ID tblStats := handle.GetPartitionStats(tbl.Meta(), pid) @@ -101,7 +101,7 @@ func TestAnalyzeDynamicPartitionedTableIndexes(t *testing.T) { job.Analyze(handle, dom.SysProcTracker()) // Check the result of analyze index. is = dom.InfoSchema() - tbl, err = is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err = is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) pid = tbl.Meta().GetPartitionInfo().Definitions[0].ID tblStats = handle.GetPartitionStats(tbl.Meta(), pid) @@ -131,7 +131,7 @@ func TestValidateAndPrepareForDynamicPartitionedTable(t *testing.T) { tk.MustExec("create database example_schema") tk.MustExec("use example_schema") tk.MustExec("create table example_table (a int, b int, index idx(a)) partition by range (a) (partition p0 values less than (2), partition p1 values less than (4))") - tableInfo, err := dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("example_schema"), model.NewCIStr("example_table")) + tableInfo, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("example_schema"), ast.NewCIStr("example_table")) require.NoError(t, err) job := &priorityqueue.DynamicPartitionedTableAnalysisJob{ SchemaName: "example_schema", @@ -185,7 +185,7 @@ func TestPerformanceOfValidateAndPrepare(t *testing.T) { tk.MustExec("create database example_schema") tk.MustExec("use example_schema") tk.MustExec("create table example_table (a int, b int, index idx(a)) partition by range (a) (partition p0 values less than (2), partition p1 values less than (4))") - tableInfo, err := dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("example_schema"), model.NewCIStr("example_table")) + tableInfo, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("example_schema"), ast.NewCIStr("example_table")) require.NoError(t, err) job := &priorityqueue.DynamicPartitionedTableAnalysisJob{ SchemaName: "example_schema", diff --git a/pkg/statistics/handle/autoanalyze/priorityqueue/non_partitioned_table_analysis_job_test.go b/pkg/statistics/handle/autoanalyze/priorityqueue/non_partitioned_table_analysis_job_test.go index 9e446283ffe0a..4dc21364546df 100644 --- a/pkg/statistics/handle/autoanalyze/priorityqueue/non_partitioned_table_analysis_job_test.go +++ b/pkg/statistics/handle/autoanalyze/priorityqueue/non_partitioned_table_analysis_job_test.go @@ -18,7 +18,7 @@ import ( "context" "testing" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/session" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/statistics/handle/autoanalyze/priorityqueue" @@ -74,7 +74,7 @@ func TestAnalyzeNonPartitionedTable(t *testing.T) { // Before analyze table. handle := dom.StatsHandle() is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tblStats := handle.GetTableStats(tbl.Meta()) require.True(t, tblStats.Pseudo) @@ -82,7 +82,7 @@ func TestAnalyzeNonPartitionedTable(t *testing.T) { job.Analyze(handle, dom.SysProcTracker()) // Check the result of analyze. is = dom.InfoSchema() - tbl, err = is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err = is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tblStats = handle.GetTableStats(tbl.Meta()) require.Equal(t, int64(3), tblStats.RealtimeCount) @@ -95,7 +95,7 @@ func TestAnalyzeNonPartitionedIndexes(t *testing.T) { tk.MustExec("create table t (a int, b int, index idx(a), index idx1(b))") tk.MustExec("insert into t values (1, 1), (2, 2), (3, 3)") - tblInfo, err := dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tblInfo, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) job := &priorityqueue.NonPartitionedTableAnalysisJob{ TableID: tblInfo.Meta().ID, @@ -105,7 +105,7 @@ func TestAnalyzeNonPartitionedIndexes(t *testing.T) { handle := dom.StatsHandle() // Before analyze indexes. is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tblStats := handle.GetTableStats(tbl.Meta()) require.False(t, tblStats.GetIdx(1).IsAnalyzed()) @@ -116,7 +116,7 @@ func TestAnalyzeNonPartitionedIndexes(t *testing.T) { job.Analyze(handle, dom.SysProcTracker()) // Check the result of analyze. is = dom.InfoSchema() - tbl, err = is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err = is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tblStats = handle.GetTableStats(tbl.Meta()) require.NotNil(t, tblStats.GetIdx(1)) @@ -136,7 +136,7 @@ func TestNonPartitionedTableValidateAndPrepare(t *testing.T) { tk.MustExec("create schema example_schema") tk.MustExec("use example_schema") tk.MustExec("create table example_table1 (a int, b int, index idx(a))") - tableInfo, err := dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("example_schema"), model.NewCIStr("example_table1")) + tableInfo, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("example_schema"), ast.NewCIStr("example_table1")) require.NoError(t, err) job := &priorityqueue.NonPartitionedTableAnalysisJob{ TableID: tableInfo.Meta().ID, @@ -180,7 +180,7 @@ func TestValidateAndPrepareWhenOnlyHasFailedAnalysisRecords(t *testing.T) { tk.MustExec("create schema example_schema") tk.MustExec("use example_schema") tk.MustExec("create table example_table1 (a int, b int, index idx(a))") - tableInfo, err := dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("example_schema"), model.NewCIStr("example_table1")) + tableInfo, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("example_schema"), ast.NewCIStr("example_table1")) require.NoError(t, err) job := &priorityqueue.NonPartitionedTableAnalysisJob{ TableID: tableInfo.Meta().ID, diff --git a/pkg/statistics/handle/autoanalyze/priorityqueue/queue_ddl_handler_test.go b/pkg/statistics/handle/autoanalyze/priorityqueue/queue_ddl_handler_test.go index 1bfb65e398fd3..66f06e9ffba3b 100644 --- a/pkg/statistics/handle/autoanalyze/priorityqueue/queue_ddl_handler_test.go +++ b/pkg/statistics/handle/autoanalyze/priorityqueue/queue_ddl_handler_test.go @@ -24,7 +24,7 @@ import ( "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/domain/infosync" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/statistics" "github.com/pingcap/tidb/pkg/statistics/handle/autoanalyze/priorityqueue" @@ -77,8 +77,8 @@ func TestHandleDDLEventsWithRunningJobs(t *testing.T) { ctx := context.Background() require.NoError(t, handle.DumpStatsDeltaToKV(true)) require.NoError(t, handle.Update(ctx, dom.InfoSchema())) - schema := pmodel.NewCIStr("test") - tbl1, err := dom.InfoSchema().TableByName(ctx, schema, pmodel.NewCIStr("t1")) + schema := ast.NewCIStr("test") + tbl1, err := dom.InfoSchema().TableByName(ctx, schema, ast.NewCIStr("t1")) require.NoError(t, err) tk.MustExec("analyze table t1") require.NoError(t, handle.Update(ctx, dom.InfoSchema())) @@ -171,7 +171,7 @@ func TestTruncateTable(t *testing.T) { testKit.MustExec("use test") testKit.MustExec("create table t (c1 int, c2 int, index idx(c1, c2))") is := do.InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo := tbl.Meta() h := do.StatsHandle() @@ -285,7 +285,7 @@ func TestDropTable(t *testing.T) { testKit.MustExec("use test") testKit.MustExec("create table t (c1 int, c2 int, index idx(c1, c2))") is := do.InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo := tbl.Meta() h := do.StatsHandle() @@ -399,7 +399,7 @@ func TestTruncateTablePartition(t *testing.T) { testKit.MustExec("use test") testKit.MustExec("create table t (c1 int, c2 int, index idx(c1, c2)) partition by range (c1) (partition p0 values less than (10), partition p1 values less than (20))") is := do.InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo := tbl.Meta() h := do.StatsHandle() @@ -458,7 +458,7 @@ func TestDropTablePartition(t *testing.T) { testKit.MustExec("use test") testKit.MustExec("create table t (c1 int, c2 int, index idx(c1, c2)) partition by range (c1) (partition p0 values less than (10), partition p1 values less than (20))") is := do.InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo := tbl.Meta() h := do.StatsHandle() @@ -518,10 +518,10 @@ func TestExchangeTablePartition(t *testing.T) { testKit.MustExec("create table t1 (c1 int, c2 int, index idx(c1, c2)) partition by range (c1) (partition p0 values less than (10), partition p1 values less than (20))") testKit.MustExec("create table t2 (c1 int, c2 int, index idx(c1, c2))") is := do.InfoSchema() - tbl1, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t1")) + tbl1, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t1")) require.NoError(t, err) tableInfo1 := tbl1.Meta() - tbl2, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t2")) + tbl2, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t2")) require.NoError(t, err) tableInfo2 := tbl2.Meta() h := do.StatsHandle() @@ -585,7 +585,7 @@ func TestReorganizeTablePartition(t *testing.T) { testKit.MustExec("use test") testKit.MustExec("create table t (c1 int, c2 int, index idx(c1, c2)) partition by range (c1) (partition p0 values less than (10), partition p1 values less than (20))") is := do.InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo := tbl.Meta() h := do.StatsHandle() @@ -644,7 +644,7 @@ func TestAlterTablePartitioning(t *testing.T) { testKit.MustExec("use test") testKit.MustExec("create table t (c1 int, c2 int, index idx(c1, c2))") is := do.InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo := tbl.Meta() h := do.StatsHandle() @@ -703,7 +703,7 @@ func TestRemovePartitioning(t *testing.T) { testKit.MustExec("use test") testKit.MustExec("create table t (c1 int, c2 int, index idx(c1, c2)) partition by range columns (c1) (partition p0 values less than (5), partition p1 values less than (10))") is := do.InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo := tbl.Meta() h := do.StatsHandle() @@ -762,7 +762,7 @@ func TestDropSchemaEventWithDynamicPartition(t *testing.T) { testKit.MustExec("use test") testKit.MustExec("create table t (c1 int, c2 int, index idx(c1, c2)) partition by range columns (c1) (partition p0 values less than (5), partition p1 values less than (10))") is := do.InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo := tbl.Meta() h := do.StatsHandle() @@ -929,7 +929,7 @@ func TestAddIndexTriggerAutoAnalyzeWithStatsVersion1(t *testing.T) { testKit.MustExec("use test") testKit.MustExec("create table t (c1 int, c2 int, index idx(c1, c2)) partition by range columns (c1) (partition p0 values less than (5), partition p1 values less than (10))") is := do.InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo := tbl.Meta() h := do.StatsHandle() diff --git a/pkg/statistics/handle/autoanalyze/priorityqueue/queue_test.go b/pkg/statistics/handle/autoanalyze/priorityqueue/queue_test.go index 89b4fd27eec68..4ae8f7e8973d5 100644 --- a/pkg/statistics/handle/autoanalyze/priorityqueue/queue_test.go +++ b/pkg/statistics/handle/autoanalyze/priorityqueue/queue_test.go @@ -20,7 +20,7 @@ import ( "time" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/statistics" "github.com/pingcap/tidb/pkg/statistics/handle/autoanalyze/priorityqueue" @@ -210,10 +210,10 @@ func testProcessDMLChanges(t *testing.T, partitioned bool) { require.NoError(t, handle.DumpStatsDeltaToKV(true)) require.NoError(t, handle.Update(ctx, dom.InfoSchema())) - schema := pmodel.NewCIStr("test") - tbl1, err := dom.InfoSchema().TableByName(ctx, schema, pmodel.NewCIStr("t1")) + schema := ast.NewCIStr("test") + tbl1, err := dom.InfoSchema().TableByName(ctx, schema, ast.NewCIStr("t1")) require.NoError(t, err) - tbl2, err := dom.InfoSchema().TableByName(ctx, schema, pmodel.NewCIStr("t2")) + tbl2, err := dom.InfoSchema().TableByName(ctx, schema, ast.NewCIStr("t2")) require.NoError(t, err) pq := priorityqueue.NewAnalysisPriorityQueue(handle) @@ -299,10 +299,10 @@ func TestProcessDMLChangesWithRunningJobs(t *testing.T) { ctx := context.Background() require.NoError(t, handle.DumpStatsDeltaToKV(true)) require.NoError(t, handle.Update(ctx, dom.InfoSchema())) - schema := pmodel.NewCIStr("test") - tbl1, err := dom.InfoSchema().TableByName(ctx, schema, pmodel.NewCIStr("t1")) + schema := ast.NewCIStr("test") + tbl1, err := dom.InfoSchema().TableByName(ctx, schema, ast.NewCIStr("t1")) require.NoError(t, err) - tbl2, err := dom.InfoSchema().TableByName(ctx, schema, pmodel.NewCIStr("t2")) + tbl2, err := dom.InfoSchema().TableByName(ctx, schema, ast.NewCIStr("t2")) require.NoError(t, err) tk.MustExec("analyze table t1") tk.MustExec("analyze table t2") @@ -444,10 +444,10 @@ func TestProcessDMLChangesWithLockedTables(t *testing.T) { defer pq.Close() require.NoError(t, pq.Initialize()) - schema := pmodel.NewCIStr("test") - tbl1, err := dom.InfoSchema().TableByName(ctx, schema, pmodel.NewCIStr("t1")) + schema := ast.NewCIStr("test") + tbl1, err := dom.InfoSchema().TableByName(ctx, schema, ast.NewCIStr("t1")) require.NoError(t, err) - tbl2, err := dom.InfoSchema().TableByName(ctx, schema, pmodel.NewCIStr("t2")) + tbl2, err := dom.InfoSchema().TableByName(ctx, schema, ast.NewCIStr("t2")) require.NoError(t, err) // Check current jobs. @@ -507,8 +507,8 @@ func TestProcessDMLChangesWithLockedPartitionsAndDynamicPruneMode(t *testing.T) defer pq.Close() require.NoError(t, pq.Initialize()) - schema := pmodel.NewCIStr("test") - tbl, err := dom.InfoSchema().TableByName(ctx, schema, pmodel.NewCIStr("t1")) + schema := ast.NewCIStr("test") + tbl, err := dom.InfoSchema().TableByName(ctx, schema, ast.NewCIStr("t1")) require.NoError(t, err) // Check current jobs. @@ -561,8 +561,8 @@ func TestProcessDMLChangesWithLockedPartitionsAndStaticPruneMode(t *testing.T) { require.NoError(t, handle.Update(ctx, dom.InfoSchema())) tk.MustExec("analyze table t1") require.NoError(t, handle.Update(ctx, dom.InfoSchema())) - schema := pmodel.NewCIStr("test") - tbl, err := dom.InfoSchema().TableByName(ctx, schema, pmodel.NewCIStr("t1")) + schema := ast.NewCIStr("test") + tbl, err := dom.InfoSchema().TableByName(ctx, schema, ast.NewCIStr("t1")) require.NoError(t, err) // Insert more rows into partition p0. @@ -652,7 +652,7 @@ func TestPQHandlesTableDeletionGracefully(t *testing.T) { require.NoError(t, err) require.NotEqual(t, 0, l) - tbl, err := dom.InfoSchema().TableByName(ctx, pmodel.NewCIStr("test"), pmodel.NewCIStr("t1")) + tbl, err := dom.InfoSchema().TableByName(ctx, ast.NewCIStr("test"), ast.NewCIStr("t1")) require.NoError(t, err) // Drop the table and mock the table stats is removed from the cache. diff --git a/pkg/statistics/handle/autoanalyze/priorityqueue/static_partitioned_table_analysis_job_test.go b/pkg/statistics/handle/autoanalyze/priorityqueue/static_partitioned_table_analysis_job_test.go index 0d72b72d86910..7c1738614059b 100644 --- a/pkg/statistics/handle/autoanalyze/priorityqueue/static_partitioned_table_analysis_job_test.go +++ b/pkg/statistics/handle/autoanalyze/priorityqueue/static_partitioned_table_analysis_job_test.go @@ -18,7 +18,7 @@ import ( "context" "testing" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/session" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/statistics/handle/autoanalyze/priorityqueue" @@ -68,7 +68,7 @@ func TestAnalyzeStaticPartitionedTable(t *testing.T) { tk.MustExec("create table t (a int, b int, index idx(a)) partition by range (a) (partition p0 values less than (2), partition p1 values less than (4))") tk.MustExec("insert into t values (1, 1), (2, 2), (3, 3)") - tableInfo, err := dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tableInfo, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) partitionInfo := tableInfo.Meta().GetPartitionInfo() require.NotNil(t, partitionInfo) @@ -82,7 +82,7 @@ func TestAnalyzeStaticPartitionedTable(t *testing.T) { // Before analyze the partition. handle := dom.StatsHandle() is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) pid := tbl.Meta().GetPartitionInfo().Definitions[0].ID tblStats := handle.GetPartitionStats(tbl.Meta(), pid) @@ -94,7 +94,7 @@ func TestAnalyzeStaticPartitionedTable(t *testing.T) { job.Analyze(handle, dom.SysProcTracker()) // Check the result of analyze. is = dom.InfoSchema() - tbl, err = is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err = is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) pid = tbl.Meta().GetPartitionInfo().Definitions[0].ID tblStats = handle.GetPartitionStats(tbl.Meta(), pid) @@ -108,7 +108,7 @@ func TestAnalyzeStaticPartitionedTableIndexes(t *testing.T) { tk.MustExec("use test") tk.MustExec("create table t (a int, b int, index idx(a), index idx1(b)) partition by range (a) (partition p0 values less than (2), partition p1 values less than (4))") tk.MustExec("insert into t values (1, 1), (2, 2), (3, 3)") - tableInfo, err := dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tableInfo, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) partitionInfo := tableInfo.Meta().GetPartitionInfo() require.NotNil(t, partitionInfo) @@ -121,7 +121,7 @@ func TestAnalyzeStaticPartitionedTableIndexes(t *testing.T) { handle := dom.StatsHandle() // Before analyze indexes. is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) pid := tbl.Meta().GetPartitionInfo().Definitions[0].ID tblStats := handle.GetPartitionStats(tbl.Meta(), pid) @@ -134,7 +134,7 @@ func TestAnalyzeStaticPartitionedTableIndexes(t *testing.T) { job.Analyze(handle, dom.SysProcTracker()) // Check the result of analyze. is = dom.InfoSchema() - tbl, err = is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err = is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) pid = tbl.Meta().GetPartitionInfo().Definitions[0].ID tblStats = handle.GetPartitionStats(tbl.Meta(), pid) @@ -155,7 +155,7 @@ func TestStaticPartitionedTableValidateAndPrepare(t *testing.T) { tk.MustExec("create schema example_schema") tk.MustExec("use example_schema") tk.MustExec("create table example_table (a int, b int, index idx(a)) partition by range (a) (partition p0 values less than (2), partition p1 values less than (4))") - tableInfo, err := dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("example_schema"), model.NewCIStr("example_table")) + tableInfo, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("example_schema"), ast.NewCIStr("example_table")) require.NoError(t, err) partitionInfo := tableInfo.Meta().GetPartitionInfo() require.NotNil(t, partitionInfo) diff --git a/pkg/statistics/handle/autoanalyze/refresher/BUILD.bazel b/pkg/statistics/handle/autoanalyze/refresher/BUILD.bazel index c5d6ff6eb9af0..1bf2f4d3732b3 100644 --- a/pkg/statistics/handle/autoanalyze/refresher/BUILD.bazel +++ b/pkg/statistics/handle/autoanalyze/refresher/BUILD.bazel @@ -36,7 +36,7 @@ go_test( shard_count = 10, deps = [ ":refresher", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/sessionctx", "//pkg/sessionctx/sysproctrack", "//pkg/statistics", diff --git a/pkg/statistics/handle/autoanalyze/refresher/refresher_test.go b/pkg/statistics/handle/autoanalyze/refresher/refresher_test.go index dcbc201fd669e..436d530ab3b80 100644 --- a/pkg/statistics/handle/autoanalyze/refresher/refresher_test.go +++ b/pkg/statistics/handle/autoanalyze/refresher/refresher_test.go @@ -18,7 +18,7 @@ import ( "context" "testing" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/statistics" "github.com/pingcap/tidb/pkg/statistics/handle/autoanalyze/refresher" @@ -207,12 +207,12 @@ func TestIgnoreTinyTable(t *testing.T) { require.NoError(t, handle.DumpStatsDeltaToKV(true)) require.NoError(t, handle.Update(context.Background(), dom.InfoSchema())) // Make sure table stats are not pseudo. - tbl1, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t1")) + tbl1, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t1")) require.NoError(t, err) pid1 := tbl1.Meta().GetPartitionInfo().Definitions[1].ID tblStats1 := handle.GetPartitionStats(tbl1.Meta(), pid1) require.False(t, tblStats1.Pseudo) - tbl2, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t2")) + tbl2, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t2")) require.NoError(t, err) pid2 := tbl2.Meta().GetPartitionInfo().Definitions[1].ID tblStats2 := handle.GetPartitionStats(tbl2.Meta(), pid2) @@ -273,14 +273,14 @@ func TestAnalyzeHighestPriorityTables(t *testing.T) { require.NoError(t, handle.DumpStatsDeltaToKV(true)) require.NoError(t, handle.Update(context.Background(), dom.InfoSchema())) // The table is analyzed. - tbl1, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t1")) + tbl1, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t1")) require.NoError(t, err) pid1 := tbl1.Meta().GetPartitionInfo().Definitions[1].ID tblStats1 := handle.GetPartitionStats(tbl1.Meta(), pid1) require.Equal(t, int64(0), tblStats1.ModifyCount) require.Equal(t, int64(12), tblStats1.RealtimeCount) // t2 is not analyzed. - tbl2, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t2")) + tbl2, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t2")) require.NoError(t, err) pid2 := tbl2.Meta().GetPartitionInfo().Definitions[1].ID tblStats2 := handle.GetPartitionStats(tbl2.Meta(), pid2) @@ -342,14 +342,14 @@ func TestAnalyzeHighestPriorityTablesConcurrently(t *testing.T) { require.NoError(t, handle.DumpStatsDeltaToKV(true)) require.NoError(t, handle.Update(context.Background(), dom.InfoSchema())) // Check if t1 and t2 are analyzed (they should be, as they have more new data). - tbl1, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t1")) + tbl1, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t1")) require.NoError(t, err) pid1 := tbl1.Meta().GetPartitionInfo().Definitions[1].ID tblStats1 := handle.GetPartitionStats(tbl1.Meta(), pid1) require.Equal(t, int64(0), tblStats1.ModifyCount) require.Equal(t, int64(12), tblStats1.RealtimeCount) - tbl2, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t2")) + tbl2, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t2")) require.NoError(t, err) pid2 := tbl2.Meta().GetPartitionInfo().Definitions[1].ID tblStats2 := handle.GetPartitionStats(tbl2.Meta(), pid2) @@ -357,7 +357,7 @@ func TestAnalyzeHighestPriorityTablesConcurrently(t *testing.T) { require.Equal(t, int64(8), tblStats2.RealtimeCount) // t3 should not be analyzed yet, as it has the least new data. - tbl3, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t3")) + tbl3, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t3")) require.NoError(t, err) pid3 := tbl3.Meta().GetPartitionInfo().Definitions[1].ID tblStats3 := handle.GetPartitionStats(tbl3.Meta(), pid3) @@ -463,7 +463,7 @@ func TestAnalyzeHighestPriorityTablesWithFailedAnalysis(t *testing.T) { is := dom.InfoSchema() // t1 is not analyzed. - tbl1, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t1")) + tbl1, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t1")) require.NoError(t, err) pid1 := tbl1.Meta().GetPartitionInfo().Definitions[0].ID tblStats1 := handle.GetPartitionStats(tbl1.Meta(), pid1) @@ -471,7 +471,7 @@ func TestAnalyzeHighestPriorityTablesWithFailedAnalysis(t *testing.T) { require.Equal(t, int64(1), tblStats1.ModifyCount) // t2 is analyzed. - tbl2, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t2")) + tbl2, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t2")) require.NoError(t, err) pid2 := tbl2.Meta().GetPartitionInfo().Definitions[0].ID tblStats2 := handle.GetPartitionStats(tbl2.Meta(), pid2) diff --git a/pkg/statistics/handle/ddl/BUILD.bazel b/pkg/statistics/handle/ddl/BUILD.bazel index fdef8d1b713fd..5a48dfcbfecb4 100644 --- a/pkg/statistics/handle/ddl/BUILD.bazel +++ b/pkg/statistics/handle/ddl/BUILD.bazel @@ -36,7 +36,7 @@ go_test( ":ddl", "//pkg/ddl/notifier", "//pkg/meta/model", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/planner/cardinality", "//pkg/statistics/handle/ddl/testutil", "//pkg/statistics/handle/storage", diff --git a/pkg/statistics/handle/ddl/ddl_test.go b/pkg/statistics/handle/ddl/ddl_test.go index 1ecd62335ddf1..c4e80c76f1db2 100644 --- a/pkg/statistics/handle/ddl/ddl_test.go +++ b/pkg/statistics/handle/ddl/ddl_test.go @@ -21,7 +21,7 @@ import ( "github.com/pingcap/tidb/pkg/ddl/notifier" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/planner/cardinality" "github.com/pingcap/tidb/pkg/statistics/handle/ddl" statstestutil "github.com/pingcap/tidb/pkg/statistics/handle/ddl/testutil" @@ -41,7 +41,7 @@ func TestDDLAfterLoad(t *testing.T) { testKit.MustExec("create table t (c1 int, c2 int, index idx(c1, c2))") testKit.MustExec("analyze table t") is := do.InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo := tbl.Meta() statsTbl := do.StatsHandle().GetTableStats(tableInfo) @@ -56,7 +56,7 @@ func TestDDLAfterLoad(t *testing.T) { // add column testKit.MustExec("alter table t add column c10 int") is = do.InfoSchema() - tbl, err = is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err = is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo = tbl.Meta() @@ -73,7 +73,7 @@ func TestDDLTable(t *testing.T) { testKit.MustExec("use test") testKit.MustExec("create table t (c1 int, c2 int)") is := do.InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo := tbl.Meta() h := do.StatsHandle() @@ -85,7 +85,7 @@ func TestDDLTable(t *testing.T) { testKit.MustExec("create table t1 (c1 int, c2 int, index idx(c1))") is = do.InfoSchema() - tbl, err = is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t1")) + tbl, err = is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t1")) require.NoError(t, err) tableInfo = tbl.Meta() err = statstestutil.HandleNextDDLEventWithTxn(h) @@ -98,7 +98,7 @@ func TestDDLTable(t *testing.T) { // https://github.com/pingcap/tidb/issues/53652 testKit.MustExec("create table t_parent (id int primary key)") is = do.InfoSchema() - tbl, err = is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t_parent")) + tbl, err = is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t_parent")) require.NoError(t, err) tableInfo = tbl.Meta() err = statstestutil.HandleNextDDLEventWithTxn(h) @@ -109,7 +109,7 @@ func TestDDLTable(t *testing.T) { testKit.MustExec("create table t_child (id int primary key, pid int, foreign key (pid) references t_parent(id) on delete cascade on update cascade);") is = do.InfoSchema() - tbl, err = is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t_child")) + tbl, err = is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t_child")) require.NoError(t, err) tableInfo = tbl.Meta() err = statstestutil.HandleNextDDLEventWithTxn(h) @@ -162,7 +162,7 @@ func TestTruncateTable(t *testing.T) { testKit.MustExec("use test") testKit.MustExec("create table t (c1 int, c2 int, index idx(c1, c2))") is := do.InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo := tbl.Meta() h := do.StatsHandle() @@ -192,7 +192,7 @@ func TestTruncateTable(t *testing.T) { // Get new table info. is = do.InfoSchema() - tbl, err = is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err = is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) newTableInfo := tbl.Meta() // Get new added table's stats meta. @@ -232,7 +232,7 @@ func TestTruncateAPartitionedTable(t *testing.T) { testKit.MustExec("analyze table t") is := do.InfoSchema() tbl, err := is.TableByName(context.Background(), - pmodel.NewCIStr("test"), pmodel.NewCIStr("t"), + ast.NewCIStr("test"), ast.NewCIStr("t"), ) require.NoError(t, err) tableInfo := tbl.Meta() @@ -265,7 +265,7 @@ func TestTruncateAPartitionedTable(t *testing.T) { // Get new table info. is = do.InfoSchema() - tbl, err = is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err = is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) newTableInfo := tbl.Meta() // Get all new added partitions ID. @@ -305,7 +305,7 @@ func TestDDLHistogram(t *testing.T) { require.NoError(t, err) is := do.InfoSchema() require.Nil(t, h.Update(context.Background(), is)) - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo := tbl.Meta() statsTbl := do.StatsHandle().GetTableStats(tableInfo) @@ -320,7 +320,7 @@ func TestDDLHistogram(t *testing.T) { require.NoError(t, err) is = do.InfoSchema() require.Nil(t, h.Update(context.Background(), is)) - tbl, err = is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err = is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo = tbl.Meta() statsTbl = do.StatsHandle().GetTableStats(tableInfo) @@ -340,7 +340,7 @@ func TestDDLHistogram(t *testing.T) { require.NoError(t, err) is = do.InfoSchema() require.Nil(t, h.Update(context.Background(), is)) - tbl, err = is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err = is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo = tbl.Meta() statsTbl = do.StatsHandle().GetTableStats(tableInfo) @@ -353,7 +353,7 @@ func TestDDLHistogram(t *testing.T) { require.NoError(t, err) is = do.InfoSchema() require.Nil(t, h.Update(context.Background(), is)) - tbl, err = is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err = is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo = tbl.Meta() statsTbl = do.StatsHandle().GetTableStats(tableInfo) @@ -367,20 +367,20 @@ func TestDDLHistogram(t *testing.T) { require.NoError(t, err) is = do.InfoSchema() require.Nil(t, h.Update(context.Background(), is)) - tbl, err = is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err = is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo = tbl.Meta() statsTbl = do.StatsHandle().GetTableStats(tableInfo) require.False(t, statsTbl.Pseudo) testKit.MustExec("create index i on t(c2, c1)") - tbl, err = is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err = is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo = tbl.Meta() statsTbl = do.StatsHandle().GetTableStats(tableInfo) require.False(t, statsTbl.ColAndIdxExistenceMap.HasAnalyzed(2, true)) testKit.MustExec("analyze table t") - tbl, err = is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err = is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo = tbl.Meta() statsTbl = do.StatsHandle().GetTableStats(tableInfo) @@ -415,7 +415,7 @@ PARTITION BY RANGE ( a ) ( )` testKit.MustExec(createTable) is := do.InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo := tbl.Meta() err = statstestutil.HandleNextDDLEventWithTxn(h) @@ -434,7 +434,7 @@ PARTITION BY RANGE ( a ) ( require.NoError(t, err) is = do.InfoSchema() require.Nil(t, h.Update(context.Background(), is)) - tbl, err = is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err = is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo = tbl.Meta() pi = tableInfo.GetPartitionInfo() @@ -447,7 +447,7 @@ PARTITION BY RANGE ( a ) ( addPartition := "alter table t add partition (partition p4 values less than (26))" testKit.MustExec(addPartition) is = do.InfoSchema() - tbl, err = is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err = is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo = tbl.Meta() err = statstestutil.HandleNextDDLEventWithTxn(h) @@ -485,7 +485,7 @@ func TestReorgPartitions(t *testing.T) { testKit.MustExec("analyze table t") is := do.InfoSchema() tbl, err := is.TableByName(context.Background(), - pmodel.NewCIStr("test"), pmodel.NewCIStr("t"), + ast.NewCIStr("test"), ast.NewCIStr("t"), ) require.NoError(t, err) tableInfo := tbl.Meta() @@ -542,7 +542,7 @@ func TestIncreasePartitionCountOfHashPartitionTable(t *testing.T) { testKit.MustExec("analyze table t") is := do.InfoSchema() tbl, err := is.TableByName(context.Background(), - pmodel.NewCIStr("test"), pmodel.NewCIStr("t"), + ast.NewCIStr("test"), ast.NewCIStr("t"), ) require.NoError(t, err) tableInfo := tbl.Meta() @@ -576,7 +576,7 @@ func TestIncreasePartitionCountOfHashPartitionTable(t *testing.T) { // Check new partitions are added. is = do.InfoSchema() tbl, err = is.TableByName(context.Background(), - pmodel.NewCIStr("test"), pmodel.NewCIStr("t"), + ast.NewCIStr("test"), ast.NewCIStr("t"), ) require.NoError(t, err) tableInfo = tbl.Meta() @@ -610,7 +610,7 @@ func TestDecreasePartitionCountOfHashPartitionTable(t *testing.T) { testKit.MustExec("analyze table t") is := do.InfoSchema() tbl, err := is.TableByName(context.Background(), - pmodel.NewCIStr("test"), pmodel.NewCIStr("t"), + ast.NewCIStr("test"), ast.NewCIStr("t"), ) require.NoError(t, err) tableInfo := tbl.Meta() @@ -650,7 +650,7 @@ func TestDecreasePartitionCountOfHashPartitionTable(t *testing.T) { // Check new partitions are added. is = do.InfoSchema() tbl, err = is.TableByName(context.Background(), - pmodel.NewCIStr("test"), pmodel.NewCIStr("t"), + ast.NewCIStr("test"), ast.NewCIStr("t"), ) require.NoError(t, err) tableInfo = tbl.Meta() @@ -699,7 +699,7 @@ func TestTruncateAPartition(t *testing.T) { testKit.MustExec("analyze table t") is := do.InfoSchema() tbl, err := is.TableByName(context.Background(), - pmodel.NewCIStr("test"), pmodel.NewCIStr("t"), + ast.NewCIStr("test"), ast.NewCIStr("t"), ) require.NoError(t, err) tableInfo := tbl.Meta() @@ -761,7 +761,7 @@ func TestTruncateAHashPartition(t *testing.T) { testKit.MustExec("analyze table t") is := do.InfoSchema() tbl, err := is.TableByName(context.Background(), - pmodel.NewCIStr("test"), pmodel.NewCIStr("t"), + ast.NewCIStr("test"), ast.NewCIStr("t"), ) require.NoError(t, err) tableInfo := tbl.Meta() @@ -829,7 +829,7 @@ func TestTruncatePartitions(t *testing.T) { testKit.MustExec("analyze table t") is := do.InfoSchema() tbl, err := is.TableByName(context.Background(), - pmodel.NewCIStr("test"), pmodel.NewCIStr("t"), + ast.NewCIStr("test"), ast.NewCIStr("t"), ) require.NoError(t, err) tableInfo := tbl.Meta() @@ -900,7 +900,7 @@ func TestDropAPartition(t *testing.T) { testKit.MustExec("analyze table t") is := do.InfoSchema() tbl, err := is.TableByName(context.Background(), - pmodel.NewCIStr("test"), pmodel.NewCIStr("t"), + ast.NewCIStr("test"), ast.NewCIStr("t"), ) require.NoError(t, err) tableInfo := tbl.Meta() @@ -967,7 +967,7 @@ func TestDropPartitions(t *testing.T) { testKit.MustExec("analyze table t") is := do.InfoSchema() tbl, err := is.TableByName(context.Background(), - pmodel.NewCIStr("test"), pmodel.NewCIStr("t"), + ast.NewCIStr("test"), ast.NewCIStr("t"), ) require.NoError(t, err) tableInfo := tbl.Meta() @@ -1045,7 +1045,7 @@ func TestExchangeAPartition(t *testing.T) { testKit.MustExec("analyze table t") is := do.InfoSchema() tbl, err := is.TableByName(context.Background(), - pmodel.NewCIStr("test"), pmodel.NewCIStr("t"), + ast.NewCIStr("test"), ast.NewCIStr("t"), ) require.NoError(t, err) tableInfo := tbl.Meta() @@ -1065,7 +1065,7 @@ func TestExchangeAPartition(t *testing.T) { testKit.MustExec("analyze table t1") is = do.InfoSchema() tbl1, err := is.TableByName(context.Background(), - pmodel.NewCIStr("test"), pmodel.NewCIStr("t1"), + ast.NewCIStr("test"), ast.NewCIStr("t1"), ) require.NoError(t, err) tableInfo1 := tbl1.Meta() @@ -1101,7 +1101,7 @@ func TestExchangeAPartition(t *testing.T) { testKit.MustExec("analyze table t2") is = do.InfoSchema() tbl2, err := is.TableByName(context.Background(), - pmodel.NewCIStr("test"), pmodel.NewCIStr("t2"), + ast.NewCIStr("test"), ast.NewCIStr("t2"), ) require.NoError(t, err) tableInfo2 := tbl2.Meta() @@ -1145,7 +1145,7 @@ func TestExchangeAPartition(t *testing.T) { testKit.MustExec("analyze table t3") is = do.InfoSchema() tbl3, err := is.TableByName(context.Background(), - pmodel.NewCIStr("test"), pmodel.NewCIStr("t3"), + ast.NewCIStr("test"), ast.NewCIStr("t3"), ) require.NoError(t, err) tableInfo3 := tbl3.Meta() @@ -1197,7 +1197,7 @@ func TestRemovePartitioning(t *testing.T) { testKit.MustExec("analyze table t") is := do.InfoSchema() tbl, err := is.TableByName(context.Background(), - pmodel.NewCIStr("test"), pmodel.NewCIStr("t"), + ast.NewCIStr("test"), ast.NewCIStr("t"), ) require.NoError(t, err) tableInfo := tbl.Meta() @@ -1233,7 +1233,7 @@ func TestRemovePartitioning(t *testing.T) { // Get new table id after remove partitioning. is = do.InfoSchema() tbl, err = is.TableByName(context.Background(), - pmodel.NewCIStr("test"), pmodel.NewCIStr("t"), + ast.NewCIStr("test"), ast.NewCIStr("t"), ) require.NoError(t, err) tableInfo = tbl.Meta() @@ -1275,7 +1275,7 @@ func TestAddPartitioning(t *testing.T) { testKit.MustExec("analyze table t") is := do.InfoSchema() tbl, err := is.TableByName(context.Background(), - pmodel.NewCIStr("test"), pmodel.NewCIStr("t"), + ast.NewCIStr("test"), ast.NewCIStr("t"), ) require.NoError(t, err) tableInfo := tbl.Meta() @@ -1296,7 +1296,7 @@ func TestAddPartitioning(t *testing.T) { // Get new table id after remove partitioning. is = do.InfoSchema() tbl, err = is.TableByName(context.Background(), - pmodel.NewCIStr("test"), pmodel.NewCIStr("t"), + ast.NewCIStr("test"), ast.NewCIStr("t"), ) require.NoError(t, err) tableInfo = tbl.Meta() @@ -1318,7 +1318,7 @@ func TestDropSchema(t *testing.T) { require.NoError(t, h.DumpStatsDeltaToKV(true)) is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo := tbl.Meta() // Check the current stats meta version. @@ -1362,7 +1362,7 @@ func TestExchangePartition(t *testing.T) { tk.MustExec("use test") tk.MustExec("create table t (c1 int)") is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) var wg util.WaitGroupWrapper for i := 0; i < 20; i++ { diff --git a/pkg/statistics/handle/globalstats/BUILD.bazel b/pkg/statistics/handle/globalstats/BUILD.bazel index b02d0225b47ae..d25d9c89a90ac 100644 --- a/pkg/statistics/handle/globalstats/BUILD.bazel +++ b/pkg/statistics/handle/globalstats/BUILD.bazel @@ -49,7 +49,7 @@ go_test( deps = [ "//pkg/domain", "//pkg/kv", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/parser/mysql", "//pkg/session", "//pkg/sessionctx/stmtctx", diff --git a/pkg/statistics/handle/globalstats/global_stats_test.go b/pkg/statistics/handle/globalstats/global_stats_test.go index 4564af9fa24ea..6f33aff67de6c 100644 --- a/pkg/statistics/handle/globalstats/global_stats_test.go +++ b/pkg/statistics/handle/globalstats/global_stats_test.go @@ -21,7 +21,7 @@ import ( "time" "github.com/pingcap/failpoint" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/session" statstestutil "github.com/pingcap/tidb/pkg/statistics/handle/ddl/testutil" "github.com/pingcap/tidb/pkg/testkit" @@ -545,7 +545,7 @@ partition by range (a) ( is := do.InfoSchema() h := do.StatsHandle() require.NoError(t, h.Update(context.Background(), is)) - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo := tbl.Meta() globalStats := h.GetTableStats(tableInfo) @@ -598,7 +598,7 @@ func TestDDLPartition4GlobalStats(t *testing.T) { tk.MustExec("analyze table t") result := tk.MustQuery("show stats_meta where table_name = 't';").Rows() require.Len(t, result, 7) - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo := tbl.Meta() globalStats := h.GetTableStats(tableInfo) diff --git a/pkg/statistics/handle/handletest/BUILD.bazel b/pkg/statistics/handle/handletest/BUILD.bazel index 1928e1250a185..422c9298082c0 100644 --- a/pkg/statistics/handle/handletest/BUILD.bazel +++ b/pkg/statistics/handle/handletest/BUILD.bazel @@ -13,7 +13,7 @@ go_test( deps = [ "//pkg/config", "//pkg/domain", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/planner/cardinality", "//pkg/sessionctx/variable", "//pkg/statistics", diff --git a/pkg/statistics/handle/handletest/analyze/BUILD.bazel b/pkg/statistics/handle/handletest/analyze/BUILD.bazel index aa55f1fb3f025..86873c6591680 100644 --- a/pkg/statistics/handle/handletest/analyze/BUILD.bazel +++ b/pkg/statistics/handle/handletest/analyze/BUILD.bazel @@ -12,7 +12,7 @@ go_test( shard_count = 5, deps = [ "//pkg/domain", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/sessionctx/variable", "//pkg/statistics", "//pkg/testkit", diff --git a/pkg/statistics/handle/handletest/analyze/analyze_test.go b/pkg/statistics/handle/handletest/analyze/analyze_test.go index c372a5cc5207e..51e67f53a3335 100644 --- a/pkg/statistics/handle/handletest/analyze/analyze_test.go +++ b/pkg/statistics/handle/handletest/analyze/analyze_test.go @@ -21,7 +21,7 @@ import ( "testing" "github.com/pingcap/tidb/pkg/domain" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/statistics" "github.com/pingcap/tidb/pkg/testkit" @@ -31,7 +31,7 @@ import ( // nolint:unused func checkForGlobalStatsWithOpts(t *testing.T, dom *domain.Domain, db, tt, pp string, topn, buckets int) { - tbl, err := dom.InfoSchema().TableByName(context.Background(), model.NewCIStr(db), model.NewCIStr(tt)) + tbl, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr(db), ast.NewCIStr(tt)) require.NoError(t, err) tblInfo := tbl.Meta() diff --git a/pkg/statistics/handle/handletest/handle_test.go b/pkg/statistics/handle/handletest/handle_test.go index 64c81379553f3..aee3485ee5d7c 100644 --- a/pkg/statistics/handle/handletest/handle_test.go +++ b/pkg/statistics/handle/handletest/handle_test.go @@ -25,7 +25,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb/pkg/config" "github.com/pingcap/tidb/pkg/domain" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/planner/cardinality" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/statistics" @@ -50,7 +50,7 @@ func TestEmptyTable(t *testing.T) { testKit.MustExec("analyze table t") do := dom is := do.InfoSchema() - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo := tbl.Meta() statsTbl := do.StatsHandle().GetTableStats(tableInfo) @@ -67,7 +67,7 @@ func TestColumnIDs(t *testing.T) { testKit.MustExec("analyze table t all columns") do := dom is := do.InfoSchema() - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo := tbl.Meta() statsTbl := do.StatsHandle().GetTableStats(tableInfo) @@ -89,7 +89,7 @@ func TestColumnIDs(t *testing.T) { do.StatsHandle().Clear() err = do.StatsHandle().Update(context.Background(), is) require.NoError(t, err) - tbl, err = is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err = is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo = tbl.Meta() statsTbl = do.StatsHandle().GetTableStats(tableInfo) @@ -116,7 +116,7 @@ func TestVersion(t *testing.T) { testKit.MustExec("analyze table t1 all columns") do := dom is := do.InfoSchema() - tbl1, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t1")) + tbl1, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t1")) require.NoError(t, err) tableInfo1 := tbl1.Meta() h, err := handle.NewHandle( @@ -145,7 +145,7 @@ func TestVersion(t *testing.T) { testKit.MustExec("create table t2 (c1 int, c2 int)") testKit.MustExec("analyze table t2 all columns") is = do.InfoSchema() - tbl2, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t2")) + tbl2, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t2")) require.NoError(t, err) tableInfo2 := tbl2.Meta() // A smaller version write, and we can still read it. @@ -196,7 +196,7 @@ func TestVersion(t *testing.T) { statsTbl2 = h.GetTableStats(tableInfo2) require.False(t, statsTbl2.Pseudo) require.Nil(t, statsTbl2.GetCol(int64(3))) - tbl2, err = is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t2")) + tbl2, err = is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t2")) require.NoError(t, err) tableInfo2 = tbl2.Meta() statsTbl2, err = h.TableStatsFromStorage(tableInfo2, tableInfo2.ID, true, 0) @@ -219,7 +219,7 @@ func TestLoadHist(t *testing.T) { } testKit.MustExec("analyze table t") is := do.InfoSchema() - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo := tbl.Meta() oldStatsTbl := h.GetTableStats(tableInfo) @@ -242,7 +242,7 @@ func TestLoadHist(t *testing.T) { err = statstestutil.HandleNextDDLEventWithTxn(h) require.NoError(t, err) is = do.InfoSchema() - tbl, err = is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err = is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo = tbl.Meta() require.NoError(t, h.Update(context.Background(), is)) @@ -455,7 +455,7 @@ func TestExtendedStatsOps(t *testing.T) { )) do := dom is := do.InfoSchema() - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo := tbl.Meta() err = do.StatsHandle().Update(context.Background(), is) @@ -499,7 +499,7 @@ func TestAdminReloadStatistics1(t *testing.T) { )) do := dom is := do.InfoSchema() - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo := tbl.Meta() err = do.StatsHandle().Update(context.Background(), is) @@ -576,7 +576,7 @@ func TestCorrelationStatsCompute(t *testing.T) { )) do := dom is := do.InfoSchema() - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo := tbl.Meta() err = do.StatsHandle().Update(context.Background(), is) @@ -659,7 +659,7 @@ func TestSyncStatsExtendedRemoval(t *testing.T) { tk.MustExec("analyze table t") do := dom is := do.InfoSchema() - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo := tbl.Meta() statsTbl := do.StatsHandle().GetTableStats(tableInfo) @@ -1127,7 +1127,7 @@ func TestLoadHistogramWithCollate(t *testing.T) { do := dom h := do.StatsHandle() is := do.InfoSchema() - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tblInfo := tbl.Meta() _, err = h.TableStatsFromStorage(tblInfo, tblInfo.ID, true, 0) @@ -1146,7 +1146,7 @@ func TestStatsCacheUpdateSkip(t *testing.T) { require.NoError(t, h.DumpStatsDeltaToKV(true)) testKit.MustExec("analyze table t") is := do.InfoSchema() - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo := tbl.Meta() statsTbl1 := h.GetTableStats(tableInfo) @@ -1172,7 +1172,7 @@ func testIncrementalModifyCountUpdateHelper(analyzeSnapshot bool) func(*testing. h := dom.StatsHandle() err := statstestutil.HandleNextDDLEventWithTxn(h) require.NoError(t, err) - tbl, err := dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tblInfo := tbl.Meta() tid := tblInfo.ID @@ -1249,7 +1249,7 @@ func TestRecordHistoricalStatsToStorage(t *testing.T) { tk.MustExec("alter table t add index multi(a, b)") tk.MustExec("analyze table t with 2 topn") - tableInfo, err := dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tableInfo, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) version, err := dom.StatsHandle().RecordHistoricalStatsToStorage("t", tableInfo.Meta(), tableInfo.Meta().ID, false) require.NoError(t, err) @@ -1274,7 +1274,7 @@ func TestEvictedColumnLoadedStatus(t *testing.T) { tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int)") tk.MustExec("analyze table test.t") - tbl, err := dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.Nil(t, err) tblStats := domain.GetDomain(tk.Session()).StatsHandle().GetTableStats(tbl.Meta()) tblStats.ForEachColumnImmutable(func(_ int64, col *statistics.Column) bool { @@ -1304,7 +1304,7 @@ func TestUninitializedStatsStatus(t *testing.T) { require.NoError(t, h.DumpStatsDeltaToKV(true)) is := dom.InfoSchema() require.NoError(t, h.Update(context.Background(), is)) - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tblInfo := tbl.Meta() tblStats := h.GetTableStats(tblInfo) @@ -1393,7 +1393,7 @@ func TestInitStatsLite(t *testing.T) { }() is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tblInfo := tbl.Meta() colBID := tblInfo.Columns[1].ID @@ -1470,7 +1470,7 @@ func TestSkipMissingPartitionStats(t *testing.T) { h := dom.StatsHandle() require.NoError(t, h.DumpStatsDeltaToKV(true)) tk.MustExec("analyze table t partition p0, p1") - tbl, err := dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tblInfo := tbl.Meta() globalStats := h.GetTableStats(tblInfo) @@ -1498,7 +1498,7 @@ func TestStatsCacheUpdateTimeout(t *testing.T) { h := dom.StatsHandle() require.NoError(t, h.DumpStatsDeltaToKV(true)) tk.MustExec("analyze table t partition p0, p1") - tbl, err := dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tblInfo := tbl.Meta() globalStats := h.GetTableStats(tblInfo) diff --git a/pkg/statistics/handle/handletest/initstats/BUILD.bazel b/pkg/statistics/handle/handletest/initstats/BUILD.bazel index 1b7e6262b46dd..4b98efa42519a 100644 --- a/pkg/statistics/handle/handletest/initstats/BUILD.bazel +++ b/pkg/statistics/handle/handletest/initstats/BUILD.bazel @@ -11,7 +11,7 @@ go_test( shard_count = 4, deps = [ "//pkg/config", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/statistics/handle", "//pkg/statistics/handle/types", "//pkg/testkit", diff --git a/pkg/statistics/handle/handletest/initstats/load_stats_test.go b/pkg/statistics/handle/handletest/initstats/load_stats_test.go index af39654e63de6..81d7689a8fe1d 100644 --- a/pkg/statistics/handle/handletest/initstats/load_stats_test.go +++ b/pkg/statistics/handle/handletest/initstats/load_stats_test.go @@ -20,7 +20,7 @@ import ( "testing" "github.com/pingcap/tidb/pkg/config" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/statistics/handle" "github.com/pingcap/tidb/pkg/statistics/handle/types" "github.com/pingcap/tidb/pkg/testkit" @@ -72,7 +72,7 @@ func testConcurrentlyInitStats(t *testing.T) { require.Equal(t, h.MemConsumed(), int64(0)) require.NoError(t, h.InitStats(context.Background(), is)) for i := 1; i < 10; i++ { - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr(fmt.Sprintf("t%v", i))) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr(fmt.Sprintf("t%v", i))) require.NoError(t, err) stats, ok := h.StatsCache.Get(tbl.Meta().ID) require.True(t, ok) @@ -91,7 +91,7 @@ func testConcurrentlyInitStats(t *testing.T) { tk.MustQuery(fmt.Sprintf("explain select * from t%v where c >= 1", i)).CheckNotContain("pseudo") } for i := 1; i < 10; i++ { - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr(fmt.Sprintf("t%v", i))) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr(fmt.Sprintf("t%v", i))) require.NoError(t, err) stats, ok := h.StatsCache.Get(tbl.Meta().ID) require.True(t, ok) diff --git a/pkg/statistics/handle/handletest/lockstats/BUILD.bazel b/pkg/statistics/handle/handletest/lockstats/BUILD.bazel index 4e83ece10b940..3844e60219027 100644 --- a/pkg/statistics/handle/handletest/lockstats/BUILD.bazel +++ b/pkg/statistics/handle/handletest/lockstats/BUILD.bazel @@ -15,7 +15,7 @@ go_test( "//pkg/domain", "//pkg/kv", "//pkg/meta/model", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/statistics", "//pkg/testkit", "//pkg/testkit/testsetup", diff --git a/pkg/statistics/handle/handletest/lockstats/lock_partition_stats_test.go b/pkg/statistics/handle/handletest/lockstats/lock_partition_stats_test.go index 1f83cd2a1dde0..02f637aa4bde2 100644 --- a/pkg/statistics/handle/handletest/lockstats/lock_partition_stats_test.go +++ b/pkg/statistics/handle/handletest/lockstats/lock_partition_stats_test.go @@ -24,7 +24,7 @@ import ( "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/statistics" "github.com/pingcap/tidb/pkg/testkit" "github.com/stretchr/testify/require" @@ -538,7 +538,7 @@ func setupTestEnvironmentWithPartitionedTableT(t *testing.T) (kv.Storage, *domai tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int, b varchar(10), index idx_b (b)) partition by range(a) (partition p0 values less than (10), partition p1 values less than (20))") tk.MustExec("analyze table test.t") - tbl, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.Nil(t, err) return store, dom, tk, tbl.Meta() diff --git a/pkg/statistics/handle/handletest/lockstats/lock_table_stats_test.go b/pkg/statistics/handle/handletest/lockstats/lock_table_stats_test.go index 78db5372acdac..1637144effe6a 100644 --- a/pkg/statistics/handle/handletest/lockstats/lock_table_stats_test.go +++ b/pkg/statistics/handle/handletest/lockstats/lock_table_stats_test.go @@ -26,7 +26,7 @@ import ( "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/statistics" "github.com/pingcap/tidb/pkg/testkit" "github.com/stretchr/testify/require" @@ -180,9 +180,9 @@ func TestLockAndUnlockTablesStats(t *testing.T) { tk.MustExec("create table t1(a int, b varchar(10), index idx_b (b))") tk.MustExec("create table t2(a int, b varchar(10), index idx_b (b))") tk.MustExec("analyze table test.t1, test.t2") - tbl1, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t1")) + tbl1, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t1")) require.Nil(t, err) - tbl2, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t2")) + tbl2, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t2")) require.Nil(t, err) handle := domain.GetDomain(tk.Session()).StatsHandle() @@ -373,7 +373,7 @@ func setupTestEnvironmentWithTableT(t *testing.T) (kv.Storage, *domain.Domain, * tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int, b varchar(10), index idx_b (b))") tk.MustExec("analyze table test.t") - tbl, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.Nil(t, err) return store, dom, tk, tbl.Meta() diff --git a/pkg/statistics/handle/handletest/statstest/BUILD.bazel b/pkg/statistics/handle/handletest/statstest/BUILD.bazel index 096e8ebac5d22..9493c25c354e0 100644 --- a/pkg/statistics/handle/handletest/statstest/BUILD.bazel +++ b/pkg/statistics/handle/handletest/statstest/BUILD.bazel @@ -12,7 +12,7 @@ go_test( shard_count = 13, deps = [ "//pkg/config", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/parser/mysql", "//pkg/statistics", "//pkg/statistics/handle/ddl/testutil", diff --git a/pkg/statistics/handle/handletest/statstest/stats_test.go b/pkg/statistics/handle/handletest/statstest/stats_test.go index 9fbde012bd3f1..37587a5b2376c 100644 --- a/pkg/statistics/handle/handletest/statstest/stats_test.go +++ b/pkg/statistics/handle/handletest/statstest/stats_test.go @@ -22,7 +22,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb/pkg/config" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/statistics" statstestutil "github.com/pingcap/tidb/pkg/statistics/handle/ddl/testutil" @@ -41,7 +41,7 @@ func TestStatsCacheProcess(t *testing.T) { analyzehelper.TriggerPredicateColumnsCollection(t, testKit, store, "t", "c1", "c2") do := dom is := do.InfoSchema() - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo := tbl.Meta() statsTbl := do.StatsHandle().GetTableStats(tableInfo) @@ -73,7 +73,7 @@ func TestStatsCache(t *testing.T) { analyzehelper.TriggerPredicateColumnsCollection(t, testKit, store, "t", "c1", "c2") do := dom is := do.InfoSchema() - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo := tbl.Meta() statsTbl := do.StatsHandle().GetTableStats(tableInfo) @@ -121,7 +121,7 @@ func TestStatsCacheMemTracker(t *testing.T) { testKit.MustExec("insert into t values(1, 2, 3)") do := dom is := do.InfoSchema() - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo := tbl.Meta() statsTbl := do.StatsHandle().GetTableStats(tableInfo) @@ -181,7 +181,7 @@ func TestStatsStoreAndLoad(t *testing.T) { analyzehelper.TriggerPredicateColumnsCollection(t, testKit, store, "t", "c1") do := dom is := do.InfoSchema() - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo := tbl.Meta() @@ -217,7 +217,7 @@ func testInitStatsMemTrace(t *testing.T) { var memCostTot int64 for i := 1; i < 10; i++ { - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr(fmt.Sprintf("t%v", i))) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr(fmt.Sprintf("t%v", i))) require.NoError(t, err) tStats := h.GetTableStats(tbl.Meta()) memCostTot += tStats.MemoryUsage().TotalMemUsage @@ -292,7 +292,7 @@ func TestInitStats(t *testing.T) { testKit.MustExec("analyze table t") h := dom.StatsHandle() is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) // `Update` will not use load by need strategy when `Lease` is 0, and `InitStats` is only called when // `Lease` is not 0, so here we just change it. @@ -345,7 +345,7 @@ func TestInitStats51358(t *testing.T) { require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/statistics/handle/cache/StatsCacheGetNil")) }() require.NoError(t, h.InitStats(context.Background(), is)) - tbl, err := dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) stats := h.GetTableStats(tbl.Meta()) stats.ForEachColumnImmutable(func(_ int64, column *statistics.Column) bool { @@ -398,7 +398,7 @@ func initStatsVer2(t *testing.T) { err = statstestutil.HandleNextDDLEventWithTxn(h) require.NoError(t, err) is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) // `Update` will not use load by need strategy when `Lease` is 0, and `InitStats` is only called when // `Lease` is not 0, so here we just change it. diff --git a/pkg/statistics/handle/storage/BUILD.bazel b/pkg/statistics/handle/storage/BUILD.bazel index 72070ce6e1554..9b77ba15d5f90 100644 --- a/pkg/statistics/handle/storage/BUILD.bazel +++ b/pkg/statistics/handle/storage/BUILD.bazel @@ -18,7 +18,6 @@ go_library( "//pkg/kv", "//pkg/meta/model", "//pkg/parser/ast", - "//pkg/parser/model", "//pkg/parser/mysql", "//pkg/parser/terror", "//pkg/sessionctx", @@ -64,7 +63,7 @@ go_test( ":storage", "//pkg/domain", "//pkg/meta/model", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/planner/cardinality", "//pkg/sessionctx/variable", "//pkg/statistics", diff --git a/pkg/statistics/handle/storage/dump_test.go b/pkg/statistics/handle/storage/dump_test.go index 5a8dcb65fbc05..bdec873c11c72 100644 --- a/pkg/statistics/handle/storage/dump_test.go +++ b/pkg/statistics/handle/storage/dump_test.go @@ -28,7 +28,7 @@ import ( "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/statistics" statstestutil "github.com/pingcap/tidb/pkg/statistics/handle/ddl/testutil" "github.com/pingcap/tidb/pkg/statistics/handle/internal" @@ -101,7 +101,7 @@ func TestConversion(t *testing.T) { require.Nil(t, h.DumpStatsDeltaToKV(true)) require.Nil(t, h.Update(context.Background(), is)) - tableInfo, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tableInfo, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) jsonTbl, err := h.DumpStatsToJSON("test", tableInfo.Meta(), nil, true) require.NoError(t, err) @@ -126,7 +126,7 @@ func getStatsJSON(t *testing.T, dom *domain.Domain, db, tableName string) *stats is := dom.InfoSchema() h := dom.StatsHandle() require.Nil(t, h.Update(context.Background(), is)) - table, err := is.TableByName(context.Background(), pmodel.NewCIStr(db), pmodel.NewCIStr(tableName)) + table, err := is.TableByName(context.Background(), ast.NewCIStr(db), ast.NewCIStr(tableName)) require.NoError(t, err) tableInfo := table.Meta() jsonTbl, err := h.DumpStatsToJSON("test", tableInfo, nil, true) @@ -138,7 +138,7 @@ func persistStats(ctx context.Context, t *testing.T, dom *domain.Domain, db, tab is := dom.InfoSchema() h := dom.StatsHandle() require.Nil(t, h.Update(context.Background(), is)) - table, err := is.TableByName(context.Background(), pmodel.NewCIStr(db), pmodel.NewCIStr(tableName)) + table, err := is.TableByName(context.Background(), ast.NewCIStr(db), ast.NewCIStr(tableName)) require.NoError(t, err) tableInfo := table.Meta() err = h.PersistStatsBySnapshot(ctx, "test", tableInfo, math.MaxUint64, persist) @@ -212,7 +212,7 @@ func TestLoadPartitionStats(t *testing.T) { tk.MustExec("insert into t values " + strings.Join(vals, ",")) tk.MustExec("analyze table t") - table, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + table, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo := table.Meta() jsonTbl, err := dom.StatsHandle().DumpStatsToJSON("test", tableInfo, nil, true) @@ -258,7 +258,7 @@ func TestLoadPredicateColumns(t *testing.T) { require.NoError(t, h.DumpColStatsUsageToKV()) tk.MustExec("analyze table t") - table, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + table, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo := table.Meta() jsonTbl, err := h.DumpStatsToJSON("test", tableInfo, nil, true) @@ -304,7 +304,7 @@ func TestLoadPartitionStatsErrPanic(t *testing.T) { tk.MustExec("insert into t values " + strings.Join(vals, ",")) tk.MustExec("analyze table t") - table, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + table, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo := table.Meta() jsonTbl, err := dom.StatsHandle().DumpStatsToJSON("test", tableInfo, nil, true) @@ -343,7 +343,7 @@ PARTITION BY RANGE ( a ) ( h := dom.StatsHandle() require.Nil(t, h.Update(context.Background(), is)) - table, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + table, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo := table.Meta() jsonTbl, err := h.DumpStatsToJSON("test", tableInfo, nil, true) @@ -379,7 +379,7 @@ func TestDumpAlteredTable(t *testing.T) { tk.MustExec("create table t(a int, b int)") tk.MustExec("analyze table t") tk.MustExec("alter table t drop column a") - table, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + table, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) _, err = h.DumpStatsToJSON("test", table.Meta(), nil, true) require.NoError(t, err) @@ -396,7 +396,7 @@ func TestDumpCMSketchWithTopN(t *testing.T) { testKit.MustExec("analyze table t") is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo := tbl.Meta() h := dom.StatsHandle() @@ -438,7 +438,7 @@ func TestDumpPseudoColumns(t *testing.T) { testKit.MustExec("analyze table t index idx") is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) h := dom.StatsHandle() _, err = h.DumpStatsToJSON("test", tbl.Meta(), nil, true) @@ -459,7 +459,7 @@ func TestDumpExtendedStats(t *testing.T) { tk.MustExec("analyze table t") is := dom.InfoSchema() - tableInfo, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tableInfo, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tbl := h.GetTableStats(tableInfo.Meta()) jsonTbl, err := h.DumpStatsToJSON("test", tableInfo.Meta(), nil, true) @@ -496,7 +496,7 @@ func TestDumpVer2Stats(t *testing.T) { tk.MustExec("analyze table t with 2 topn") h := dom.StatsHandle() is := dom.InfoSchema() - tableInfo, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tableInfo, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) storageTbl, err := h.TableStatsFromStorage(tableInfo.Meta(), tableInfo.Meta().ID, false, 0) @@ -548,7 +548,7 @@ func TestLoadStatsForNewCollation(t *testing.T) { tk.MustExec("analyze table t with 2 topn") h := dom.StatsHandle() is := dom.InfoSchema() - tableInfo, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tableInfo, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) storageTbl, err := h.TableStatsFromStorage(tableInfo.Meta(), tableInfo.Meta().ID, false, 0) @@ -599,7 +599,7 @@ func TestJSONTableToBlocks(t *testing.T) { tk.MustExec("analyze table t with 2 topn") h := dom.StatsHandle() is := dom.InfoSchema() - tableInfo, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tableInfo, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) dumpJSONTable, err := h.DumpStatsToJSON("test", tableInfo.Meta(), nil, true) @@ -682,7 +682,7 @@ func TestLoadStatsFromOldVersion(t *testing.T) { jsonTbl := &statsutil.JSONTable{} require.NoError(t, json.Unmarshal([]byte(statsJSONFromOldVersion), jsonTbl)) require.NoError(t, h.LoadStatsFromJSON(context.Background(), is, jsonTbl, 0)) - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) statsTbl := h.GetTableStats(tbl.Meta()) statsTbl.ForEachColumnImmutable(func(i int64, col *statistics.Column) bool { diff --git a/pkg/statistics/handle/storage/gc_test.go b/pkg/statistics/handle/storage/gc_test.go index b82e5dceb8d95..9aaf9c6a4f753 100644 --- a/pkg/statistics/handle/storage/gc_test.go +++ b/pkg/statistics/handle/storage/gc_test.go @@ -20,7 +20,7 @@ import ( "time" "github.com/pingcap/failpoint" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/sessionctx/variable" statstestutil "github.com/pingcap/tidb/pkg/statistics/handle/ddl/testutil" "github.com/pingcap/tidb/pkg/testkit" @@ -190,7 +190,7 @@ func TestExtremCaseOfGC(t *testing.T) { testKit.MustExec("create table t(a int, b int)") testKit.MustExec("insert into t values (1,2),(3,4)") testKit.MustExec("analyze table t") - tbl, err := dom.InfoSchema().TableByName(context.TODO(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := dom.InfoSchema().TableByName(context.TODO(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tid := tbl.Meta().ID rs := testKit.MustQuery("select * from mysql.stats_meta where table_id = ?", tid) diff --git a/pkg/statistics/handle/storage/read_test.go b/pkg/statistics/handle/storage/read_test.go index 1e67edb990127..78b5e590e68a6 100644 --- a/pkg/statistics/handle/storage/read_test.go +++ b/pkg/statistics/handle/storage/read_test.go @@ -18,7 +18,7 @@ import ( "context" "testing" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/planner/cardinality" "github.com/pingcap/tidb/pkg/statistics" "github.com/pingcap/tidb/pkg/testkit" @@ -43,7 +43,7 @@ func TestLoadStats(t *testing.T) { testKit.MustExec("analyze table t") is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo := tbl.Meta() colAID := tableInfo.Columns[0].ID diff --git a/pkg/statistics/handle/storage/stats_read_writer.go b/pkg/statistics/handle/storage/stats_read_writer.go index 26eaa6ffd3107..3b1d1f4b0812f 100644 --- a/pkg/statistics/handle/storage/stats_read_writer.go +++ b/pkg/statistics/handle/storage/stats_read_writer.go @@ -26,7 +26,7 @@ import ( "github.com/pingcap/tidb/pkg/config" "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "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" @@ -567,7 +567,7 @@ func (s *statsReadWriter) LoadStatsFromJSONConcurrently( // LoadStatsFromJSONNoUpdate will load statistic from JSONTable, and save it to the storage. func (s *statsReadWriter) LoadStatsFromJSONNoUpdate(ctx context.Context, is infoschema.InfoSchema, jsonTbl *statsutil.JSONTable, concurrencyForPartition int) error { - table, err := is.TableByName(context.Background(), pmodel.NewCIStr(jsonTbl.DatabaseName), pmodel.NewCIStr(jsonTbl.TableName)) + table, err := is.TableByName(context.Background(), ast.NewCIStr(jsonTbl.DatabaseName), ast.NewCIStr(jsonTbl.TableName)) if err != nil { return errors.Trace(err) } diff --git a/pkg/statistics/handle/storage/stats_read_writer_test.go b/pkg/statistics/handle/storage/stats_read_writer_test.go index ecf5408f4dcbc..a17254cfb5881 100644 --- a/pkg/statistics/handle/storage/stats_read_writer_test.go +++ b/pkg/statistics/handle/storage/stats_read_writer_test.go @@ -18,7 +18,7 @@ import ( "context" "testing" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/testkit" "github.com/stretchr/testify/require" ) @@ -47,7 +47,7 @@ func TestUpdateStatsMetaVersionForGC(t *testing.T) { testKit.MustExec("analyze table t") is := do.InfoSchema() tbl, err := is.TableByName(context.Background(), - model.NewCIStr("test"), model.NewCIStr("t"), + ast.NewCIStr("test"), ast.NewCIStr("t"), ) require.NoError(t, err) tableInfo := tbl.Meta() diff --git a/pkg/statistics/handle/syncload/BUILD.bazel b/pkg/statistics/handle/syncload/BUILD.bazel index 04148626bece3..478ae548d12f3 100644 --- a/pkg/statistics/handle/syncload/BUILD.bazel +++ b/pkg/statistics/handle/syncload/BUILD.bazel @@ -40,7 +40,7 @@ go_test( ":syncload", "//pkg/config", "//pkg/meta/model", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/sessionctx", "//pkg/sessionctx/stmtctx", "//pkg/statistics/handle/ddl/testutil", diff --git a/pkg/statistics/handle/syncload/stats_syncload_test.go b/pkg/statistics/handle/syncload/stats_syncload_test.go index dec30cd82bb3f..199d886443d67 100644 --- a/pkg/statistics/handle/syncload/stats_syncload_test.go +++ b/pkg/statistics/handle/syncload/stats_syncload_test.go @@ -22,7 +22,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb/pkg/config" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" statstestutil "github.com/pingcap/tidb/pkg/statistics/handle/ddl/testutil" @@ -90,7 +90,7 @@ func TestConcurrentLoadHist(t *testing.T) { testKit.MustExec("analyze table t") is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo := tbl.Meta() h := dom.StatsHandle() @@ -133,7 +133,7 @@ func TestConcurrentLoadHistTimeout(t *testing.T) { testKit.MustExec("analyze table t") is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo := tbl.Meta() h := dom.StatsHandle() @@ -185,7 +185,7 @@ func TestConcurrentLoadHistWithPanicAndFail(t *testing.T) { testKit.MustExec("analyze table t") is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo := tbl.Meta() h := dom.StatsHandle() @@ -300,7 +300,7 @@ func TestRetry(t *testing.T) { testKit.MustExec("analyze table t") is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo := tbl.Meta() @@ -389,7 +389,7 @@ func TestSendLoadRequestsWaitTooLong(t *testing.T) { tk.MustExec("analyze table t all columns") h := dom.StatsHandle() is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo := tbl.Meta() neededColumns := make([]model.StatsLoadItem, 0, len(tableInfo.Columns)) @@ -421,7 +421,7 @@ func TestSyncLoadOnObjectWhichCanNotFoundInStorage(t *testing.T) { <-h.DDLEventCh() tk.MustExec("insert into t values (1,1,1),(2,2,2),(3,3,3)") tk.MustExec("analyze table t columns a, b") - tbl, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, h.InitStatsLite(context.TODO())) require.NoError(t, err) require.NotNil(t, tbl) @@ -439,7 +439,7 @@ func TestSyncLoadOnObjectWhichCanNotFoundInStorage(t *testing.T) { err = statstestutil.HandleNextDDLEventWithTxn(h) require.NoError(t, err) require.NoError(t, h.Update(context.Background(), dom.InfoSchema())) - tbl, err = dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err = dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) require.NotNil(t, tbl) tblInfo = tbl.Meta() diff --git a/pkg/statistics/handle/updatetest/BUILD.bazel b/pkg/statistics/handle/updatetest/BUILD.bazel index 73f1c44ff8b50..d450a0e1e029b 100644 --- a/pkg/statistics/handle/updatetest/BUILD.bazel +++ b/pkg/statistics/handle/updatetest/BUILD.bazel @@ -10,7 +10,7 @@ go_test( flaky = True, shard_count = 23, deps = [ - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/parser/mysql", "//pkg/planner/cardinality", "//pkg/sessionctx", diff --git a/pkg/statistics/handle/updatetest/update_test.go b/pkg/statistics/handle/updatetest/update_test.go index ef2ecddae9548..1e96df5550bbc 100644 --- a/pkg/statistics/handle/updatetest/update_test.go +++ b/pkg/statistics/handle/updatetest/update_test.go @@ -23,7 +23,7 @@ import ( "testing" "time" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/cardinality" "github.com/pingcap/tidb/pkg/sessionctx" @@ -60,7 +60,7 @@ func TestSingleSessionInsert(t *testing.T) { } is := dom.InfoSchema() - tbl1, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t1")) + tbl1, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t1")) require.NoError(t, err) tableInfo1 := tbl1.Meta() h := dom.StatsHandle() @@ -74,7 +74,7 @@ func TestSingleSessionInsert(t *testing.T) { stats1 := h.GetTableStats(tableInfo1) require.Equal(t, int64(rowCount1), stats1.RealtimeCount) - tbl2, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t2")) + tbl2, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t2")) require.NoError(t, err) tableInfo2 := tbl2.Meta() stats2 := h.GetTableStats(tableInfo2) @@ -177,7 +177,7 @@ func TestRollback(t *testing.T) { testKit.MustExec("rollback") is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo := tbl.Meta() h := dom.StatsHandle() @@ -211,7 +211,7 @@ func TestMultiSession(t *testing.T) { testKit2.MustExec("delete from test.t1 limit 1") } is := dom.InfoSchema() - tbl1, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t1")) + tbl1, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t1")) require.NoError(t, err) tableInfo1 := tbl1.Meta() h := dom.StatsHandle() @@ -254,7 +254,7 @@ func TestTxnWithFailure(t *testing.T) { testKit.MustExec("create table t1 (c1 int primary key, c2 int)") is := dom.InfoSchema() - tbl1, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t1")) + tbl1, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t1")) require.NoError(t, err) tableInfo1 := tbl1.Meta() h := dom.StatsHandle() @@ -307,7 +307,7 @@ func TestUpdatePartition(t *testing.T) { testKit.MustExec(createTable) do := dom is := do.InfoSchema() - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo := tbl.Meta() h := do.StatsHandle() @@ -373,7 +373,7 @@ func TestAutoUpdate(t *testing.T) { do := dom is := do.InfoSchema() - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo := tbl.Meta() h := do.StatsHandle() @@ -440,7 +440,7 @@ func TestAutoUpdate(t *testing.T) { _, err = testKit.Exec("create index idx on t(a)") require.NoError(t, err) is = do.InfoSchema() - tbl, err = is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err = is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo = tbl.Meta() require.Eventually(t, func() bool { @@ -478,7 +478,7 @@ func TestAutoUpdatePartition(t *testing.T) { do := dom is := do.InfoSchema() - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo := tbl.Meta() pi := tableInfo.GetPartitionInfo() @@ -594,7 +594,7 @@ func TestOutOfOrderUpdate(t *testing.T) { do := dom is := do.InfoSchema() - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo := tbl.Meta() h := do.StatsHandle() @@ -812,7 +812,7 @@ func TestAutoUpdatePartitionInDynamicOnlyMode(t *testing.T) { }() require.NoError(t, h.Update(context.Background(), is)) - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tableInfo := tbl.Meta() pi := tableInfo.GetPartitionInfo() @@ -1081,7 +1081,7 @@ func TestStatsLockUnlockForAutoAnalyze(t *testing.T) { require.NoError(t, h.Update(context.Background(), is)) require.True(t, h.HandleAutoAnalyze()) - tbl, err := dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.Nil(t, err) tblStats := h.GetTableStats(tbl.Meta()) @@ -1123,7 +1123,7 @@ func TestStatsLockForDelta(t *testing.T) { testKit.MustExec("create table t2 (c1 int, c2 int)") is := dom.InfoSchema() - tbl1, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t1")) + tbl1, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t1")) require.NoError(t, err) tableInfo1 := tbl1.Meta() h := dom.StatsHandle() @@ -1149,7 +1149,7 @@ func TestStatsLockForDelta(t *testing.T) { stats1 := h.GetTableStats(tableInfo1) require.Equal(t, stats1.RealtimeCount, int64(0)) - tbl2, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t2")) + tbl2, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t2")) require.NoError(t, err) tableInfo2 := tbl2.Meta() stats2 := h.GetTableStats(tableInfo2) @@ -1189,10 +1189,10 @@ func TestFillMissingStatsMeta(t *testing.T) { tk.MustQuery("select * from mysql.stats_meta").Check(testkit.Rows()) is := dom.InfoSchema() - tbl1, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t1")) + tbl1, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t1")) require.NoError(t, err) tbl1ID := tbl1.Meta().ID - tbl2, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t2")) + tbl2, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t2")) require.NoError(t, err) tbl2Info := tbl2.Meta() tbl2ID := tbl2Info.ID @@ -1247,7 +1247,7 @@ func TestNotDumpSysTable(t *testing.T) { tk.MustExec("delete from mysql.stats_meta") require.NoError(t, h.DumpStatsDeltaToKV(true)) is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), model.NewCIStr("mysql"), model.NewCIStr("stats_meta")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("mysql"), ast.NewCIStr("stats_meta")) require.NoError(t, err) tblID := tbl.Meta().ID tk.MustQuery(fmt.Sprintf("select * from mysql.stats_meta where table_id = %v", tblID)).Check(testkit.Rows()) @@ -1280,7 +1280,7 @@ func TestAutoAnalyzePartitionTableAfterAddingIndex(t *testing.T) { tk.MustExec("analyze table t") require.False(t, h.HandleAutoAnalyze()) tk.MustExec("alter table t add index idx(a)") - tbl, err := dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tblInfo := tbl.Meta() idxInfo := tblInfo.Indices[0] diff --git a/pkg/statistics/handle/usage/BUILD.bazel b/pkg/statistics/handle/usage/BUILD.bazel index 9a02fa38858d7..6738d529aab9a 100644 --- a/pkg/statistics/handle/usage/BUILD.bazel +++ b/pkg/statistics/handle/usage/BUILD.bazel @@ -42,7 +42,7 @@ go_test( shard_count = 10, deps = [ "//pkg/meta/model", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/statistics/handle/usage/indexusage", "//pkg/testkit", "@com_github_stretchr_testify//require", diff --git a/pkg/statistics/handle/usage/index_usage_integration_test.go b/pkg/statistics/handle/usage/index_usage_integration_test.go index d9fb767f8fb4a..9300b18ba6cff 100644 --- a/pkg/statistics/handle/usage/index_usage_integration_test.go +++ b/pkg/statistics/handle/usage/index_usage_integration_test.go @@ -20,7 +20,7 @@ import ( "testing" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/statistics/handle/usage/indexusage" "github.com/pingcap/tidb/pkg/testkit" "github.com/stretchr/testify/require" @@ -52,7 +52,7 @@ func TestGCIndexUsage(t *testing.T) { c := dom.StatsHandle().NewSessionIndexUsageCollector() is := tk.Session().GetDomainInfoSchema() - db, ok := is.SchemaByName(pmodel.NewCIStr("test")) + db, ok := is.SchemaByName(ast.NewCIStr("test")) require.True(t, ok) tblInfos, err := is.SchemaTableInfos(context.Background(), db.Name) diff --git a/pkg/statistics/handle/usage/predicate_column_test.go b/pkg/statistics/handle/usage/predicate_column_test.go index 6f3fc01cecd76..3ed7b144166b4 100644 --- a/pkg/statistics/handle/usage/predicate_column_test.go +++ b/pkg/statistics/handle/usage/predicate_column_test.go @@ -18,7 +18,7 @@ import ( "context" "testing" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/testkit" "github.com/stretchr/testify/require" ) @@ -47,7 +47,7 @@ func TestCleanupPredicateColumns(t *testing.T) { tk.MustExec("alter table t drop column b") // Get table ID. is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) columns, err := h.GetPredicateColumns(tbl.Meta().ID) require.NoError(t, err) diff --git a/pkg/statistics/histogram_test.go b/pkg/statistics/histogram_test.go index cf26b1cfd97d1..e7e2999f66be7 100644 --- a/pkg/statistics/histogram_test.go +++ b/pkg/statistics/histogram_test.go @@ -20,7 +20,7 @@ import ( "time" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/codec" @@ -539,7 +539,7 @@ func TestMergeBucketNDV(t *testing.T) { func TestIndexQueryBytes(t *testing.T) { ctx := mock.NewContext() sc := ctx.GetSessionVars().StmtCtx - idx := &Index{Info: &model.IndexInfo{Columns: []*model.IndexColumn{{Name: pmodel.NewCIStr("a"), Offset: 0}}}} + idx := &Index{Info: &model.IndexInfo{Columns: []*model.IndexColumn{{Name: ast.NewCIStr("a"), Offset: 0}}}} idx.Histogram = *NewHistogram(0, 15, 0, 0, types.NewFieldType(mysql.TypeBlob), 0, 0) low, err1 := codec.EncodeKey(sc.TimeZone(), nil, types.NewBytesDatum([]byte("0"))) require.NoError(t, err1) diff --git a/pkg/statistics/integration_test.go b/pkg/statistics/integration_test.go index 45bbe55950e6d..a6b5d2a3f4929 100644 --- a/pkg/statistics/integration_test.go +++ b/pkg/statistics/integration_test.go @@ -25,7 +25,7 @@ import ( "github.com/pingcap/failpoint" metamodel "github.com/pingcap/tidb/pkg/meta/model" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/statistics" statstestutil "github.com/pingcap/tidb/pkg/statistics/handle/ddl/testutil" "github.com/pingcap/tidb/pkg/testkit" @@ -50,7 +50,7 @@ func TestChangeVerTo2Behavior(t *testing.T) { analyzehelper.TriggerPredicateColumnsCollection(t, tk, store, "t", "a", "b") tk.MustExec("analyze table t") is := dom.InfoSchema() - tblT, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tblT, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) h := dom.StatsHandle() require.NoError(t, h.Update(context.Background(), is)) @@ -140,7 +140,7 @@ func TestChangeVerTo2BehaviorWithPersistedOptions(t *testing.T) { analyzehelper.TriggerPredicateColumnsCollection(t, tk, store, "t", "a", "b") tk.MustExec("analyze table t") is := dom.InfoSchema() - tblT, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tblT, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) h := dom.StatsHandle() require.NoError(t, h.Update(context.Background(), is)) @@ -267,7 +267,7 @@ func TestNULLOnFullSampling(t *testing.T) { ) tk.MustExec("analyze table t with 2 topn") is := dom.InfoSchema() - tblT, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tblT, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) h := dom.StatsHandle() require.NoError(t, h.Update(context.Background(), is)) @@ -462,7 +462,7 @@ func TestColumnStatsLazyLoad(t *testing.T) { analyzehelper.TriggerPredicateColumnsCollection(t, tk, store, "t", "a", "b") tk.MustExec("analyze table t") is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tblInfo := tbl.Meta() c1 := tblInfo.Columns[0] @@ -485,7 +485,7 @@ func TestUpdateNotLoadIndexFMSketch(t *testing.T) { require.NoError(t, err) tk.MustExec("analyze table t") is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tblInfo := tbl.Meta() idxInfo := tblInfo.Indices[0] @@ -528,7 +528,7 @@ func TestTableLastAnalyzeVersion(t *testing.T) { require.NoError(t, err) is := dom.InfoSchema() require.NoError(t, h.Update(context.Background(), is)) - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) statsTbl, found := h.Get(tbl.Meta().ID) require.True(t, found) @@ -537,7 +537,7 @@ func TestTableLastAnalyzeVersion(t *testing.T) { // Only alter table should not set the last_analyze_version tk.MustExec("alter table t add column b int default 0") is = dom.InfoSchema() - tbl, err = is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err = is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) err = statstestutil.HandleNextDDLEventWithTxn(h) require.NoError(t, err) @@ -547,7 +547,7 @@ func TestTableLastAnalyzeVersion(t *testing.T) { require.Equal(t, uint64(0), statsTbl.LastAnalyzeVersion) tk.MustExec("alter table t add index idx(a)") is = dom.InfoSchema() - tbl, err = is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err = is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) e := <-h.DDLEventCh() require.Equal(t, metamodel.ActionAddIndex, e.GetType()) require.Equal(t, 0, len(h.DDLEventCh())) diff --git a/pkg/store/helper/BUILD.bazel b/pkg/store/helper/BUILD.bazel index aa4ff8b708b26..95a09dbc89eb4 100644 --- a/pkg/store/helper/BUILD.bazel +++ b/pkg/store/helper/BUILD.bazel @@ -39,7 +39,7 @@ go_test( deps = [ "//pkg/infoschema/context", "//pkg/meta/model", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/store/mockstore", "//pkg/tablecodec", "//pkg/testkit/testsetup", diff --git a/pkg/store/helper/helper_test.go b/pkg/store/helper/helper_test.go index a22e04fc810fa..354ed3c50ba95 100644 --- a/pkg/store/helper/helper_test.go +++ b/pkg/store/helper/helper_test.go @@ -30,7 +30,7 @@ import ( "github.com/pingcap/log" infoschema "github.com/pingcap/tidb/pkg/infoschema/context" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/store/helper" "github.com/pingcap/tidb/pkg/store/mockstore" "github.com/pingcap/tidb/pkg/tablecodec" @@ -67,7 +67,7 @@ func TestHotRegion(t *testing.T) { require.Equal(t, expected, regionMetric) dbInfo := &model.DBInfo{ - Name: pmodel.NewCIStr("test"), + Name: ast.NewCIStr("test"), } require.NoError(t, err) @@ -215,7 +215,7 @@ func mockHotRegionResponse(w http.ResponseWriter, _ *http.Request) { } func getMockRegionsTableInfoSchema() []*model.DBInfo { - dbInfo := &model.DBInfo{Name: pmodel.NewCIStr("test")} + dbInfo := &model.DBInfo{Name: ast.NewCIStr("test")} dbInfo.Deprecated.Tables = []*model.TableInfo{ { ID: 41, diff --git a/pkg/store/mockstore/mockcopr/BUILD.bazel b/pkg/store/mockstore/mockcopr/BUILD.bazel index b95b7c77e29d8..56811e932febf 100644 --- a/pkg/store/mockstore/mockcopr/BUILD.bazel +++ b/pkg/store/mockstore/mockcopr/BUILD.bazel @@ -61,7 +61,7 @@ go_test( deps = [ "//pkg/domain", "//pkg/kv", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/session", "//pkg/store/mockstore/mockstorage", "//pkg/tablecodec", diff --git a/pkg/store/mockstore/mockcopr/executor_test.go b/pkg/store/mockstore/mockcopr/executor_test.go index 211e67d0fd024..08d4b820c0a05 100644 --- a/pkg/store/mockstore/mockcopr/executor_test.go +++ b/pkg/store/mockstore/mockcopr/executor_test.go @@ -23,7 +23,7 @@ import ( "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/kv" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/session" "github.com/pingcap/tidb/pkg/store/mockstore/mockcopr" "github.com/pingcap/tidb/pkg/store/mockstore/mockstorage" @@ -68,7 +68,7 @@ func TestResolvedLargeTxnLocks(t *testing.T) { tk.MustExec("create table t (id int primary key, val int)") dom = domain.GetDomain(tk.Session()) schema := dom.InfoSchema() - tbl, err := schema.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := schema.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tk.MustExec("insert into t values (1, 1)") diff --git a/pkg/table/BUILD.bazel b/pkg/table/BUILD.bazel index 12383de4ef8c6..805974db08c95 100644 --- a/pkg/table/BUILD.bazel +++ b/pkg/table/BUILD.bazel @@ -22,7 +22,6 @@ go_library( "//pkg/parser", "//pkg/parser/ast", "//pkg/parser/charset", - "//pkg/parser/model", "//pkg/parser/mysql", "//pkg/parser/types", "//pkg/sessionctx", @@ -62,7 +61,6 @@ go_test( "//pkg/meta/model", "//pkg/parser/ast", "//pkg/parser/charset", - "//pkg/parser/model", "//pkg/parser/mysql", "//pkg/parser/terror", "//pkg/sessionctx/stmtctx", diff --git a/pkg/table/column_test.go b/pkg/table/column_test.go index 90bc46eda80ac..37505f2e4e18d 100644 --- a/pkg/table/column_test.go +++ b/pkg/table/column_test.go @@ -24,7 +24,6 @@ import ( "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/charset" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" "github.com/pingcap/tidb/pkg/types" @@ -522,7 +521,7 @@ func TestGetDefaultValue(t *testing.T) { func newCol(name string) *Column { return ToColumn(&model.ColumnInfo{ - Name: pmodel.NewCIStr(name), + Name: ast.NewCIStr(name), State: model.StatePublic, }) } diff --git a/pkg/table/constraint.go b/pkg/table/constraint.go index 958b699bf7f6d..15188c3521dc8 100644 --- a/pkg/table/constraint.go +++ b/pkg/table/constraint.go @@ -20,7 +20,6 @@ import ( "github.com/pingcap/tidb/pkg/expression/exprstatic" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/ast" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/util/dbterror" "github.com/pingcap/tidb/pkg/util/logutil" @@ -174,7 +173,7 @@ func (checker *checkConstraintChecker) Leave(in ast.Node) (out ast.Node, ok bool } // ContainsAutoIncrementCol checks if there is auto-increment col in given cols -func ContainsAutoIncrementCol(cols []pmodel.CIStr, tblInfo *model.TableInfo) bool { +func ContainsAutoIncrementCol(cols []ast.CIStr, tblInfo *model.TableInfo) bool { if autoIncCol := tblInfo.GetAutoIncrementColInfo(); autoIncCol != nil { for _, col := range cols { if col.L == autoIncCol.Name.L { @@ -186,7 +185,7 @@ func ContainsAutoIncrementCol(cols []pmodel.CIStr, tblInfo *model.TableInfo) boo } // HasForeignKeyRefAction checks if there is foreign key with referential action in check constraints -func HasForeignKeyRefAction(fkInfos []*model.FKInfo, constraints []*ast.Constraint, checkConstr *ast.Constraint, dependedCols []pmodel.CIStr) error { +func HasForeignKeyRefAction(fkInfos []*model.FKInfo, constraints []*ast.Constraint, checkConstr *ast.Constraint, dependedCols []ast.CIStr) error { if fkInfos != nil { return checkForeignKeyRefActionByFKInfo(fkInfos, checkConstr, dependedCols) } @@ -195,8 +194,8 @@ func HasForeignKeyRefAction(fkInfos []*model.FKInfo, constraints []*ast.Constrai continue } refCol := cons.Refer - if refCol.OnDelete.ReferOpt != pmodel.ReferOptionNoOption || refCol.OnUpdate.ReferOpt != pmodel.ReferOptionNoOption { - var fkCols []pmodel.CIStr + if refCol.OnDelete.ReferOpt != ast.ReferOptionNoOption || refCol.OnUpdate.ReferOpt != ast.ReferOptionNoOption { + var fkCols []ast.CIStr for _, key := range cons.Keys { fkCols = append(fkCols, key.Column.Name) } @@ -210,7 +209,7 @@ func HasForeignKeyRefAction(fkInfos []*model.FKInfo, constraints []*ast.Constrai return nil } -func checkForeignKeyRefActionByFKInfo(fkInfos []*model.FKInfo, checkConstr *ast.Constraint, dependedCols []pmodel.CIStr) error { +func checkForeignKeyRefActionByFKInfo(fkInfos []*model.FKInfo, checkConstr *ast.Constraint, dependedCols []ast.CIStr) error { for _, fkInfo := range fkInfos { if fkInfo.OnDelete != 0 || fkInfo.OnUpdate != 0 { for _, col := range dependedCols { @@ -223,7 +222,7 @@ func checkForeignKeyRefActionByFKInfo(fkInfos []*model.FKInfo, checkConstr *ast. return nil } -func hasSpecifiedCol(cols []pmodel.CIStr, col pmodel.CIStr) bool { +func hasSpecifiedCol(cols []ast.CIStr, col ast.CIStr) bool { for _, c := range cols { if c.L == col.L { return true diff --git a/pkg/table/table.go b/pkg/table/table.go index bffc4d898d8f6..73fc9f9260fdc 100644 --- a/pkg/table/table.go +++ b/pkg/table/table.go @@ -28,7 +28,7 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta/autoid" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/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/table/tblctx" @@ -510,7 +510,7 @@ type PartitionedTable interface { GetPartitionIdxByRow(expression.EvalContext, []types.Datum) (int, error) GetAllPartitionIDs() []int64 GetPartitionColumnIDs() []int64 - GetPartitionColumnNames() []pmodel.CIStr + GetPartitionColumnNames() []ast.CIStr CheckForExchangePartition(ctx expression.EvalContext, pi *model.PartitionInfo, r []types.Datum, partID, ntID int64) error } diff --git a/pkg/table/tables/BUILD.bazel b/pkg/table/tables/BUILD.bazel index d6519927d68bc..2f3b6a1347fba 100644 --- a/pkg/table/tables/BUILD.bazel +++ b/pkg/table/tables/BUILD.bazel @@ -25,7 +25,6 @@ go_library( "//pkg/metrics", "//pkg/parser", "//pkg/parser/ast", - "//pkg/parser/model", "//pkg/parser/mysql", "//pkg/parser/terror", "//pkg/sessionctx", @@ -91,7 +90,6 @@ go_test( "//pkg/parser", "//pkg/parser/ast", "//pkg/parser/auth", - "//pkg/parser/model", "//pkg/parser/mysql", "//pkg/session", "//pkg/session/types", diff --git a/pkg/table/tables/bench_test.go b/pkg/table/tables/bench_test.go index c389a947778c5..0e16594c8ca26 100644 --- a/pkg/table/tables/bench_test.go +++ b/pkg/table/tables/bench_test.go @@ -20,7 +20,7 @@ import ( "github.com/pingcap/log" "github.com/pingcap/tidb/pkg/kv" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/sessiontxn" "github.com/pingcap/tidb/pkg/table" @@ -44,7 +44,7 @@ func BenchmarkAddRecordInPipelinedDML(b *testing.B) { "CREATE TABLE IF NOT EXISTS test.t (a int primary key auto_increment, b varchar(255))", ) require.NoError(b, err) - tb, err := dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tb, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(b, err) variable.EnableMDL.Store(true) @@ -98,7 +98,7 @@ func BenchmarkRemoveRecordInPipelinedDML(b *testing.B) { "CREATE TABLE IF NOT EXISTS test.t (a int primary key clustered, b varchar(255))", ) require.NoError(b, err) - tb, err := dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tb, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(b, err) variable.EnableMDL.Store(true) @@ -159,7 +159,7 @@ func BenchmarkUpdateRecordInPipelinedDML(b *testing.B) { "CREATE TABLE IF NOT EXISTS test.t (a int primary key clustered, b varchar(255))", ) require.NoError(b, err) - tb, err := dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tb, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(b, err) // Pre-create data to be inserted and then updated diff --git a/pkg/table/tables/cache_test.go b/pkg/table/tables/cache_test.go index 17eb32a96eb0e..c5b9c17d77ec3 100644 --- a/pkg/table/tables/cache_test.go +++ b/pkg/table/tables/cache_test.go @@ -23,8 +23,8 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/metrics" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/auth" - "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/table/tables" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/util/stmtsummary" @@ -339,7 +339,7 @@ func TestRenewLease(t *testing.T) { se := tk.Session() tk.MustExec("create table cache_renew_t (id int)") tk.MustExec("alter table cache_renew_t cache") - tbl, err := se.GetInfoSchema().(infoschema.InfoSchema).TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("cache_renew_t")) + tbl, err := se.GetInfoSchema().(infoschema.InfoSchema).TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("cache_renew_t")) require.NoError(t, err) var i int tk.MustExec("select * from cache_renew_t") @@ -536,7 +536,7 @@ func TestRenewLeaseABAFailPoint(t *testing.T) { // Mock reading from another TiDB instance: write lock -> read lock is := tk2.Session().GetInfoSchema().(infoschema.InfoSchema) - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t_lease")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t_lease")) require.NoError(t, err) lease := oracle.GoTimeToTS(time.Now().Add(20 * time.Second)) // A big enough future time tk2.MustExec("update mysql.table_cache_meta set lock_type = 'READ', lease = ? where tid = ?", lease, tbl.Meta().ID) diff --git a/pkg/table/tables/index_test.go b/pkg/table/tables/index_test.go index 047f664d8912d..7b405a0efbe96 100644 --- a/pkg/table/tables/index_test.go +++ b/pkg/table/tables/index_test.go @@ -28,7 +28,6 @@ import ( "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser" "github.com/pingcap/tidb/pkg/parser/ast" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/table/tables" @@ -324,7 +323,7 @@ func TestTableOperationsInDDLDropIndexWriteOnly(t *testing.T) { for { time.Sleep(20 * time.Millisecond) // wait the DDL state change to `StateWriteOnly` - tblInfo, err := do.InfoSchema().TableInfoByName(pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tblInfo, err := do.InfoSchema().TableInfoByName(ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) if state := tblInfo.Indices[0].State; state != model.StatePublic { require.Equal(t, model.StateWriteOnly, state) @@ -347,7 +346,7 @@ func TestTableOperationsInDDLDropIndexWriteOnly(t *testing.T) { // update some rows: 1 in storage, 1 in memory buffer. tk2.MustExec("update t set a = a + 10 where a in (2, 6)") // should be tested in `StateWriteOnly` state. - tblInfo, err := tk2.Session().GetInfoSchema().TableInfoByName(pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tblInfo, err := tk2.Session().GetInfoSchema().TableInfoByName(ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) require.Equal(t, model.StateWriteOnly, tblInfo.Indices[0].State) // commit should success without any assertion fail. diff --git a/pkg/table/tables/mutation_checker_test.go b/pkg/table/tables/mutation_checker_test.go index b527c563aa23a..525360c75ad7f 100644 --- a/pkg/table/tables/mutation_checker_test.go +++ b/pkg/table/tables/mutation_checker_test.go @@ -21,7 +21,7 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/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/table" @@ -76,12 +76,12 @@ func TestCompareIndexData(t *testing.T) { cols := make([]*table.Column, 0) indexCols := make([]*model.IndexColumn, 0) for i, ft := range data.fts { - cols = append(cols, &table.Column{ColumnInfo: &model.ColumnInfo{Name: pmodel.NewCIStr(fmt.Sprintf("c%d", i)), FieldType: *ft}}) + cols = append(cols, &table.Column{ColumnInfo: &model.ColumnInfo{Name: ast.NewCIStr(fmt.Sprintf("c%d", i)), FieldType: *ft}}) indexCols = append(indexCols, &model.IndexColumn{Offset: i, Length: data.indexLength[i]}) } - indexInfo := &model.IndexInfo{Name: pmodel.NewCIStr("i0"), Columns: indexCols} + indexInfo := &model.IndexInfo{Name: ast.NewCIStr("i0"), Columns: indexCols} - err := compareIndexData(tc, cols, data.indexData, data.inputData, indexInfo, &model.TableInfo{Name: pmodel.NewCIStr("t")}, nil) + err := compareIndexData(tc, cols, data.indexData, data.inputData, indexInfo, &model.TableInfo{Name: ast.NewCIStr("t")}, nil) require.Equal(t, data.correct, err == nil, "case id = %v", caseID) } } @@ -257,7 +257,7 @@ func TestCheckIndexKeysAndCheckHandleConsistency(t *testing.T) { tc = tc.WithLocation(lc) tableInfo := model.TableInfo{ ID: 1, - Name: pmodel.NewCIStr("t"), + Name: ast.NewCIStr("t"), Columns: columnInfos, Indices: indexInfos, PKIsHandle: false, diff --git a/pkg/table/tables/partition.go b/pkg/table/tables/partition.go index b19df8706fa4e..0f12063575c9f 100644 --- a/pkg/table/tables/partition.go +++ b/pkg/table/tables/partition.go @@ -35,7 +35,6 @@ import ( "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser" "github.com/pingcap/tidb/pkg/parser/ast" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/table" @@ -293,24 +292,24 @@ func NewPartitionExprBuildCtx() expression.BuildContext { ) } -func newPartitionExpr(tblInfo *model.TableInfo, tp pmodel.PartitionType, expr string, partCols []pmodel.CIStr, defs []model.PartitionDefinition) (*PartitionExpr, error) { +func newPartitionExpr(tblInfo *model.TableInfo, tp ast.PartitionType, expr string, partCols []ast.CIStr, defs []model.PartitionDefinition) (*PartitionExpr, error) { ctx := NewPartitionExprBuildCtx() - dbName := pmodel.NewCIStr(ctx.GetEvalCtx().CurrentDB()) + dbName := ast.NewCIStr(ctx.GetEvalCtx().CurrentDB()) columns, names, err := expression.ColumnInfos2ColumnsAndNames(ctx, dbName, tblInfo.Name, tblInfo.Cols(), tblInfo) if err != nil { return nil, err } switch tp { - case pmodel.PartitionTypeNone: + case ast.PartitionTypeNone: // Nothing to do return nil, nil - case pmodel.PartitionTypeRange: + case ast.PartitionTypeRange: return generateRangePartitionExpr(ctx, expr, partCols, defs, columns, names) - case pmodel.PartitionTypeHash: + case ast.PartitionTypeHash: return generateHashPartitionExpr(ctx, expr, columns, names) - case pmodel.PartitionTypeKey: + case ast.PartitionTypeKey: return generateKeyPartitionExpr(ctx, expr, partCols, columns, names) - case pmodel.PartitionTypeList: + case ast.PartitionTypeList: return generateListPartitionExpr(ctx, tblInfo, expr, partCols, defs, columns, names) } panic("cannot reach here") @@ -715,7 +714,7 @@ func fixOldVersionPartitionInfo(sctx expression.BuildContext, str string) (int64 return ret, true } -func rangePartitionExprStrings(cols []pmodel.CIStr, expr string) []string { +func rangePartitionExprStrings(cols []ast.CIStr, expr string) []string { var s []string if len(cols) > 0 { s = make([]string, 0, len(cols)) @@ -728,7 +727,7 @@ func rangePartitionExprStrings(cols []pmodel.CIStr, expr string) []string { return s } -func generateKeyPartitionExpr(ctx expression.BuildContext, expr string, partCols []pmodel.CIStr, +func generateKeyPartitionExpr(ctx expression.BuildContext, expr string, partCols []ast.CIStr, columns []*expression.Column, names types.NameSlice) (*PartitionExpr, error) { ret := &PartitionExpr{ ForKeyPruning: &ForKeyPruning{}, @@ -743,7 +742,7 @@ func generateKeyPartitionExpr(ctx expression.BuildContext, expr string, partCols return ret, nil } -func generateRangePartitionExpr(ctx expression.BuildContext, expr string, partCols []pmodel.CIStr, +func generateRangePartitionExpr(ctx expression.BuildContext, expr string, partCols []ast.CIStr, defs []model.PartitionDefinition, columns []*expression.Column, names types.NameSlice) (*PartitionExpr, error) { // The caller should assure partition info is not nil. p := parser.New() @@ -833,7 +832,7 @@ func findIdxByColUniqueID(cols []*expression.Column, col *expression.Column) int return -1 } -func extractPartitionExprColumns(ctx expression.BuildContext, expr string, partCols []pmodel.CIStr, columns []*expression.Column, names types.NameSlice) (expression.Expression, []*expression.Column, []int, error) { +func extractPartitionExprColumns(ctx expression.BuildContext, expr string, partCols []ast.CIStr, columns []*expression.Column, names types.NameSlice) (expression.Expression, []*expression.Column, []int, error) { var cols []*expression.Column var partExpr expression.Expression if len(partCols) == 0 { @@ -867,7 +866,7 @@ func extractPartitionExprColumns(ctx expression.BuildContext, expr string, partC return partExpr, deDupCols, offset, nil } -func generateListPartitionExpr(ctx expression.BuildContext, tblInfo *model.TableInfo, expr string, partCols []pmodel.CIStr, +func generateListPartitionExpr(ctx expression.BuildContext, tblInfo *model.TableInfo, expr string, partCols []ast.CIStr, defs []model.PartitionDefinition, columns []*expression.Column, names types.NameSlice) (*PartitionExpr, error) { // The caller should assure partition info is not nil. partExpr, exprCols, offset, err := extractPartitionExprColumns(ctx, expr, partCols, columns, names) @@ -944,7 +943,7 @@ func (lp *ForListPruning) buildListPruner(ctx expression.BuildContext, exprStr s } func (lp *ForListPruning) buildListColumnsPruner(ctx expression.BuildContext, - tblInfo *model.TableInfo, partCols []pmodel.CIStr, defs []model.PartitionDefinition, + tblInfo *model.TableInfo, partCols []ast.CIStr, defs []model.PartitionDefinition, columns []*expression.Column, names types.NameSlice) error { schema := expression.NewSchema(columns...) p := parser.New() @@ -1377,13 +1376,13 @@ func (t *partitionedTable) GetPartitionColumnIDs() []int64 { return colIDs } -func (t *partitionedTable) GetPartitionColumnNames() []pmodel.CIStr { +func (t *partitionedTable) GetPartitionColumnNames() []ast.CIStr { pi := t.Meta().Partition if len(pi.Columns) > 0 { return pi.Columns } colIDs := t.GetPartitionColumnIDs() - colNames := make([]pmodel.CIStr, 0, len(colIDs)) + colNames := make([]ast.CIStr, 0, len(colIDs)) for _, colID := range colIDs { for _, col := range t.Cols() { if col.ID == colID { @@ -1412,11 +1411,11 @@ func (t *partitionedTable) CheckForExchangePartition(ctx expression.EvalContext, } // locatePartitionCommon returns the partition idx of the input record. -func (t *partitionedTable) locatePartitionCommon(ctx expression.EvalContext, tp pmodel.PartitionType, partitionExpr *PartitionExpr, num uint64, columnsPartitioned bool, r []types.Datum) (int, error) { +func (t *partitionedTable) locatePartitionCommon(ctx expression.EvalContext, tp ast.PartitionType, partitionExpr *PartitionExpr, num uint64, columnsPartitioned bool, r []types.Datum) (int, error) { var err error var idx int switch tp { - case pmodel.PartitionTypeRange: + case ast.PartitionTypeRange: if columnsPartitioned { idx, err = t.locateRangeColumnPartition(ctx, partitionExpr, r) } else { @@ -1435,18 +1434,18 @@ func (t *partitionedTable) locatePartitionCommon(ctx expression.EvalContext, tp return idx, table.ErrNoPartitionForGivenValue.GenWithStackByArgs(fmt.Sprintf("matching a partition being dropped, '%s'", pi.Definitions[idx].Name.String())) } } - case pmodel.PartitionTypeHash: + case ast.PartitionTypeHash: // Note that only LIST and RANGE supports REORGANIZE PARTITION idx, err = t.locateHashPartition(ctx, partitionExpr, num, r) - case pmodel.PartitionTypeKey: + case ast.PartitionTypeKey: idx, err = partitionExpr.LocateKeyPartition(num, r) - case pmodel.PartitionTypeList: + case ast.PartitionTypeList: idx, err = partitionExpr.locateListPartition(ctx, r) pi := t.Meta().Partition if idx != pi.GetOverlappingDroppingPartitionIdx(idx) { return idx, table.ErrNoPartitionForGivenValue.GenWithStackByArgs(fmt.Sprintf("matching a partition being dropped, '%s'", pi.Definitions[idx].Name.String())) } - case pmodel.PartitionTypeNone: + case ast.PartitionTypeNone: idx = 0 } if err != nil { diff --git a/pkg/table/tables/tables_test.go b/pkg/table/tables/tables_test.go index 4f1ac053cfe17..2c60e0e804333 100644 --- a/pkg/table/tables/tables_test.go +++ b/pkg/table/tables/tables_test.go @@ -28,8 +28,8 @@ import ( "github.com/pingcap/tidb/pkg/kv" "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/parser/auth" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/session" "github.com/pingcap/tidb/pkg/sessionctx" @@ -81,7 +81,7 @@ func TestBasic(t *testing.T) { require.Nil(t, sessiontxn.NewTxn(context.Background(), tk.Session())) txn, err := tk.Session().Txn(true) require.NoError(t, err) - tb, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tb, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) require.Greater(t, tb.Meta().ID, int64(0)) require.Equal(t, "t", tb.Meta().Name.L) @@ -181,7 +181,7 @@ func TestTypes(t *testing.T) { tk := testkit.NewTestKit(t, store) _, err := tk.Session().Execute(context.Background(), "CREATE TABLE test.t (c1 tinyint, c2 smallint, c3 int, c4 bigint, c5 text, c6 blob, c7 varchar(64), c8 time, c9 timestamp null default CURRENT_TIMESTAMP, c10 decimal(10,1))") require.NoError(t, err) - _, err = dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + _, err = dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) _, err = tk.Session().Execute(ctx, "insert test.t values (1, 2, 3, 4, '5', '6', '7', '10:10:10', null, 1.4)") require.NoError(t, err) @@ -235,7 +235,7 @@ func TestUniqueIndexMultipleNullEntries(t *testing.T) { require.NoError(t, err) _, err = tk.Session().Execute(ctx, "CREATE TABLE test.t (a int primary key auto_increment, b varchar(255) unique)") require.NoError(t, err) - tb, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tb, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) require.Greater(t, tb.Meta().ID, int64(0)) require.Equal(t, "t", tb.Meta().Name.L) @@ -315,7 +315,7 @@ func TestUnsignedPK(t *testing.T) { require.NoError(t, err) _, err = tk.Session().Execute(context.Background(), "CREATE TABLE test.tPK (a bigint unsigned primary key, b varchar(255))") require.NoError(t, err) - tb, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("tPK")) + tb, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("tPK")) require.NoError(t, err) require.Nil(t, sessiontxn.NewTxn(context.Background(), tk.Session())) txn, err := tk.Session().Txn(true) @@ -343,7 +343,7 @@ func TestIterRecords(t *testing.T) { require.Nil(t, sessiontxn.NewTxn(context.Background(), tk.Session())) txn, err := tk.Session().Txn(true) require.NoError(t, err) - tb, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("tIter")) + tb, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("tIter")) require.NoError(t, err) totalCount := 0 err = tables.IterRecords(tb, tk.Session(), tb.Cols(), func(_ kv.Handle, rec []types.Datum, cols []*table.Column) (bool, error) { @@ -364,7 +364,7 @@ func TestTableFromMeta(t *testing.T) { require.Nil(t, sessiontxn.NewTxn(context.Background(), tk.Session())) _, err := tk.Session().Txn(true) require.NoError(t, err) - tb, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("meta")) + tb, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("meta")) require.NoError(t, err) tbInfo := tb.Meta().Clone() @@ -386,7 +386,7 @@ func TestTableFromMeta(t *testing.T) { require.Error(t, err) tk.MustExec(`create table t_mock (id int) partition by range (id) (partition p0 values less than maxvalue)`) - tb, err = dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t_mock")) + tb, err = dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t_mock")) require.NoError(t, err) tt := table.MockTableFromMeta(tb.Meta()) _, ok := tt.(table.PartitionedTable) @@ -395,7 +395,7 @@ func TestTableFromMeta(t *testing.T) { require.Equal(t, table.NormalTable, tt.Type()) tk.MustExec("create table t_meta (a int) shard_row_id_bits = 15") - tb, err = domain.GetDomain(tk.Session()).InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t_meta")) + tb, err = domain.GetDomain(tk.Session()).InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t_meta")) require.NoError(t, err) _, err = tables.AllocHandle(context.Background(), tk.Session().GetTableCtx(), tb) require.NoError(t, err) @@ -416,7 +416,7 @@ func TestHiddenColumn(t *testing.T) { tk.MustExec("USE test_hidden;") tk.MustExec("CREATE TABLE t (a int primary key, b int as (a+1), c int, d int as (c+1) stored, e int, f tinyint as (a+1));") tk.MustExec("insert into t values (1, default, 3, default, 5, default);") - tb, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test_hidden"), pmodel.NewCIStr("t")) + tb, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test_hidden"), ast.NewCIStr("t")) require.NoError(t, err) colInfo := tb.Meta().Columns // Set column b, d, f to hidden @@ -581,7 +581,7 @@ func TestAddRecordWithCtx(t *testing.T) { require.NoError(t, err) _, err = tk.Session().Execute(context.Background(), "CREATE TABLE test.tRecord (a bigint unsigned primary key, b varchar(255))") require.NoError(t, err) - tb, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("tRecord")) + tb, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("tRecord")) require.NoError(t, err) defer func() { _, err := tk.Session().Execute(context.Background(), "DROP TABLE test.tRecord") @@ -944,7 +944,7 @@ func TestSkipWriteUntouchedIndices(t *testing.T) { tk.MustExec("insert into t values(4, 5, 6)") defer tk.MustExec("rollback") - tbl, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) ctx := tk.Session().GetTableCtx() @@ -1018,7 +1018,7 @@ func TestDupKeyCheckMode(t *testing.T) { require.NoError(t, err) return txn } - tbl, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) ctx := tk.Session().GetTableCtx() getHandleFlags := func(h kv.Handle, memBuffer kv.MemBuffer) kv.KeyFlags { diff --git a/pkg/table/tables/test/partition/BUILD.bazel b/pkg/table/tables/test/partition/BUILD.bazel index e09b6d24b732c..299e44f0d0f40 100644 --- a/pkg/table/tables/test/partition/BUILD.bazel +++ b/pkg/table/tables/test/partition/BUILD.bazel @@ -13,7 +13,7 @@ go_test( "//pkg/domain", "//pkg/kv", "//pkg/meta/model", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/sessiontxn", "//pkg/table", "//pkg/table/tables", diff --git a/pkg/table/tables/test/partition/partition_test.go b/pkg/table/tables/test/partition/partition_test.go index 508ecaaf7accb..ecd768e1a3ff7 100644 --- a/pkg/table/tables/test/partition/partition_test.go +++ b/pkg/table/tables/test/partition/partition_test.go @@ -27,7 +27,7 @@ import ( "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/sessiontxn" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/table/tables" @@ -58,11 +58,11 @@ PARTITION BY RANGE ( id ) ( require.NoError(t, err) _, err = tk.Session().Execute(ctx, createTable1) require.NoError(t, err) - tb, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t1")) + tb, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t1")) require.NoError(t, err) tbInfo := tb.Meta() p0 := tbInfo.Partition.Definitions[0] - require.Equal(t, pmodel.NewCIStr("p0"), p0.Name) + require.Equal(t, ast.NewCIStr("p0"), p0.Name) require.Nil(t, sessiontxn.NewTxn(ctx, tk.Session())) txn, err := tk.Session().Txn(true) require.NoError(t, err) @@ -109,7 +109,7 @@ PARTITION BY RANGE ( id ) ( require.Nil(t, sessiontxn.NewTxn(ctx, tk.Session())) txn, err = tk.Session().Txn(true) require.NoError(t, err) - tb, err = dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t2")) + tb, err = dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t2")) require.NoError(t, err) _, err = tb.AddRecord(tk.Session().GetTableCtx(), txn, types.MakeDatums(22)) require.NoError(t, err) @@ -123,7 +123,7 @@ PARTITION BY RANGE ( id ) ( require.Nil(t, sessiontxn.NewTxn(ctx, tk.Session())) txn, err = tk.Session().Txn(true) require.NoError(t, err) - tb, err = dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t3")) + tb, err = dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t3")) require.NoError(t, err) _, err = tb.AddRecord(tk.Session().GetTableCtx(), txn, types.MakeDatums(11)) require.True(t, table.ErrNoPartitionForGivenValue.Equal(err)) @@ -141,7 +141,7 @@ PARTITION BY RANGE ( id ) ( require.Nil(t, sessiontxn.NewTxn(ctx, tk.Session())) txn, err = tk.Session().Txn(true) require.NoError(t, err) - tb, err = dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t4")) + tb, err = dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t4")) require.NoError(t, err) _, err = tb.AddRecord(tk.Session().GetTableCtx(), txn, types.MakeDatums(1, 11)) require.True(t, table.ErrNoPartitionForGivenValue.Equal(err)) @@ -156,7 +156,7 @@ func TestHashPartitionAddRecord(t *testing.T) { require.NoError(t, err) _, err = tk.Session().Execute(context.Background(), `CREATE TABLE test.t1 (id int(11), index(id)) PARTITION BY HASH (id) partitions 4;`) require.NoError(t, err) - tb, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t1")) + tb, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t1")) require.NoError(t, err) tbInfo := tb.Meta() p0 := tbInfo.Partition.Definitions[0] @@ -193,7 +193,7 @@ func TestHashPartitionAddRecord(t *testing.T) { // Test for partition expression is negative number. _, err = tk.Session().Execute(context.Background(), `CREATE TABLE test.t2 (id int(11), index(id)) PARTITION BY HASH (id) partitions 11;`) require.NoError(t, err) - tb, err = dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t2")) + tb, err = dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t2")) require.NoError(t, err) tbInfo = tb.Meta() for i := 0; i < 11; i++ { @@ -227,7 +227,7 @@ PARTITION BY RANGE ( id ) ( require.NoError(t, err) _, err = tk.Session().Execute(context.Background(), createTable1) require.NoError(t, err) - tb, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t1")) + tb, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t1")) require.NoError(t, err) tbInfo := tb.Meta() ps := tbInfo.GetPartitionInfo() @@ -253,7 +253,7 @@ func TestGeneratePartitionExpr(t *testing.T) { partition p3 values less than maxvalue)`) require.NoError(t, err) - tbl, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t1")) + tbl, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t1")) require.NoError(t, err) type partitionExpr interface { PartitionExpr() *tables.PartitionExpr @@ -359,14 +359,14 @@ func TestIssue31629(t *testing.T) { require.NoError(t, err) tk.MustQuery("show warnings").Check(testkit.Rows()) - tb, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("Issue31629"), pmodel.NewCIStr("t1")) + tb, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("Issue31629"), ast.NewCIStr("t1")) require.NoError(t, err) tbp, ok := tb.(table.PartitionedTable) require.Truef(t, ok, "test %d does not generate a table.PartitionedTable: %s (%T, %+v)", i, createTable, tb, tb) colNames := tbp.GetPartitionColumnNames() - checkNames := []pmodel.CIStr{pmodel.NewCIStr(tt.cols[0])} + checkNames := []ast.CIStr{ast.NewCIStr(tt.cols[0])} for i := 1; i < len(tt.cols); i++ { - checkNames = append(checkNames, pmodel.NewCIStr(tt.cols[i])) + checkNames = append(checkNames, ast.NewCIStr(tt.cols[i])) } require.ElementsMatchf(t, colNames, checkNames, "test %d %s", i, createTable) tk.MustExec("drop table t1") @@ -2675,7 +2675,7 @@ func checkDMLInAllStates(t *testing.T, tk, tk2 *testkit.TestKit, schemaName, alt transitions := 0 var currTbl table.Table currSchema := sessiontxn.GetTxnManager(tk2.Session()).GetTxnInfoSchema() - prevTbl, err := currSchema.TableByName(context.Background(), pmodel.NewCIStr(schemaName), pmodel.NewCIStr("t")) + prevTbl, err := currSchema.TableByName(context.Background(), ast.NewCIStr(schemaName), ast.NewCIStr("t")) require.NoError(t, err) var hookErr error testfailpoint.EnableCall(t, "github.com/pingcap/tidb/pkg/ddl/beforeRunOneJobStep", func(job *model.Job) { @@ -2723,7 +2723,7 @@ func checkDMLInAllStates(t *testing.T, tk, tk2 *testkit.TestKit, schemaName, alt tk2.MustQuery(`select count(*) from (select a from t except select a from t2) a`).Check(testkit.Rows("0")) tk2.MustQuery(`select count(*) from (select a from t2 except select a from t) a`).Check(testkit.Rows("0")) currSchema = sessiontxn.GetTxnManager(tk2.Session()).GetTxnInfoSchema() - currTbl, hookErr = currSchema.TableByName(context.Background(), pmodel.NewCIStr(schemaName), pmodel.NewCIStr("t")) + currTbl, hookErr = currSchema.TableByName(context.Background(), ast.NewCIStr(schemaName), ast.NewCIStr("t")) require.True(t, tables.SwapReorgPartFields(currTbl, prevTbl)) // Now using previous schema version diff --git a/pkg/table/temptable/BUILD.bazel b/pkg/table/temptable/BUILD.bazel index ff7d98f7656de..eab08f54c4ac8 100644 --- a/pkg/table/temptable/BUILD.bazel +++ b/pkg/table/temptable/BUILD.bazel @@ -16,7 +16,6 @@ go_library( "//pkg/meta/autoid", "//pkg/meta/model", "//pkg/parser/ast", - "//pkg/parser/model", "//pkg/sessionctx", "//pkg/sessionctx/variable", "//pkg/store/driver/txn", @@ -44,7 +43,7 @@ go_test( "//pkg/kv", "//pkg/meta/autoid", "//pkg/meta/model", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/parser/mysql", "//pkg/sessionctx", "//pkg/store/driver/txn", diff --git a/pkg/table/temptable/ddl.go b/pkg/table/temptable/ddl.go index dc673f019c458..abdca0f8b456e 100644 --- a/pkg/table/temptable/ddl.go +++ b/pkg/table/temptable/ddl.go @@ -25,7 +25,6 @@ import ( "github.com/pingcap/tidb/pkg/meta/autoid" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/ast" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/table" @@ -37,8 +36,8 @@ import ( // TemporaryTableDDL is an interface providing ddl operations for temporary table type TemporaryTableDDL interface { CreateLocalTemporaryTable(db *model.DBInfo, info *model.TableInfo) error - DropLocalTemporaryTable(schema pmodel.CIStr, tblName pmodel.CIStr) error - TruncateLocalTemporaryTable(schema pmodel.CIStr, tblName pmodel.CIStr) error + DropLocalTemporaryTable(schema ast.CIStr, tblName ast.CIStr) error + TruncateLocalTemporaryTable(schema ast.CIStr, tblName ast.CIStr) error } // temporaryTableDDL implements temptable.TemporaryTableDDL @@ -59,7 +58,7 @@ func (d *temporaryTableDDL) CreateLocalTemporaryTable(db *model.DBInfo, info *mo return ensureLocalTemporaryTables(d.sctx).AddTable(db, tbl) } -func (d *temporaryTableDDL) DropLocalTemporaryTable(schema pmodel.CIStr, tblName pmodel.CIStr) error { +func (d *temporaryTableDDL) DropLocalTemporaryTable(schema ast.CIStr, tblName ast.CIStr) error { tbl, err := checkLocalTemporaryExistsAndReturn(d.sctx, schema, tblName) if err != nil { return err @@ -69,7 +68,7 @@ func (d *temporaryTableDDL) DropLocalTemporaryTable(schema pmodel.CIStr, tblName return d.clearTemporaryTableRecords(tbl.Meta().ID) } -func (d *temporaryTableDDL) TruncateLocalTemporaryTable(schema pmodel.CIStr, tblName pmodel.CIStr) error { +func (d *temporaryTableDDL) TruncateLocalTemporaryTable(schema ast.CIStr, tblName ast.CIStr) error { oldTbl, err := checkLocalTemporaryExistsAndReturn(d.sctx, schema, tblName) if err != nil { return err @@ -124,7 +123,7 @@ func (d *temporaryTableDDL) clearTemporaryTableRecords(tblID int64) error { return nil } -func checkLocalTemporaryExistsAndReturn(sctx sessionctx.Context, schema pmodel.CIStr, tblName pmodel.CIStr) (table.Table, error) { +func checkLocalTemporaryExistsAndReturn(sctx sessionctx.Context, schema ast.CIStr, tblName ast.CIStr) (table.Table, error) { ident := ast.Ident{Schema: schema, Name: tblName} localTemporaryTables := getLocalTemporaryTables(sctx) if localTemporaryTables == nil { diff --git a/pkg/table/temptable/ddl_test.go b/pkg/table/temptable/ddl_test.go index cc3fc5cc915ae..0b97b7aa13287 100644 --- a/pkg/table/temptable/ddl_test.go +++ b/pkg/table/temptable/ddl_test.go @@ -21,7 +21,7 @@ import ( "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "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/store/mockstore" @@ -64,7 +64,7 @@ func TestAddLocalTemporaryTable(t *testing.T) { require.NotNil(t, sessVars.LocalTemporaryTables) require.NotNil(t, sessVars.TemporaryTableData) require.Equal(t, int64(1), tbl1.ID) - got, exists := sessVars.LocalTemporaryTables.(*infoschema.SessionTables).TableByName(context.Background(), pmodel.NewCIStr("db1"), pmodel.NewCIStr("t1")) + got, exists := sessVars.LocalTemporaryTables.(*infoschema.SessionTables).TableByName(context.Background(), ast.NewCIStr("db1"), ast.NewCIStr("t1")) require.True(t, exists) require.Equal(t, got.Meta(), tbl1) @@ -72,7 +72,7 @@ func TestAddLocalTemporaryTable(t *testing.T) { err = ddl.CreateLocalTemporaryTable(db1, tbl2) require.NoError(t, err) require.Equal(t, int64(2), tbl2.ID) - got, exists = sessVars.LocalTemporaryTables.(*infoschema.SessionTables).TableByName(context.Background(), pmodel.NewCIStr("db1"), pmodel.NewCIStr("t2")) + got, exists = sessVars.LocalTemporaryTables.(*infoschema.SessionTables).TableByName(context.Background(), ast.NewCIStr("db1"), ast.NewCIStr("t2")) require.True(t, exists) require.Equal(t, got.Meta(), tbl2) @@ -89,20 +89,20 @@ func TestAddLocalTemporaryTable(t *testing.T) { tbl1x := newMockTable("t1") err = ddl.CreateLocalTemporaryTable(db1, tbl1x) require.True(t, infoschema.ErrTableExists.Equal(err)) - got, exists = sessVars.LocalTemporaryTables.(*infoschema.SessionTables).TableByName(context.Background(), pmodel.NewCIStr("db1"), pmodel.NewCIStr("t1")) + got, exists = sessVars.LocalTemporaryTables.(*infoschema.SessionTables).TableByName(context.Background(), ast.NewCIStr("db1"), ast.NewCIStr("t1")) require.True(t, exists) require.Equal(t, got.Meta(), tbl1) // insert should be success for same table name in different db err = ddl.CreateLocalTemporaryTable(db2, tbl1x) require.NoError(t, err) - got, exists = sessVars.LocalTemporaryTables.(*infoschema.SessionTables).TableByName(context.Background(), pmodel.NewCIStr("db2"), pmodel.NewCIStr("t1")) + got, exists = sessVars.LocalTemporaryTables.(*infoschema.SessionTables).TableByName(context.Background(), ast.NewCIStr("db2"), ast.NewCIStr("t1")) require.Equal(t, int64(4), got.Meta().ID) require.True(t, exists) require.Equal(t, got.Meta(), tbl1x) // tbl1 still exist - got, exists = sessVars.LocalTemporaryTables.(*infoschema.SessionTables).TableByName(context.Background(), pmodel.NewCIStr("db1"), pmodel.NewCIStr("t1")) + got, exists = sessVars.LocalTemporaryTables.(*infoschema.SessionTables).TableByName(context.Background(), ast.NewCIStr("db1"), ast.NewCIStr("t1")) require.True(t, exists) require.Equal(t, got.Meta(), tbl1) } @@ -114,7 +114,7 @@ func TestRemoveLocalTemporaryTable(t *testing.T) { db1 := newMockSchema("db1") // remove when empty - err := ddl.DropLocalTemporaryTable(pmodel.NewCIStr("db1"), pmodel.NewCIStr("t1")) + err := ddl.DropLocalTemporaryTable(ast.NewCIStr("db1"), ast.NewCIStr("t1")) require.True(t, infoschema.ErrTableNotExists.Equal(err)) // add one table @@ -127,11 +127,11 @@ func TestRemoveLocalTemporaryTable(t *testing.T) { require.NoError(t, err) // remove failed when table not found - err = ddl.DropLocalTemporaryTable(pmodel.NewCIStr("db1"), pmodel.NewCIStr("t2")) + err = ddl.DropLocalTemporaryTable(ast.NewCIStr("db1"), ast.NewCIStr("t2")) require.True(t, infoschema.ErrTableNotExists.Equal(err)) // remove failed when table not found (same table name in different db) - err = ddl.DropLocalTemporaryTable(pmodel.NewCIStr("db2"), pmodel.NewCIStr("t1")) + err = ddl.DropLocalTemporaryTable(ast.NewCIStr("db2"), ast.NewCIStr("t1")) require.True(t, infoschema.ErrTableNotExists.Equal(err)) // check failed remove should have no effects @@ -143,9 +143,9 @@ func TestRemoveLocalTemporaryTable(t *testing.T) { require.Equal(t, []byte("v1"), val) // remove success - err = ddl.DropLocalTemporaryTable(pmodel.NewCIStr("db1"), pmodel.NewCIStr("t1")) + err = ddl.DropLocalTemporaryTable(ast.NewCIStr("db1"), ast.NewCIStr("t1")) require.NoError(t, err) - got, exists = sessVars.LocalTemporaryTables.(*infoschema.SessionTables).TableByName(context.Background(), pmodel.NewCIStr("db1"), pmodel.NewCIStr("t1")) + got, exists = sessVars.LocalTemporaryTables.(*infoschema.SessionTables).TableByName(context.Background(), ast.NewCIStr("db1"), ast.NewCIStr("t1")) require.Nil(t, got) require.False(t, exists) val, err = sessVars.TemporaryTableData.Get(context.Background(), k) @@ -160,7 +160,7 @@ func TestTruncateLocalTemporaryTable(t *testing.T) { db1 := newMockSchema("db1") // truncate when empty - err := ddl.TruncateLocalTemporaryTable(pmodel.NewCIStr("db1"), pmodel.NewCIStr("t1")) + err := ddl.TruncateLocalTemporaryTable(ast.NewCIStr("db1"), ast.NewCIStr("t1")) require.True(t, infoschema.ErrTableNotExists.Equal(err)) require.Nil(t, sessVars.LocalTemporaryTables) require.Nil(t, sessVars.TemporaryTableData) @@ -175,13 +175,13 @@ func TestTruncateLocalTemporaryTable(t *testing.T) { require.NoError(t, err) // truncate failed for table not exist - err = ddl.TruncateLocalTemporaryTable(pmodel.NewCIStr("db1"), pmodel.NewCIStr("t2")) + err = ddl.TruncateLocalTemporaryTable(ast.NewCIStr("db1"), ast.NewCIStr("t2")) require.True(t, infoschema.ErrTableNotExists.Equal(err)) - err = ddl.TruncateLocalTemporaryTable(pmodel.NewCIStr("db2"), pmodel.NewCIStr("t1")) + err = ddl.TruncateLocalTemporaryTable(ast.NewCIStr("db2"), ast.NewCIStr("t1")) require.True(t, infoschema.ErrTableNotExists.Equal(err)) // check failed should have no effects - got, exists := sessVars.LocalTemporaryTables.(*infoschema.SessionTables).TableByName(context.Background(), pmodel.NewCIStr("db1"), pmodel.NewCIStr("t1")) + got, exists := sessVars.LocalTemporaryTables.(*infoschema.SessionTables).TableByName(context.Background(), ast.NewCIStr("db1"), ast.NewCIStr("t1")) require.True(t, exists) require.Equal(t, got.Meta(), tbl1) val, err := sessVars.TemporaryTableData.Get(context.Background(), k) @@ -198,9 +198,9 @@ func TestTruncateLocalTemporaryTable(t *testing.T) { require.NoError(t, err) // truncate success - err = ddl.TruncateLocalTemporaryTable(pmodel.NewCIStr("db1"), pmodel.NewCIStr("t1")) + err = ddl.TruncateLocalTemporaryTable(ast.NewCIStr("db1"), ast.NewCIStr("t1")) require.NoError(t, err) - got, exists = sessVars.LocalTemporaryTables.(*infoschema.SessionTables).TableByName(context.Background(), pmodel.NewCIStr("db1"), pmodel.NewCIStr("t1")) + got, exists = sessVars.LocalTemporaryTables.(*infoschema.SessionTables).TableByName(context.Background(), ast.NewCIStr("db1"), ast.NewCIStr("t1")) require.True(t, exists) require.NotEqual(t, got.Meta(), tbl1) require.Equal(t, int64(3), got.Meta().ID) @@ -215,13 +215,13 @@ func TestTruncateLocalTemporaryTable(t *testing.T) { } func newMockTable(tblName string) *model.TableInfo { - c1 := &model.ColumnInfo{ID: 1, Name: pmodel.NewCIStr("c1"), State: model.StatePublic, Offset: 0, FieldType: *types.NewFieldType(mysql.TypeLonglong)} - c2 := &model.ColumnInfo{ID: 2, Name: pmodel.NewCIStr("c2"), State: model.StatePublic, Offset: 1, FieldType: *types.NewFieldType(mysql.TypeVarchar)} + c1 := &model.ColumnInfo{ID: 1, Name: ast.NewCIStr("c1"), State: model.StatePublic, Offset: 0, FieldType: *types.NewFieldType(mysql.TypeLonglong)} + c2 := &model.ColumnInfo{ID: 2, Name: ast.NewCIStr("c2"), State: model.StatePublic, Offset: 1, FieldType: *types.NewFieldType(mysql.TypeVarchar)} - tblInfo := &model.TableInfo{Name: pmodel.NewCIStr(tblName), Columns: []*model.ColumnInfo{c1, c2}, PKIsHandle: true} + tblInfo := &model.TableInfo{Name: ast.NewCIStr(tblName), Columns: []*model.ColumnInfo{c1, c2}, PKIsHandle: true} return tblInfo } func newMockSchema(schemaName string) *model.DBInfo { - return &model.DBInfo{ID: 10, Name: pmodel.NewCIStr(schemaName), State: model.StatePublic} + return &model.DBInfo{ID: 10, Name: ast.NewCIStr(schemaName), State: model.StatePublic} } diff --git a/pkg/table/temptable/main_test.go b/pkg/table/temptable/main_test.go index 359aee02463a6..c965fb88bfbd9 100644 --- a/pkg/table/temptable/main_test.go +++ b/pkg/table/temptable/main_test.go @@ -25,7 +25,7 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta/autoid" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/testkit/testsetup" @@ -76,10 +76,10 @@ func (is *mockedInfoSchema) TableByID(_ context.Context, tblID int64) (table.Tab tblInfo := &model.TableInfo{ ID: tblID, - Name: pmodel.NewCIStr(fmt.Sprintf("tb%d", tblID)), + Name: ast.NewCIStr(fmt.Sprintf("tb%d", tblID)), Columns: []*model.ColumnInfo{{ ID: 1, - Name: pmodel.NewCIStr("col1"), + Name: ast.NewCIStr("col1"), Offset: 0, FieldType: *types.NewFieldType(mysql.TypeLonglong), State: model.StatePublic, diff --git a/pkg/testkit/BUILD.bazel b/pkg/testkit/BUILD.bazel index 93e29f8184c73..225feefecf2f5 100644 --- a/pkg/testkit/BUILD.bazel +++ b/pkg/testkit/BUILD.bazel @@ -23,7 +23,6 @@ go_library( "//pkg/meta/model", "//pkg/parser/ast", "//pkg/parser/auth", - "//pkg/parser/model", "//pkg/parser/mysql", "//pkg/parser/terror", "//pkg/planner/core", diff --git a/pkg/testkit/external/BUILD.bazel b/pkg/testkit/external/BUILD.bazel index 29a20f24882f8..0e4ebfc7b68ea 100644 --- a/pkg/testkit/external/BUILD.bazel +++ b/pkg/testkit/external/BUILD.bazel @@ -7,7 +7,7 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/domain", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/table", "//pkg/table/tables", "//pkg/testkit", diff --git a/pkg/testkit/external/util.go b/pkg/testkit/external/util.go index 9b7b812baf786..acc01eae5e29d 100644 --- a/pkg/testkit/external/util.go +++ b/pkg/testkit/external/util.go @@ -21,7 +21,7 @@ import ( "testing" "github.com/pingcap/tidb/pkg/domain" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/table/tables" "github.com/pingcap/tidb/pkg/testkit" @@ -33,7 +33,7 @@ func GetTableByName(t *testing.T, tk *testkit.TestKit, db, table string) table.T dom := domain.GetDomain(tk.Session()) // Make sure the table schema is the new schema. require.NoError(t, dom.Reload()) - tbl, err := dom.InfoSchema().TableByName(context.Background(), model.NewCIStr(db), model.NewCIStr(table)) + tbl, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr(db), ast.NewCIStr(table)) require.NoError(t, err) return tbl } @@ -59,7 +59,7 @@ func GetModifyColumn(t *testing.T, tk *testkit.TestKit, db, tbl, colName string, // GetIndexID is used to get the index ID from full qualified name. func GetIndexID(t *testing.T, tk *testkit.TestKit, dbName, tblName, idxName string) int64 { is := domain.GetDomain(tk.Session()).InfoSchema() - tt, err := is.TableByName(context.Background(), model.NewCIStr(dbName), model.NewCIStr(tblName)) + tt, err := is.TableByName(context.Background(), ast.NewCIStr(dbName), ast.NewCIStr(tblName)) require.NoError(t, err) for _, idx := range tt.Indices() { diff --git a/pkg/testkit/mockstore.go b/pkg/testkit/mockstore.go index 896ca67f2d174..0830c4ea81493 100644 --- a/pkg/testkit/mockstore.go +++ b/pkg/testkit/mockstore.go @@ -31,7 +31,7 @@ import ( "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/resourcemanager" "github.com/pingcap/tidb/pkg/session" "github.com/pingcap/tidb/pkg/store/driver" @@ -282,7 +282,7 @@ func CreateMockStoreAndDomainWithSchemaLease(t testing.TB, lease time.Duration, // SetTiFlashReplica is to set TiFlash replica func SetTiFlashReplica(t testing.TB, dom *domain.Domain, dbName, tableName string) { is := dom.InfoSchema() - tblInfo, err := is.TableByName(context.Background(), pmodel.NewCIStr(dbName), pmodel.NewCIStr(tableName)) + tblInfo, err := is.TableByName(context.Background(), ast.NewCIStr(dbName), ast.NewCIStr(tableName)) require.NoError(t, err) tblInfo.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ Count: 1, diff --git a/pkg/ttl/cache/BUILD.bazel b/pkg/ttl/cache/BUILD.bazel index 03147d4983e44..2f10e5abea3ba 100644 --- a/pkg/ttl/cache/BUILD.bazel +++ b/pkg/ttl/cache/BUILD.bazel @@ -20,7 +20,6 @@ go_library( "//pkg/meta/model", "//pkg/parser/ast", "//pkg/parser/charset", - "//pkg/parser/model", "//pkg/parser/mysql", "//pkg/parser/terror", "//pkg/sessionctx", @@ -59,7 +58,6 @@ go_test( "//pkg/kv", "//pkg/meta/model", "//pkg/parser/ast", - "//pkg/parser/model", "//pkg/parser/mysql", "//pkg/server", "//pkg/session", diff --git a/pkg/ttl/cache/infoschema.go b/pkg/ttl/cache/infoschema.go index 6138e2c9671b0..b31359a968647 100644 --- a/pkg/ttl/cache/infoschema.go +++ b/pkg/ttl/cache/infoschema.go @@ -20,7 +20,7 @@ import ( "github.com/pingcap/tidb/pkg/infoschema" infoschemacontext "github.com/pingcap/tidb/pkg/infoschema/context" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/ttl/session" "github.com/pingcap/tidb/pkg/util/logutil" "go.uber.org/zap" @@ -92,7 +92,7 @@ func (isc *InfoSchemaCache) Update(se session.Session) error { return nil } -func (isc *InfoSchemaCache) newTable(schema pmodel.CIStr, tblInfo *model.TableInfo, +func (isc *InfoSchemaCache) newTable(schema ast.CIStr, tblInfo *model.TableInfo, par *model.PartitionDefinition) (*PhysicalTable, error) { id := tblInfo.ID if par != nil { @@ -106,7 +106,7 @@ func (isc *InfoSchemaCache) newTable(schema pmodel.CIStr, tblInfo *model.TableIn } } - partitionName := pmodel.NewCIStr("") + partitionName := ast.NewCIStr("") if par != nil { partitionName = par.Name } diff --git a/pkg/ttl/cache/split_test.go b/pkg/ttl/cache/split_test.go index 83bbc55a49594..231c2f13f4d22 100644 --- a/pkg/ttl/cache/split_test.go +++ b/pkg/ttl/cache/split_test.go @@ -25,7 +25,7 @@ import ( "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/kv" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/store/helper" "github.com/pingcap/tidb/pkg/tablecodec" @@ -246,9 +246,9 @@ func createTTLTableWithSQL(t *testing.T, tk *testkit.TestKit, name string, sql s tk.MustExec(sql) is, ok := tk.Session().GetDomainInfoSchema().(infoschema.InfoSchema) require.True(t, ok) - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr(name)) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr(name)) require.NoError(t, err) - ttlTbl, err := cache.NewPhysicalTable(model.NewCIStr("test"), tbl.Meta(), model.NewCIStr("")) + ttlTbl, err := cache.NewPhysicalTable(ast.NewCIStr("test"), tbl.Meta(), ast.NewCIStr("")) require.NoError(t, err) return ttlTbl } diff --git a/pkg/ttl/cache/table.go b/pkg/ttl/cache/table.go index 98feed3739e47..c608063294103 100644 --- a/pkg/ttl/cache/table.go +++ b/pkg/ttl/cache/table.go @@ -29,7 +29,6 @@ import ( "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/charset" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/table/tables" @@ -101,10 +100,10 @@ type PhysicalTable struct { // ID is the physical ID of the table ID int64 // Schema is the database name of the table - Schema pmodel.CIStr + Schema ast.CIStr *model.TableInfo // Partition is the partition name - Partition pmodel.CIStr + Partition ast.CIStr // PartitionDef is the partition definition PartitionDef *model.PartitionDefinition // KeyColumns is the cluster index key columns for the table @@ -116,9 +115,9 @@ type PhysicalTable struct { } // NewBasePhysicalTable create a new PhysicalTable with specific timeColumn. -func NewBasePhysicalTable(schema pmodel.CIStr, +func NewBasePhysicalTable(schema ast.CIStr, tbl *model.TableInfo, - partition pmodel.CIStr, + partition ast.CIStr, timeColumn *model.ColumnInfo, ) (*PhysicalTable, error) { if tbl.State != model.StatePublic { @@ -169,7 +168,7 @@ func NewBasePhysicalTable(schema pmodel.CIStr, } // NewPhysicalTable create a new PhysicalTable -func NewPhysicalTable(schema pmodel.CIStr, tbl *model.TableInfo, partition pmodel.CIStr) (*PhysicalTable, error) { +func NewPhysicalTable(schema ast.CIStr, tbl *model.TableInfo, partition ast.CIStr) (*PhysicalTable, error) { ttlInfo := tbl.TTLInfo if ttlInfo == nil { return nil, errors.Errorf("table '%s.%s' is not a ttl table", schema, tbl.Name) diff --git a/pkg/ttl/cache/table_test.go b/pkg/ttl/cache/table_test.go index d26627c3d04cb..a3c7d07db6177 100644 --- a/pkg/ttl/cache/table_test.go +++ b/pkg/ttl/cache/table_test.go @@ -22,7 +22,6 @@ import ( "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/ast" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/ttl/cache" "github.com/pingcap/tidb/pkg/ttl/session" @@ -95,12 +94,12 @@ func TestNewTTLTable(t *testing.T) { for _, c := range cases { is := do.InfoSchema() - tbl, err := is.TableByName(context.Background(), pmodel.NewCIStr(c.db), pmodel.NewCIStr(c.tbl)) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr(c.db), ast.NewCIStr(c.tbl)) require.NoError(t, err) tblInfo := tbl.Meta() var physicalTbls []*cache.PhysicalTable if tblInfo.Partition == nil { - ttlTbl, err := cache.NewPhysicalTable(pmodel.NewCIStr(c.db), tblInfo, pmodel.NewCIStr("")) + ttlTbl, err := cache.NewPhysicalTable(ast.NewCIStr(c.db), tblInfo, ast.NewCIStr("")) if c.timeCol == "" { require.Error(t, err) continue @@ -109,7 +108,7 @@ func TestNewTTLTable(t *testing.T) { physicalTbls = append(physicalTbls, ttlTbl) } else { for _, partition := range tblInfo.Partition.Definitions { - ttlTbl, err := cache.NewPhysicalTable(pmodel.NewCIStr(c.db), tblInfo, partition.Name) + ttlTbl, err := cache.NewPhysicalTable(ast.NewCIStr(c.db), tblInfo, partition.Name) if c.timeCol == "" { require.Error(t, err) continue @@ -170,10 +169,10 @@ func TestTableEvalTTLExpireTime(t *testing.T) { tk.MustExec("set @@time_zone='Asia/Tokyo'") tk.MustExec("create table test.t(a int, t datetime) ttl = `t` + interval 1 month") - tb, err := do.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tb, err := do.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tblInfo := tb.Meta() - ttlTbl, err := cache.NewPhysicalTable(pmodel.NewCIStr("test"), tblInfo, pmodel.NewCIStr("")) + ttlTbl, err := cache.NewPhysicalTable(ast.NewCIStr("test"), tblInfo, ast.NewCIStr("")) require.NoError(t, err) se := session.NewSession(tk.Session(), tk.Session(), nil) @@ -194,10 +193,10 @@ func TestTableEvalTTLExpireTime(t *testing.T) { // should support a string format interval tk.MustExec("create table test.t2(a int, t datetime) ttl = `t` + interval '1:3' hour_minute") - tb2, err := do.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t2")) + tb2, err := do.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t2")) require.NoError(t, err) tblInfo2 := tb2.Meta() - ttlTbl2, err := cache.NewPhysicalTable(pmodel.NewCIStr("test"), tblInfo2, pmodel.NewCIStr("")) + ttlTbl2, err := cache.NewPhysicalTable(ast.NewCIStr("test"), tblInfo2, ast.NewCIStr("")) require.NoError(t, err) now, err = time.ParseInLocation(time.DateTime, "2020-01-01 15:00:00", tz1) require.NoError(t, err) diff --git a/pkg/ttl/sqlbuilder/BUILD.bazel b/pkg/ttl/sqlbuilder/BUILD.bazel index 4096caae7dc4c..68d7fe599bbde 100644 --- a/pkg/ttl/sqlbuilder/BUILD.bazel +++ b/pkg/ttl/sqlbuilder/BUILD.bazel @@ -32,7 +32,6 @@ go_test( "//pkg/meta/model", "//pkg/parser", "//pkg/parser/ast", - "//pkg/parser/model", "//pkg/parser/mysql", "//pkg/parser/terror", "//pkg/testkit", diff --git a/pkg/ttl/sqlbuilder/sql_test.go b/pkg/ttl/sqlbuilder/sql_test.go index 83c415eae9bde..de487d44f488f 100644 --- a/pkg/ttl/sqlbuilder/sql_test.go +++ b/pkg/ttl/sqlbuilder/sql_test.go @@ -25,7 +25,6 @@ import ( "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser" "github.com/pingcap/tidb/pkg/parser/ast" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/testkit" @@ -39,19 +38,19 @@ import ( func TestEscape(t *testing.T) { tb := &cache.PhysicalTable{ - Schema: pmodel.NewCIStr("testp;\"';123`456"), + Schema: ast.NewCIStr("testp;\"';123`456"), TableInfo: &model.TableInfo{ - Name: pmodel.NewCIStr("tp\"';123`456"), + Name: ast.NewCIStr("tp\"';123`456"), }, KeyColumns: []*model.ColumnInfo{ - {Name: pmodel.NewCIStr("col1\"';123`456"), FieldType: *types.NewFieldType(mysql.TypeString)}, + {Name: ast.NewCIStr("col1\"';123`456"), FieldType: *types.NewFieldType(mysql.TypeString)}, }, TimeColumn: &model.ColumnInfo{ - Name: pmodel.NewCIStr("time\"';123`456"), + Name: ast.NewCIStr("time\"';123`456"), FieldType: *types.NewFieldType(mysql.TypeDatetime), }, PartitionDef: &model.PartitionDefinition{ - Name: pmodel.NewCIStr("p1\"';123`456"), + Name: ast.NewCIStr("p1\"';123`456"), }, } @@ -357,7 +356,7 @@ func TestFormatSQLDatum(t *testing.T) { sb.WriteString("\n);") tk.MustExec(sb.String()) - tbl, err := do.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := do.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) for i, c := range cases { @@ -404,43 +403,43 @@ func TestSQLBuilder(t *testing.T) { var b *sqlbuilder.SQLBuilder t1 := &cache.PhysicalTable{ - Schema: pmodel.NewCIStr("test"), + Schema: ast.NewCIStr("test"), TableInfo: &model.TableInfo{ - Name: pmodel.NewCIStr("t1"), + Name: ast.NewCIStr("t1"), }, KeyColumns: []*model.ColumnInfo{ - {Name: pmodel.NewCIStr("id"), FieldType: *types.NewFieldType(mysql.TypeVarchar)}, + {Name: ast.NewCIStr("id"), FieldType: *types.NewFieldType(mysql.TypeVarchar)}, }, TimeColumn: &model.ColumnInfo{ - Name: pmodel.NewCIStr("time"), + Name: ast.NewCIStr("time"), FieldType: *types.NewFieldType(mysql.TypeDatetime), }, } t2 := &cache.PhysicalTable{ - Schema: pmodel.NewCIStr("test2"), + Schema: ast.NewCIStr("test2"), TableInfo: &model.TableInfo{ - Name: pmodel.NewCIStr("t2"), + Name: ast.NewCIStr("t2"), }, KeyColumns: []*model.ColumnInfo{ - {Name: pmodel.NewCIStr("a"), FieldType: *types.NewFieldType(mysql.TypeVarchar)}, - {Name: pmodel.NewCIStr("b"), FieldType: *types.NewFieldType(mysql.TypeInt24)}, + {Name: ast.NewCIStr("a"), FieldType: *types.NewFieldType(mysql.TypeVarchar)}, + {Name: ast.NewCIStr("b"), FieldType: *types.NewFieldType(mysql.TypeInt24)}, }, TimeColumn: &model.ColumnInfo{ - Name: pmodel.NewCIStr("time"), + Name: ast.NewCIStr("time"), FieldType: *types.NewFieldType(mysql.TypeDatetime), }, } tp := &cache.PhysicalTable{ - Schema: pmodel.NewCIStr("testp"), + Schema: ast.NewCIStr("testp"), TableInfo: &model.TableInfo{ - Name: pmodel.NewCIStr("tp"), + Name: ast.NewCIStr("tp"), }, KeyColumns: t1.KeyColumns, TimeColumn: t1.TimeColumn, PartitionDef: &model.PartitionDefinition{ - Name: pmodel.NewCIStr("p1"), + Name: ast.NewCIStr("p1"), }, } @@ -580,31 +579,31 @@ func TestSQLBuilder(t *testing.T) { func TestScanQueryGenerator(t *testing.T) { t1 := &cache.PhysicalTable{ - Schema: pmodel.NewCIStr("test"), + Schema: ast.NewCIStr("test"), TableInfo: &model.TableInfo{ - Name: pmodel.NewCIStr("t1"), + Name: ast.NewCIStr("t1"), }, KeyColumns: []*model.ColumnInfo{ - {Name: pmodel.NewCIStr("id"), FieldType: *types.NewFieldType(mysql.TypeInt24)}, + {Name: ast.NewCIStr("id"), FieldType: *types.NewFieldType(mysql.TypeInt24)}, }, TimeColumn: &model.ColumnInfo{ - Name: pmodel.NewCIStr("time"), + Name: ast.NewCIStr("time"), FieldType: *types.NewFieldType(mysql.TypeDatetime), }, } t2 := &cache.PhysicalTable{ - Schema: pmodel.NewCIStr("test2"), + Schema: ast.NewCIStr("test2"), TableInfo: &model.TableInfo{ - Name: pmodel.NewCIStr("t2"), + Name: ast.NewCIStr("t2"), }, KeyColumns: []*model.ColumnInfo{ - {Name: pmodel.NewCIStr("a"), FieldType: *types.NewFieldType(mysql.TypeInt24)}, - {Name: pmodel.NewCIStr("b"), FieldType: *types.NewFieldType(mysql.TypeVarchar)}, - {Name: pmodel.NewCIStr("c"), FieldType: types.NewFieldTypeBuilder().SetType(mysql.TypeString).SetFlag(mysql.BinaryFlag).Build()}, + {Name: ast.NewCIStr("a"), FieldType: *types.NewFieldType(mysql.TypeInt24)}, + {Name: ast.NewCIStr("b"), FieldType: *types.NewFieldType(mysql.TypeVarchar)}, + {Name: ast.NewCIStr("c"), FieldType: types.NewFieldTypeBuilder().SetType(mysql.TypeString).SetFlag(mysql.BinaryFlag).Build()}, }, TimeColumn: &model.ColumnInfo{ - Name: pmodel.NewCIStr("time"), + Name: ast.NewCIStr("time"), FieldType: *types.NewFieldType(mysql.TypeDatetime), }, } @@ -864,30 +863,30 @@ func TestScanQueryGenerator(t *testing.T) { func TestBuildDeleteSQL(t *testing.T) { t1 := &cache.PhysicalTable{ - Schema: pmodel.NewCIStr("test"), + Schema: ast.NewCIStr("test"), TableInfo: &model.TableInfo{ - Name: pmodel.NewCIStr("t1"), + Name: ast.NewCIStr("t1"), }, KeyColumns: []*model.ColumnInfo{ - {Name: pmodel.NewCIStr("id"), FieldType: *types.NewFieldType(mysql.TypeInt24)}, + {Name: ast.NewCIStr("id"), FieldType: *types.NewFieldType(mysql.TypeInt24)}, }, TimeColumn: &model.ColumnInfo{ - Name: pmodel.NewCIStr("time"), + Name: ast.NewCIStr("time"), FieldType: *types.NewFieldType(mysql.TypeDatetime), }, } t2 := &cache.PhysicalTable{ - Schema: pmodel.NewCIStr("test2"), + Schema: ast.NewCIStr("test2"), TableInfo: &model.TableInfo{ - Name: pmodel.NewCIStr("t2"), + Name: ast.NewCIStr("t2"), }, KeyColumns: []*model.ColumnInfo{ - {Name: pmodel.NewCIStr("a"), FieldType: *types.NewFieldType(mysql.TypeInt24)}, - {Name: pmodel.NewCIStr("b"), FieldType: *types.NewFieldType(mysql.TypeVarchar)}, + {Name: ast.NewCIStr("a"), FieldType: *types.NewFieldType(mysql.TypeInt24)}, + {Name: ast.NewCIStr("b"), FieldType: *types.NewFieldType(mysql.TypeVarchar)}, }, TimeColumn: &model.ColumnInfo{ - Name: pmodel.NewCIStr("time"), + Name: ast.NewCIStr("time"), FieldType: *types.NewFieldType(mysql.TypeDatetime), }, } diff --git a/pkg/ttl/ttlworker/BUILD.bazel b/pkg/ttl/ttlworker/BUILD.bazel index 5ceccc3b2d219..3f8ff3165d52e 100644 --- a/pkg/ttl/ttlworker/BUILD.bazel +++ b/pkg/ttl/ttlworker/BUILD.bazel @@ -23,7 +23,6 @@ go_library( "//pkg/meta/model", "//pkg/metrics", "//pkg/parser/ast", - "//pkg/parser/model", "//pkg/parser/terror", "//pkg/sessionctx", "//pkg/sessionctx/variable", @@ -82,7 +81,6 @@ go_test( "//pkg/meta/model", "//pkg/metrics", "//pkg/parser/ast", - "//pkg/parser/model", "//pkg/parser/mysql", "//pkg/sessionctx", "//pkg/sessionctx/variable", diff --git a/pkg/ttl/ttlworker/job_manager_integration_test.go b/pkg/ttl/ttlworker/job_manager_integration_test.go index b0b5bbce9c0f6..efe12c646ec2c 100644 --- a/pkg/ttl/ttlworker/job_manager_integration_test.go +++ b/pkg/ttl/ttlworker/job_manager_integration_test.go @@ -34,7 +34,6 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/ast" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/statistics" "github.com/pingcap/tidb/pkg/testkit" @@ -184,7 +183,7 @@ func TestFinishJob(t *testing.T) { sessionFactory := sessionFactory(t, dom) - testTable := &cache.PhysicalTable{ID: 2, Schema: pmodel.NewCIStr("db1"), TableInfo: &model.TableInfo{ID: 1, Name: pmodel.NewCIStr("t1"), TTLInfo: &model.TTLInfo{IntervalExprStr: "1", IntervalTimeUnit: int(ast.TimeUnitDay)}}} + testTable := &cache.PhysicalTable{ID: 2, Schema: ast.NewCIStr("db1"), TableInfo: &model.TableInfo{ID: 1, Name: ast.NewCIStr("t1"), TTLInfo: &model.TTLInfo{IntervalExprStr: "1", IntervalTimeUnit: int(ast.TimeUnitDay)}}} tk.MustExec("insert into mysql.tidb_ttl_table_status(table_id) values (2)") @@ -292,7 +291,7 @@ func TestTriggerTTLJob(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("create table t(id int primary key, t timestamp) TTL=`t` + INTERVAL 1 DAY") - tbl, err := do.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := do.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) tblID := tbl.Meta().ID require.NoError(t, err) @@ -343,13 +342,13 @@ func TestTTLDeleteWithTimeZoneChange(t *testing.T) { tk.MustExec("set @@global.tidb_ttl_running_tasks=32") tk.MustExec("create table t1(id int primary key, t datetime) TTL=`t` + INTERVAL 1 DAY TTL_ENABLE='OFF'") - tbl1, err := do.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t1")) + tbl1, err := do.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t1")) require.NoError(t, err) tblID1 := tbl1.Meta().ID tk.MustExec("insert into t1 values(1, NOW()), (2, NOW() - INTERVAL 31 HOUR), (3, NOW() - INTERVAL 33 HOUR)") tk.MustExec("create table t2(id int primary key, t timestamp) TTL=`t` + INTERVAL 1 DAY TTL_ENABLE='OFF'") - tbl2, err := do.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t2")) + tbl2, err := do.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t2")) require.NoError(t, err) tblID2 := tbl2.Meta().ID tk.MustExec("insert into t2 values(1, NOW()), (2, NOW() - INTERVAL 31 HOUR), (3, NOW() - INTERVAL 33 HOUR)") @@ -451,7 +450,7 @@ func TestSubmitJob(t *testing.T) { "PARTITION p0 VALUES LESS THAN (10)," + "PARTITION p1 VALUES LESS THAN (100)" + ")") - table, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("ttlp1")) + table, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("ttlp1")) require.NoError(t, err) tableID := table.Meta().ID var physicalID int64 @@ -526,7 +525,7 @@ func TestRescheduleJobs(t *testing.T) { waitAndStopTTLManager(t, dom) tk.MustExec("create table test.t (id int, created_at datetime) ttl = `created_at` + interval 1 minute ttl_job_interval = '1m'") - table, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + table, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnTTL) @@ -591,7 +590,7 @@ func TestRescheduleJobsAfterTableDropped(t *testing.T) { now := time.Now().In(time.UTC) createTableSQL := "create table test.t (id int, created_at datetime) ttl = `created_at` + interval 1 minute ttl_job_interval = '1m'" tk.MustExec("create table test.t (id int, created_at datetime) ttl = `created_at` + interval 1 minute ttl_job_interval = '1m'") - table, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + table, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnTTL) @@ -631,7 +630,7 @@ func TestRescheduleJobsAfterTableDropped(t *testing.T) { // resume the table tk.MustExec(rb.resume) - table, err = dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + table, err = dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) m.DoGC(context.TODO(), se, now) @@ -647,7 +646,7 @@ func TestJobTimeout(t *testing.T) { waitAndStopTTLManager(t, dom) tk.MustExec("create table test.t (id int, created_at datetime) ttl = `created_at` + interval 1 minute ttl_job_interval = '1m'") - table, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + table, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) tableID := table.Meta().ID require.NoError(t, err) ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnTTL) @@ -718,7 +717,7 @@ func TestTriggerScanTask(t *testing.T) { waitAndStopTTLManager(t, dom) tk.MustExec("create table test.t (id int, created_at datetime) ttl = `created_at` + interval 1 minute ttl_job_interval = '1m'") - tbl, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tblID := tbl.Meta().ID @@ -894,7 +893,7 @@ func TestJobMetrics(t *testing.T) { waitAndStopTTLManager(t, dom) tk.MustExec("create table test.t (id int, created_at datetime) ttl = `created_at` + interval 1 minute ttl_job_interval = '1m'") - table, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + table, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnTTL) @@ -1094,19 +1093,19 @@ func TestManagerJobAdapterCanSubmitJob(t *testing.T) { // not ttl table tk.MustExec("create table t1(t timestamp)") - tbl, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t1")) + tbl, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t1")) require.NoError(t, err) require.False(t, adapter.CanSubmitJob(tbl.Meta().ID, tbl.Meta().ID)) // ttl table tk.MustExec("create table ttl1(t timestamp) TTL=`t`+interval 1 DAY") - tbl, err = dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("ttl1")) + tbl, err = dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("ttl1")) require.NoError(t, err) require.True(t, adapter.CanSubmitJob(tbl.Meta().ID, tbl.Meta().ID)) // ttl table but disabled tk.MustExec("create table ttl2(t timestamp) TTL=`t`+interval 1 DAY TTL_ENABLE='OFF'") - tbl, err = dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("ttl2")) + tbl, err = dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("ttl2")) require.NoError(t, err) require.False(t, adapter.CanSubmitJob(tbl.Meta().ID, tbl.Meta().ID)) @@ -1115,7 +1114,7 @@ func TestManagerJobAdapterCanSubmitJob(t *testing.T) { "PARTITION p0 VALUES LESS THAN (10)," + "PARTITION p1 VALUES LESS THAN (100)" + ")") - tbl, err = dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("ttlp1")) + tbl, err = dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("ttlp1")) require.NoError(t, err) for _, def := range tbl.Meta().Partition.Definitions { require.True(t, adapter.CanSubmitJob(tbl.Meta().ID, def.ID)) @@ -1142,7 +1141,7 @@ func TestManagerJobAdapterCanSubmitJob(t *testing.T) { tk.MustExec("update mysql.tidb_ttl_task set status='finished' where job_id=?", jobID) } } - tbl, err = dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("ttl1")) + tbl, err = dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("ttl1")) require.NoError(t, err) require.True(t, adapter.CanSubmitJob(tbl.Meta().ID, tbl.Meta().ID)) tk.MustExec("update mysql.tidb_ttl_task set status='running' where job_id='8'") @@ -1336,7 +1335,7 @@ func TestFinishAndUpdateOwnerAtSameTime(t *testing.T) { tk.MustExec("use test") tk.MustExec("CREATE TABLE t (id INT PRIMARY KEY, created_at DATETIME) TTL = created_at + INTERVAL 1 HOUR") - testTable, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + testTable, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) now := se.Now().Add(time.Hour * 48) @@ -1381,7 +1380,7 @@ func TestFinishError(t *testing.T) { tk.MustExec("use test") tk.MustExec("CREATE TABLE t (id INT PRIMARY KEY, created_at DATETIME) TTL = created_at + INTERVAL 1 HOUR") - testTable, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + testTable, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) errCount := 5 @@ -1450,7 +1449,7 @@ func TestFinishError(t *testing.T) { tk.MustQuery("select count(*) from mysql.tidb_ttl_task").Check(testkit.Rows("0")) tk.MustExec("CREATE TABLE t (id INT PRIMARY KEY, created_at DATETIME) TTL = created_at + INTERVAL 1 HOUR") require.NoError(t, m.InfoSchemaCache().Update(se)) - testTable, err = dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + testTable, err = dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) // Teset the `updateHeartBeat` can tolerate the `job.finish` error @@ -1499,7 +1498,7 @@ func TestDisableTTLAfterLoseHeartbeat(t *testing.T) { tk.MustExec("use test") tk.MustExec("CREATE TABLE t (id INT PRIMARY KEY, created_at DATETIME) TTL = created_at + INTERVAL 1 HOUR") - testTable, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + testTable, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) ctx := context.Background() @@ -1575,7 +1574,7 @@ func TestDisableTTLAfterLoseHeartbeat(t *testing.T) { tk.MustExec("DROP TABLE t") defer func() { tk.MustExec("CREATE TABLE t (id INT PRIMARY KEY, created_at DATETIME) TTL = created_at + INTERVAL 1 HOUR") - testTable, err = dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + testTable, err = dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) }() @@ -1606,9 +1605,9 @@ func TestJobHeartBeatFailNotBlockOthers(t *testing.T) { tk.MustExec("use test") tk.MustExec("CREATE TABLE t1 (id INT PRIMARY KEY, created_at DATETIME) TTL = created_at + INTERVAL 1 HOUR") tk.MustExec("CREATE TABLE t2 (id INT PRIMARY KEY, created_at DATETIME) TTL = created_at + INTERVAL 1 HOUR") - testTable1, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t1")) + testTable1, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t1")) require.NoError(t, err) - testTable2, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t2")) + testTable2, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t2")) require.NoError(t, err) ctx := context.Background() @@ -1783,7 +1782,7 @@ func TestJobManagerWithFault(t *testing.T) { tk.MustExec("use test") tk.MustExec("DROP TABLE if exists t") tk.MustExec("CREATE TABLE t (id INT PRIMARY KEY, created_at DATETIME) TTL = created_at + INTERVAL 1 HOUR TTL_ENABLE='OFF'") - tbl, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) logutil.BgLogger().Info("create table", zap.Int64("table_id", tbl.Meta().ID)) @@ -1846,7 +1845,7 @@ func TestTimerJobAfterDropTable(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("create table t (created_at datetime) TTL = created_at + INTERVAL 1 HOUR") - tbl, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) m := ttlworker.NewJobManager("test-job-manager", pool, store, nil, func() bool { return true }) diff --git a/pkg/ttl/ttlworker/job_manager_test.go b/pkg/ttl/ttlworker/job_manager_test.go index 7f3260ffa999b..335b156b83352 100644 --- a/pkg/ttl/ttlworker/job_manager_test.go +++ b/pkg/ttl/ttlworker/job_manager_test.go @@ -23,7 +23,6 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/ast" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" timerapi "github.com/pingcap/tidb/pkg/timer/api" "github.com/pingcap/tidb/pkg/ttl/cache" @@ -393,7 +392,7 @@ func TestLockTable(t *testing.T) { oldJobExpireTime := now.Add(-time.Hour) oldJobStartTime := now.Add(-30 * time.Minute) - testPhysicalTable := &cache.PhysicalTable{ID: 1, Schema: pmodel.NewCIStr("test"), TableInfo: &model.TableInfo{ID: 1, Name: pmodel.NewCIStr("t1"), TTLInfo: &model.TTLInfo{ColumnName: pmodel.NewCIStr("test"), IntervalExprStr: "1", IntervalTimeUnit: int(ast.TimeUnitMinute), JobInterval: "1h"}}} + testPhysicalTable := &cache.PhysicalTable{ID: 1, Schema: ast.NewCIStr("test"), TableInfo: &model.TableInfo{ID: 1, Name: ast.NewCIStr("t1"), TTLInfo: &model.TTLInfo{ColumnName: ast.NewCIStr("test"), IntervalExprStr: "1", IntervalTimeUnit: int(ast.TimeUnitMinute), JobInterval: "1h"}}} type executeInfo struct { sql string diff --git a/pkg/ttl/ttlworker/session_test.go b/pkg/ttl/ttlworker/session_test.go index 59e0fcb437d01..c945f362bbb59 100644 --- a/pkg/ttl/ttlworker/session_test.go +++ b/pkg/ttl/ttlworker/session_test.go @@ -28,7 +28,6 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/ast" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/variable" @@ -44,18 +43,18 @@ var idAllocator atomic.Int64 func newMockTTLTbl(t *testing.T, name string) *cache.PhysicalTable { tblInfo := &model.TableInfo{ ID: idAllocator.Add(1), - Name: pmodel.NewCIStr(name), + Name: ast.NewCIStr(name), Columns: []*model.ColumnInfo{ { ID: 1, - Name: pmodel.NewCIStr("time"), + Name: ast.NewCIStr("time"), Offset: 0, FieldType: *types.NewFieldType(mysql.TypeDatetime), State: model.StatePublic, }, }, TTLInfo: &model.TTLInfo{ - ColumnName: pmodel.NewCIStr("time"), + ColumnName: ast.NewCIStr("time"), IntervalExprStr: "1", IntervalTimeUnit: int(ast.TimeUnitSecond), Enable: true, @@ -64,7 +63,7 @@ func newMockTTLTbl(t *testing.T, name string) *cache.PhysicalTable { State: model.StatePublic, } - tbl, err := cache.NewPhysicalTable(pmodel.NewCIStr("test"), tblInfo, pmodel.NewCIStr("")) + tbl, err := cache.NewPhysicalTable(ast.NewCIStr("test"), tblInfo, ast.NewCIStr("")) require.NoError(t, err) return tbl } @@ -330,7 +329,7 @@ func TestValidateTTLWork(t *testing.T) { // test table name changed tbl2 = tbl.TableInfo.Clone() - tbl2.Name = pmodel.NewCIStr("testcc") + tbl2.Name = ast.NewCIStr("testcc") s.sessionInfoSchema = newMockInfoSchema(tbl2) err = validateTTLWork(ctx, s, tbl, expire) require.EqualError(t, err, "[schema:1146]Table 'test.t1' doesn't exist") @@ -345,8 +344,8 @@ func TestValidateTTLWork(t *testing.T) { // test time column name changed tbl2 = tbl.TableInfo.Clone() tbl2.Columns[0] = tbl2.Columns[0].Clone() - tbl2.Columns[0].Name = pmodel.NewCIStr("time2") - tbl2.TTLInfo.ColumnName = pmodel.NewCIStr("time2") + tbl2.Columns[0].Name = ast.NewCIStr("time2") + tbl2.TTLInfo.ColumnName = ast.NewCIStr("time2") s.sessionInfoSchema = newMockInfoSchema(tbl2) err = validateTTLWork(ctx, s, tbl, expire) require.EqualError(t, err, "time column name changed") @@ -381,14 +380,14 @@ func TestValidateTTLWork(t *testing.T) { tp := tbl.TableInfo.Clone() tp.Partition = &model.PartitionInfo{ Definitions: []model.PartitionDefinition{ - {ID: 1023, Name: pmodel.NewCIStr("p0")}, + {ID: 1023, Name: ast.NewCIStr("p0")}, }, } - tbl, err = cache.NewPhysicalTable(pmodel.NewCIStr("test"), tp, pmodel.NewCIStr("p0")) + tbl, err = cache.NewPhysicalTable(ast.NewCIStr("test"), tp, ast.NewCIStr("p0")) require.NoError(t, err) tbl2 = tp.Clone() tbl2.Partition = tp.Partition.Clone() - tbl2.Partition.Definitions[0].Name = pmodel.NewCIStr("p1") + tbl2.Partition.Definitions[0].Name = ast.NewCIStr("p1") s.sessionInfoSchema = newMockInfoSchema(tbl2) err = validateTTLWork(ctx, s, tbl, expire) require.EqualError(t, err, "partition 'p0' is not found in ttl table 'test.t1'") diff --git a/pkg/ttl/ttlworker/task_manager_integration_test.go b/pkg/ttl/ttlworker/task_manager_integration_test.go index e17675392e020..5de39c5448a7c 100644 --- a/pkg/ttl/ttlworker/task_manager_integration_test.go +++ b/pkg/ttl/ttlworker/task_manager_integration_test.go @@ -25,7 +25,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/kv" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/store/mockstore" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/testkit/testflag" @@ -47,7 +47,7 @@ func TestParallelLockNewTask(t *testing.T) { tk.MustExec("set global tidb_ttl_running_tasks = 1000") ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnTTL) tk.MustExec("create table test.t (id int, created_at datetime) TTL= created_at + interval 1 hour") - testTable, err := tk.Session().GetDomainInfoSchema().(infoschema.InfoSchema).TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + testTable, err := tk.Session().GetDomainInfoSchema().(infoschema.InfoSchema).TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) sessionFactory := sessionFactory(t, dom) @@ -133,7 +133,7 @@ func TestParallelSchedule(t *testing.T) { sessionFactory := sessionFactory(t, dom) tk.MustExec("create table test.t(id int, created_at datetime) ttl=created_at + interval 1 day") - table, err := dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + table, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) // 16 tasks and 16 scan workers (in 4 task manager) should be able to be scheduled in a single "reschedule" for i := 0; i < 16; i++ { @@ -190,7 +190,7 @@ func TestTaskScheduleExpireHeartBeat(t *testing.T) { // create table and scan task tk.MustExec("create table test.t(id int, created_at datetime) ttl=created_at + interval 1 day") - table, err := dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + table, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) sql := fmt.Sprintf("insert into mysql.tidb_ttl_task(job_id,table_id,scan_id,expire_time,created_time) values ('test-job', %d, %d, NOW(), NOW())", table.Meta().ID, 1) tk.MustExec(sql) @@ -234,7 +234,7 @@ func TestTaskMetrics(t *testing.T) { // create table and scan task tk.MustExec("create table test.t(id int, created_at datetime) ttl=created_at + interval 1 day") - table, err := dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + table, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) sql := fmt.Sprintf("insert into mysql.tidb_ttl_task(job_id,table_id,scan_id,expire_time,created_time) values ('test-job', %d, %d, NOW(), NOW())", table.Meta().ID, 1) tk.MustExec(sql) @@ -298,7 +298,7 @@ func TestTTLRunningTasksLimitation(t *testing.T) { tk.MustExec("set global tidb_ttl_running_tasks = 32") tk.MustExec("create table test.t(id int, created_at datetime) ttl=created_at + interval 1 day") - table, err := dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + table, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) // 64 tasks and 128 scan workers (in 16 task manager) should only schedule 32 tasks for i := 0; i < 128; i++ { @@ -370,7 +370,7 @@ func TestShrinkScanWorkerAndResignOwner(t *testing.T) { tk.MustExec("set global tidb_ttl_running_tasks = 32") tk.MustExec("create table test.t(id int, created_at datetime) ttl=created_at + interval 1 day") - testTable, err := dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + testTable, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) taskCnt := 8 for id := 0; id < taskCnt; id++ { @@ -550,7 +550,7 @@ func TestTaskCancelledAfterHeartbeatTimeout(t *testing.T) { defer tk.MustExec("set global tidb_ttl_running_tasks = -1") tk.MustExec("create table test.t(id int, created_at datetime) ttl=created_at + interval 1 day") - table, err := dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + table, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) // 4 tasks are inserted into the table for i := 0; i < 4; i++ { @@ -648,7 +648,7 @@ func TestHeartBeatErrorNotBlockOthers(t *testing.T) { tk.MustExec("set global tidb_ttl_running_tasks = 32") tk.MustExec("create table test.t(id int, created_at datetime) ttl=created_at + interval 1 day") - testTable, err := dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + testTable, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) for id := 0; id < 4; id++ { sql := fmt.Sprintf("insert into mysql.tidb_ttl_task(job_id,table_id,scan_id,expire_time,created_time) values ('test-job', %d, %d, NOW() - INTERVAL 1 DAY, NOW())", testTable.Meta().ID, id) diff --git a/pkg/ttl/ttlworker/timer_sync.go b/pkg/ttl/ttlworker/timer_sync.go index f621a627d1907..3d8affe576c3f 100644 --- a/pkg/ttl/ttlworker/timer_sync.go +++ b/pkg/ttl/ttlworker/timer_sync.go @@ -26,7 +26,7 @@ import ( infoschemacontext "github.com/pingcap/tidb/pkg/infoschema/context" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/metrics" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" timerapi "github.com/pingcap/tidb/pkg/timer/api" "github.com/pingcap/tidb/pkg/ttl/cache" "github.com/pingcap/tidb/pkg/ttl/session" @@ -233,7 +233,7 @@ func (g *TTLTimersSyncer) SyncTimers(ctx context.Context, is infoschema.InfoSche } } -func (g *TTLTimersSyncer) syncTimersForTable(ctx context.Context, se session.Session, schema pmodel.CIStr, tblInfo *model.TableInfo) []string { +func (g *TTLTimersSyncer) syncTimersForTable(ctx context.Context, se session.Session, schema ast.CIStr, tblInfo *model.TableInfo) []string { if tblInfo.Partition == nil { key := buildTimerKey(tblInfo, nil) if _, err := g.syncOneTimer(ctx, se, schema, tblInfo, nil, false); err != nil { @@ -255,7 +255,7 @@ func (g *TTLTimersSyncer) syncTimersForTable(ctx context.Context, se session.Ses return keys } -func (g *TTLTimersSyncer) shouldSyncTimer(timer *timerapi.TimerRecord, schema pmodel.CIStr, tblInfo *model.TableInfo, partition *model.PartitionDefinition) bool { +func (g *TTLTimersSyncer) shouldSyncTimer(timer *timerapi.TimerRecord, schema ast.CIStr, tblInfo *model.TableInfo, partition *model.PartitionDefinition) bool { if timer == nil { return true } @@ -269,7 +269,7 @@ func (g *TTLTimersSyncer) shouldSyncTimer(timer *timerapi.TimerRecord, schema pm timer.SchedPolicyExpr != policyExpr } -func (g *TTLTimersSyncer) syncOneTimer(ctx context.Context, se session.Session, schema pmodel.CIStr, tblInfo *model.TableInfo, partition *model.PartitionDefinition, skipCache bool) (*timerapi.TimerRecord, error) { +func (g *TTLTimersSyncer) syncOneTimer(ctx context.Context, se session.Session, schema ast.CIStr, tblInfo *model.TableInfo, partition *model.PartitionDefinition, skipCache bool) (*timerapi.TimerRecord, error) { key := buildTimerKey(tblInfo, partition) tags := getTimerTags(schema, tblInfo, partition) ttlInfo := tblInfo.TTLInfo @@ -365,7 +365,7 @@ func (g *TTLTimersSyncer) syncOneTimer(ctx context.Context, se session.Session, return timer, nil } -func getTimerTags(schema pmodel.CIStr, tblInfo *model.TableInfo, partition *model.PartitionDefinition) []string { +func getTimerTags(schema ast.CIStr, tblInfo *model.TableInfo, partition *model.PartitionDefinition) []string { dbTag := fmt.Sprintf("db=%s", schema.O) tblTag := fmt.Sprintf("table=%s", tblInfo.Name.O) if partition != nil { diff --git a/pkg/ttl/ttlworker/timer_sync_test.go b/pkg/ttl/ttlworker/timer_sync_test.go index 618f93bb6a80c..0dd58acfdd10a 100644 --- a/pkg/ttl/ttlworker/timer_sync_test.go +++ b/pkg/ttl/ttlworker/timer_sync_test.go @@ -25,7 +25,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/metrics" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/testkit" timerapi "github.com/pingcap/tidb/pkg/timer/api" "github.com/pingcap/tidb/pkg/timer/tablestore" @@ -455,17 +455,17 @@ func checkTimersNotChange(t *testing.T, cli timerapi.TimerClient, timers ...*tim func getPhysicalTableInfo(t *testing.T, do *domain.Domain, db, table, partition string) (string, *cache.PhysicalTable) { is := do.InfoSchema() - tbl, err := is.TableByName(context.Background(), model.NewCIStr(db), model.NewCIStr(table)) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr(db), ast.NewCIStr(table)) require.NoError(t, err) tblInfo := tbl.Meta() - physical, err := cache.NewPhysicalTable(model.NewCIStr(db), tblInfo, model.NewCIStr(partition)) + physical, err := cache.NewPhysicalTable(ast.NewCIStr(db), tblInfo, ast.NewCIStr(partition)) require.NoError(t, err) return fmt.Sprintf("/tidb/ttl/physical_table/%d/%d", tblInfo.ID, physical.ID), physical } func checkTimerWithTableMeta(t *testing.T, do *domain.Domain, cli timerapi.TimerClient, db, table, partition string, watermark time.Time) *timerapi.TimerRecord { is := do.InfoSchema() - dbInfo, ok := is.SchemaByName(model.NewCIStr(db)) + dbInfo, ok := is.SchemaByName(ast.NewCIStr(db)) require.True(t, ok) key, physical := getPhysicalTableInfo(t, do, db, table, partition) diff --git a/pkg/types/BUILD.bazel b/pkg/types/BUILD.bazel index 4fb334fc30a85..29dca59582a2c 100644 --- a/pkg/types/BUILD.bazel +++ b/pkg/types/BUILD.bazel @@ -48,7 +48,6 @@ go_library( "//pkg/errno", "//pkg/parser/ast", "//pkg/parser/charset", - "//pkg/parser/model", "//pkg/parser/mysql", "//pkg/parser/opcode", "//pkg/parser/terror", diff --git a/pkg/types/field_name.go b/pkg/types/field_name.go index 3ebf0f74fb616..74f90607a7a13 100644 --- a/pkg/types/field_name.go +++ b/pkg/types/field_name.go @@ -18,17 +18,16 @@ import ( "strings" "github.com/pingcap/tidb/pkg/parser/ast" - "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/util/size" ) // FieldName records the names used for mysql protocol. type FieldName struct { - OrigTblName model.CIStr - OrigColName model.CIStr - DBName model.CIStr - TblName model.CIStr - ColName model.CIStr + OrigTblName ast.CIStr + OrigColName ast.CIStr + DBName ast.CIStr + TblName ast.CIStr + ColName ast.CIStr Hidden bool diff --git a/pkg/util/BUILD.bazel b/pkg/util/BUILD.bazel index 23cd407573828..79b9e1d58c546 100644 --- a/pkg/util/BUILD.bazel +++ b/pkg/util/BUILD.bazel @@ -33,8 +33,8 @@ go_library( "//pkg/meta/model", "//pkg/metrics", "//pkg/parser", + "//pkg/parser/ast", "//pkg/parser/auth", - "//pkg/parser/model", "//pkg/parser/mysql", "//pkg/parser/terror", "//pkg/resourcegroup", @@ -89,7 +89,7 @@ go_test( "//pkg/kv", "//pkg/meta/model", "//pkg/parser", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/parser/mysql", "//pkg/parser/terror", "//pkg/sessionctx", diff --git a/pkg/util/admin/BUILD.bazel b/pkg/util/admin/BUILD.bazel index 0343daad14376..29abe75673b3a 100644 --- a/pkg/util/admin/BUILD.bazel +++ b/pkg/util/admin/BUILD.bazel @@ -9,7 +9,7 @@ go_library( "//pkg/errno", "//pkg/expression", "//pkg/kv", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/parser/mysql", "//pkg/sessionctx", "//pkg/table", diff --git a/pkg/util/admin/admin.go b/pkg/util/admin/admin.go index 2d7f870549a34..c9000d95d6c76 100644 --- a/pkg/util/admin/admin.go +++ b/pkg/util/admin/admin.go @@ -23,7 +23,7 @@ import ( "github.com/pingcap/tidb/pkg/errno" "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/kv" - "github.com/pingcap/tidb/pkg/parser/model" + "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/table" @@ -194,7 +194,7 @@ func CheckRecordAndIndex(ctx context.Context, sessCtx sessionctx.Context, txn kv } func makeRowDecoder(t table.Table, sctx sessionctx.Context) (*decoder.RowDecoder, error) { - dbName := model.NewCIStr(sctx.GetSessionVars().CurrentDB) + dbName := ast.NewCIStr(sctx.GetSessionVars().CurrentDB) exprCols, _, err := expression.ColumnInfos2ColumnsAndNames(sctx.GetExprCtx(), dbName, t.Meta().Name, t.Meta().Cols(), t.Meta()) if err != nil { return nil, err diff --git a/pkg/util/dbutil/dbutiltest/BUILD.bazel b/pkg/util/dbutil/dbutiltest/BUILD.bazel index 67914e45b010b..455cfcd851df9 100644 --- a/pkg/util/dbutil/dbutiltest/BUILD.bazel +++ b/pkg/util/dbutil/dbutiltest/BUILD.bazel @@ -11,7 +11,6 @@ go_library( "//pkg/meta/model", "//pkg/parser", "//pkg/parser/ast", - "//pkg/parser/model", "//pkg/planner/core", "//pkg/types", "@com_github_pingcap_errors//:errors", diff --git a/pkg/util/dbutil/dbutiltest/utils.go b/pkg/util/dbutil/dbutiltest/utils.go index 0cb5afadf230c..d1ad5e1376aad 100644 --- a/pkg/util/dbutil/dbutiltest/utils.go +++ b/pkg/util/dbutil/dbutiltest/utils.go @@ -23,7 +23,6 @@ import ( "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser" "github.com/pingcap/tidb/pkg/parser/ast" - pmodel "github.com/pingcap/tidb/pkg/parser/model" _ "github.com/pingcap/tidb/pkg/planner/core" // to setup expression.EvalAstExpr. See: https://github.com/pingcap/tidb/blob/a94cff903cd1e7f3b050db782da84273ef5592f4/planner/core/optimizer.go#L202 "github.com/pingcap/tidb/pkg/types" ) @@ -45,11 +44,11 @@ func GetTableInfoBySQL(createTableSQL string, parser2 *parser.Parser) (table *mo // put primary key in indices if table.PKIsHandle { pkIndex := &model.IndexInfo{ - Name: pmodel.NewCIStr("PRIMARY"), + Name: ast.NewCIStr("PRIMARY"), Primary: true, State: model.StatePublic, Unique: true, - Tp: pmodel.IndexTypeBtree, + Tp: ast.IndexTypeBtree, Columns: []*model.IndexColumn{ { Name: table.GetPkName(), diff --git a/pkg/util/deadlockhistory/BUILD.bazel b/pkg/util/deadlockhistory/BUILD.bazel index f9415cb09dd07..6af4f7dab78f2 100644 --- a/pkg/util/deadlockhistory/BUILD.bazel +++ b/pkg/util/deadlockhistory/BUILD.bazel @@ -27,7 +27,7 @@ go_test( deps = [ "//pkg/meta/model", "//pkg/parser", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/testkit/testsetup", "//pkg/types", "@com_github_pingcap_kvproto//pkg/deadlock", diff --git a/pkg/util/deadlockhistory/deadlock_history_test.go b/pkg/util/deadlockhistory/deadlock_history_test.go index b6ff479a8960d..fb9066cf3b066 100644 --- a/pkg/util/deadlockhistory/deadlock_history_test.go +++ b/pkg/util/deadlockhistory/deadlock_history_test.go @@ -22,7 +22,7 @@ import ( "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tipb/go-tipb" "github.com/stretchr/testify/assert" @@ -197,15 +197,15 @@ func TestGetDatum(t *testing.T) { }) dummyColumnInfo := []*model.ColumnInfo{ - {Name: pmodel.NewCIStr(ColDeadlockIDStr)}, - {Name: pmodel.NewCIStr(ColOccurTimeStr)}, - {Name: pmodel.NewCIStr(ColRetryableStr)}, - {Name: pmodel.NewCIStr(ColTryLockTrxIDStr)}, - {Name: pmodel.NewCIStr(ColCurrentSQLDigestStr)}, - {Name: pmodel.NewCIStr(ColCurrentSQLDigestTextStr)}, - {Name: pmodel.NewCIStr(ColKeyStr)}, - {Name: pmodel.NewCIStr(ColKeyInfoStr)}, - {Name: pmodel.NewCIStr(ColTrxHoldingLockStr)}, + {Name: ast.NewCIStr(ColDeadlockIDStr)}, + {Name: ast.NewCIStr(ColOccurTimeStr)}, + {Name: ast.NewCIStr(ColRetryableStr)}, + {Name: ast.NewCIStr(ColTryLockTrxIDStr)}, + {Name: ast.NewCIStr(ColCurrentSQLDigestStr)}, + {Name: ast.NewCIStr(ColCurrentSQLDigestTextStr)}, + {Name: ast.NewCIStr(ColKeyStr)}, + {Name: ast.NewCIStr(ColKeyInfoStr)}, + {Name: ast.NewCIStr(ColTrxHoldingLockStr)}, } res := getAllDatum(h, dummyColumnInfo) diff --git a/pkg/util/hint/BUILD.bazel b/pkg/util/hint/BUILD.bazel index 9d345f43d1ae3..2fed52261e498 100644 --- a/pkg/util/hint/BUILD.bazel +++ b/pkg/util/hint/BUILD.bazel @@ -15,7 +15,6 @@ go_library( "//pkg/parser", "//pkg/parser/ast", "//pkg/parser/format", - "//pkg/parser/model", "//pkg/types", "//pkg/util/dbterror", "//pkg/util/dbterror/plannererrors", diff --git a/pkg/util/hint/hint.go b/pkg/util/hint/hint.go index 02b240711ab80..88c40c7042b6c 100644 --- a/pkg/util/hint/hint.go +++ b/pkg/util/hint/hint.go @@ -25,7 +25,6 @@ import ( "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/format" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/dbterror" ) @@ -291,7 +290,7 @@ func (sh *StmtHints) addHypoIndex(db, tbl, idx string, idxInfo *model.IndexInfo) // ParseStmtHints parses statement hints. func ParseStmtHints(hints []*ast.TableOptimizerHint, setVarHintChecker func(varName, hint string) (ok bool, warning error), - hypoIndexChecker func(db, tbl, col pmodel.CIStr) (colOffset int, err error), + hypoIndexChecker func(db, tbl, col ast.CIStr) (colOffset int, err error), currentDB string, replicaReadFollower byte) ( // to avoid cycle import stmtHints StmtHints, offs []int, warns []error) { if len(hints) == 0 { @@ -344,12 +343,12 @@ func ParseStmtHints(hints []*ast.TableOptimizerHint, } tbl := hint.Tables[0].TableName idx := hint.Tables[1].TableName - var colNames []pmodel.CIStr + var colNames []ast.CIStr var cols []*model.IndexColumn invalid := false for i := 2; i < len(hint.Tables); i++ { colNames = append(colNames, hint.Tables[i].TableName) - offset, err := hypoIndexChecker(pmodel.NewCIStr(db), tbl, hint.Tables[i].TableName) + offset, err := hypoIndexChecker(ast.NewCIStr(db), tbl, hint.Tables[i].TableName) if err != nil { invalid = true warns = append(warns, errors.NewNoStackErrorf("invalid HYPO_INDEX hint: %v", err)) @@ -368,7 +367,7 @@ func ParseStmtHints(hints []*ast.TableOptimizerHint, Name: idx, Columns: cols, State: model.StatePublic, - Tp: pmodel.IndexTypeHypo, + Tp: ast.IndexTypeHypo, } stmtHints.addHypoIndex(db, tbl.L, idx.L, idxInfo) case "set_var": @@ -552,18 +551,18 @@ type PlanHints struct { // HintedTable indicates which table this hint should take effect on. type HintedTable struct { - DBName pmodel.CIStr // the database name - TblName pmodel.CIStr // the table name - Partitions []pmodel.CIStr // partition information - SelectOffset int // the select block offset of this hint - Matched bool // whether this hint is applied successfully + DBName ast.CIStr // the database name + TblName ast.CIStr // the table name + Partitions []ast.CIStr // partition information + SelectOffset int // the select block offset of this hint + Matched bool // whether this hint is applied successfully } // HintedIndex indicates which index this hint should take effect on. type HintedIndex struct { - DBName pmodel.CIStr // the database name - TblName pmodel.CIStr // the table name - Partitions []pmodel.CIStr // partition information + DBName ast.CIStr // the database name + TblName ast.CIStr // the table name + Partitions []ast.CIStr // partition information IndexHint *ast.IndexHint // the original parser index hint structure // Matched indicates whether this index hint // has been successfully applied to a DataSource. @@ -573,7 +572,7 @@ type HintedIndex struct { } // Match checks whether the hint is matched with the given dbName and tblName. -func (hint *HintedIndex) Match(dbName, tblName pmodel.CIStr) bool { +func (hint *HintedIndex) Match(dbName, tblName ast.CIStr) bool { return hint.TblName.L == tblName.L && (hint.DBName.L == dbName.L || hint.DBName.L == "*") // for universal bindings, e.g. *.t @@ -810,7 +809,7 @@ func ParsePlanHints(hints []*ast.TableOptimizerHint, case HintUseIndex, HintIgnoreIndex, HintForceIndex, HintOrderIndex, HintNoOrderIndex: dbName := hint.Tables[0].DBName if dbName.L == "" { - dbName = pmodel.NewCIStr(currentDB) + dbName = ast.NewCIStr(currentDB) } var hintType ast.IndexHintType switch hint.HintName.L { @@ -836,7 +835,7 @@ func ParsePlanHints(hints []*ast.TableOptimizerHint, }, }) case HintReadFromStorage: - switch hint.HintData.(pmodel.CIStr).L { + switch hint.HintData.(ast.CIStr).L { case HintTiFlash: tiflashTables = append(tiflashTables, tableNames2HintTableInfo(currentDB, hint.HintName.L, hint.Tables, hintProcessor, currentLevel, warnHandler)...) case HintTiKV: @@ -845,7 +844,7 @@ func ParsePlanHints(hints []*ast.TableOptimizerHint, case HintIndexMerge: dbName := hint.Tables[0].DBName if dbName.L == "" { - dbName = pmodel.NewCIStr(currentDB) + dbName = ast.NewCIStr(currentDB) } indexMergeHintList = append(indexMergeHintList, HintedIndex{ DBName: dbName, @@ -946,7 +945,7 @@ func tableNames2HintTableInfo(currentDB, hintName string, hintTables []ast.HintT return nil } hintTableInfos := make([]HintedTable, 0, len(hintTables)) - defaultDBName := pmodel.NewCIStr(currentDB) + defaultDBName := ast.NewCIStr(currentDB) isInapplicable := false for _, hintTable := range hintTables { tableInfo := HintedTable{ diff --git a/pkg/util/hint/hint_processor.go b/pkg/util/hint/hint_processor.go index d76ce6c2864b9..ed0b1810933a2 100644 --- a/pkg/util/hint/hint_processor.go +++ b/pkg/util/hint/hint_processor.go @@ -21,7 +21,6 @@ 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/parser/model" "github.com/pingcap/tidb/pkg/util/dbterror/plannererrors" "github.com/pingcap/tidb/pkg/util/logutil" "go.uber.org/zap" @@ -318,7 +317,7 @@ func ParseHintsSet(p *parser.Parser, sql, charset, collation, db string) (*Hints } for i, tbl := range tblHint.Tables { if tbl.DBName.String() == "" { - tblHint.Tables[i].DBName = model.NewCIStr(db) + tblHint.Tables[i].DBName = ast.NewCIStr(db) } } newHints = append(newHints, tblHint) @@ -387,7 +386,7 @@ func CheckBindingFromHistoryComplete(node ast.Node, hintStr string) (complete bo checker := bindableChecker{ complete: true, - tables: make(map[model.CIStr]struct{}, 2), + tables: make(map[ast.CIStr]struct{}, 2), } node.Accept(&checker) return checker.complete, checker.reason @@ -397,7 +396,7 @@ func CheckBindingFromHistoryComplete(node ast.Node, hintStr string) (complete bo type bindableChecker struct { complete bool reason string - tables map[model.CIStr]struct{} + tables map[ast.CIStr]struct{} } // Enter implements Visitor interface. diff --git a/pkg/util/hint/hint_query_block.go b/pkg/util/hint/hint_query_block.go index 99bfc646837e6..eb85910b59827 100644 --- a/pkg/util/hint/hint_query_block.go +++ b/pkg/util/hint/hint_query_block.go @@ -21,7 +21,6 @@ import ( "strings" "github.com/pingcap/tidb/pkg/parser/ast" - "github.com/pingcap/tidb/pkg/parser/model" ) // QBHintHandler is used to handle hints at different query blocks. @@ -149,7 +148,7 @@ func (p *QBHintHandler) handleViewHints(hints []*ast.TableOptimizerHint, offset // we should add the query block number where it is located to the first table in the view's qb_name hint table list. qbNum := hint.Tables[0].QBName.L if qbNum == "" { - hint.Tables[0].QBName = model.NewCIStr(fmt.Sprintf("%s%d", defaultSelectBlockPrefix, offset)) + hint.Tables[0].QBName = ast.NewCIStr(fmt.Sprintf("%s%d", defaultSelectBlockPrefix, offset)) } } p.ViewQBNameToTable[qbName] = hint.Tables @@ -221,7 +220,7 @@ const ( // getBlockName finds the offset of query block name. It uses 0 as offset for top level update or delete, // -1 for invalid block name. -func (p *QBHintHandler) getBlockOffset(blockName model.CIStr) int { +func (p *QBHintHandler) getBlockOffset(blockName ast.CIStr) int { if p.QBNameToSelOffset != nil { level, ok := p.QBNameToSelOffset[blockName.L] if ok { @@ -244,7 +243,7 @@ func (p *QBHintHandler) getBlockOffset(blockName model.CIStr) int { } // GetHintOffset gets the offset of stmt that the hints take effects. -func (p *QBHintHandler) GetHintOffset(qbName model.CIStr, currentOffset int) int { +func (p *QBHintHandler) GetHintOffset(qbName ast.CIStr, currentOffset int) int { if qbName.L != "" { return p.getBlockOffset(qbName) } @@ -304,15 +303,15 @@ func (p *QBHintHandler) GetCurrentStmtHints(hints []*ast.TableOptimizerHint, cur } // GenerateQBName builds QBName from offset. -func GenerateQBName(nodeType NodeType, qbOffset int) (model.CIStr, error) { +func GenerateQBName(nodeType NodeType, qbOffset int) (ast.CIStr, error) { if qbOffset == 0 { if nodeType == TypeDelete { - return model.NewCIStr(defaultDeleteBlockName), nil + return ast.NewCIStr(defaultDeleteBlockName), nil } if nodeType == TypeUpdate { - return model.NewCIStr(defaultUpdateBlockName), nil + return ast.NewCIStr(defaultUpdateBlockName), nil } - return model.NewCIStr(""), fmt.Errorf("Unexpected NodeType %d when block offset is 0", nodeType) + return ast.NewCIStr(""), fmt.Errorf("Unexpected NodeType %d when block offset is 0", nodeType) } - return model.NewCIStr(fmt.Sprintf("%s%d", defaultSelectBlockPrefix, qbOffset)), nil + return ast.NewCIStr(fmt.Sprintf("%s%d", defaultSelectBlockPrefix, qbOffset)), nil } diff --git a/pkg/util/keydecoder/BUILD.bazel b/pkg/util/keydecoder/BUILD.bazel index 8d7d53a5f03e2..f2929526feb64 100644 --- a/pkg/util/keydecoder/BUILD.bazel +++ b/pkg/util/keydecoder/BUILD.bazel @@ -28,7 +28,7 @@ go_test( deps = [ "//pkg/infoschema", "//pkg/meta/model", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/planner/core", "//pkg/sessionctx/stmtctx", "//pkg/table", diff --git a/pkg/util/keydecoder/keydecoder_test.go b/pkg/util/keydecoder/keydecoder_test.go index e37e674882bff..0f39b878e5b1c 100644 --- a/pkg/util/keydecoder/keydecoder_test.go +++ b/pkg/util/keydecoder/keydecoder_test.go @@ -19,7 +19,7 @@ import ( "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" _ "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" "github.com/pingcap/tidb/pkg/table" @@ -36,28 +36,28 @@ func TestDecodeKey(t *testing.T) { table.MockTableFromMeta = tables.MockTableFromMeta tableInfo1 := &model.TableInfo{ ID: 1, - Name: pmodel.NewCIStr("table1"), + Name: ast.NewCIStr("table1"), Indices: []*model.IndexInfo{ - {ID: 1, Name: pmodel.NewCIStr("index1"), State: model.StatePublic}, + {ID: 1, Name: ast.NewCIStr("index1"), State: model.StatePublic}, }, } - tableInfo2 := &model.TableInfo{ID: 2, Name: pmodel.NewCIStr("table2")} + tableInfo2 := &model.TableInfo{ID: 2, Name: ast.NewCIStr("table2")} tableInfo3 := &model.TableInfo{ ID: 3, - Name: pmodel.NewCIStr("table3"), + Name: ast.NewCIStr("table3"), Columns: []*model.ColumnInfo{ - {ID: 10, Name: pmodel.NewCIStr("col"), State: model.StatePublic}, + {ID: 10, Name: ast.NewCIStr("col"), State: model.StatePublic}, }, Indices: []*model.IndexInfo{ - {ID: 4, Name: pmodel.NewCIStr("index4"), State: model.StatePublic}, + {ID: 4, Name: ast.NewCIStr("index4"), State: model.StatePublic}, }, Partition: &model.PartitionInfo{ - Type: pmodel.PartitionTypeRange, + Type: ast.PartitionTypeRange, Expr: "`col`", Enable: true, Definitions: []model.PartitionDefinition{ - {ID: 5, Name: pmodel.NewCIStr("p0"), LessThan: []string{"10"}}, - {ID: 6, Name: pmodel.NewCIStr("p1"), LessThan: []string{"MAXVALUE"}}, + {ID: 5, Name: ast.NewCIStr("p0"), LessThan: []string{"10"}}, + {ID: 6, Name: ast.NewCIStr("p1"), LessThan: []string{"MAXVALUE"}}, }, }, } diff --git a/pkg/util/misc.go b/pkg/util/misc.go index 3bb68aae20db4..c8c536111ce2a 100644 --- a/pkg/util/misc.go +++ b/pkg/util/misc.go @@ -43,7 +43,7 @@ import ( "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/metrics" "github.com/pingcap/tidb/pkg/parser" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/types" @@ -181,11 +181,11 @@ func SyntaxWarn(err error) error { var ( // InformationSchemaName is the `INFORMATION_SCHEMA` database name. - InformationSchemaName = pmodel.NewCIStr("INFORMATION_SCHEMA") + InformationSchemaName = ast.NewCIStr("INFORMATION_SCHEMA") // PerformanceSchemaName is the `PERFORMANCE_SCHEMA` database name. - PerformanceSchemaName = pmodel.NewCIStr("PERFORMANCE_SCHEMA") + PerformanceSchemaName = ast.NewCIStr("PERFORMANCE_SCHEMA") // MetricSchemaName is the `METRICS_SCHEMA` database name. - MetricSchemaName = pmodel.NewCIStr("METRICS_SCHEMA") + MetricSchemaName = ast.NewCIStr("METRICS_SCHEMA") // ClusterTableInstanceColumnName is the `INSTANCE` column name of the cluster table. ClusterTableInstanceColumnName = "INSTANCE" ) @@ -450,7 +450,7 @@ func init() { } // GetSequenceByName could be used in expression package without import cycle problem. -var GetSequenceByName func(is infoschema.MetaOnlyInfoSchema, schema, sequence pmodel.CIStr) (SequenceTable, error) +var GetSequenceByName func(is infoschema.MetaOnlyInfoSchema, schema, sequence ast.CIStr) (SequenceTable, error) // SequenceTable is implemented by tableCommon, // and it is specialised in handling sequence operation. diff --git a/pkg/util/misc_test.go b/pkg/util/misc_test.go index 4908ccd50667a..4b5aecff46c78 100644 --- a/pkg/util/misc_test.go +++ b/pkg/util/misc_test.go @@ -23,7 +23,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" @@ -156,7 +156,7 @@ func TestBasicFuncRandomBuf(t *testing.T) { func TestToPB(t *testing.T) { column := &model.ColumnInfo{ ID: 1, - Name: pmodel.NewCIStr("c"), + Name: ast.NewCIStr("c"), Offset: 0, DefaultValue: 0, FieldType: *types.NewFieldType(0), @@ -166,7 +166,7 @@ func TestToPB(t *testing.T) { column2 := &model.ColumnInfo{ ID: 1, - Name: pmodel.NewCIStr("c"), + Name: ast.NewCIStr("c"), Offset: 0, DefaultValue: 0, FieldType: *types.NewFieldType(0), diff --git a/pkg/util/mock/BUILD.bazel b/pkg/util/mock/BUILD.bazel index 28c64a80ad7df..1816beac7c6f2 100644 --- a/pkg/util/mock/BUILD.bazel +++ b/pkg/util/mock/BUILD.bazel @@ -21,7 +21,6 @@ go_library( "//pkg/kv", "//pkg/meta/model", "//pkg/parser/ast", - "//pkg/parser/model", "//pkg/parser/terror", "//pkg/planner/core/resolve", "//pkg/planner/planctx", diff --git a/pkg/util/mock/context.go b/pkg/util/mock/context.go index 14ed7fc40e911..3248c7e18bd27 100644 --- a/pkg/util/mock/context.go +++ b/pkg/util/mock/context.go @@ -32,7 +32,6 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/ast" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/planner/core/resolve" "github.com/pingcap/tidb/pkg/planner/planctx" @@ -519,8 +518,8 @@ func (*Context) ReleaseTableLockByTableIDs(_ []int64) { } // CheckTableLocked implements the sessionctx.Context interface. -func (*Context) CheckTableLocked(_ int64) (bool, pmodel.TableLockType) { - return false, pmodel.TableLockNone +func (*Context) CheckTableLocked(_ int64) (bool, ast.TableLockType) { + return false, ast.TableLockNone } // GetAllTableLocks implements the sessionctx.Context interface. diff --git a/pkg/util/ranger/BUILD.bazel b/pkg/util/ranger/BUILD.bazel index dc4d0a202d6fe..bdc4f0003877a 100644 --- a/pkg/util/ranger/BUILD.bazel +++ b/pkg/util/ranger/BUILD.bazel @@ -54,7 +54,6 @@ go_test( "//pkg/expression", "//pkg/expression/exprstatic", "//pkg/parser/ast", - "//pkg/parser/model", "//pkg/parser/mysql", "//pkg/planner/core", "//pkg/planner/core/base", diff --git a/pkg/util/ranger/ranger_test.go b/pkg/util/ranger/ranger_test.go index 46ae80fa73008..10410b619c280 100644 --- a/pkg/util/ranger/ranger_test.go +++ b/pkg/util/ranger/ranger_test.go @@ -24,7 +24,6 @@ import ( "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/expression/exprstatic" "github.com/pingcap/tidb/pkg/parser/ast" - "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" plannercore "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/planner/core/base" @@ -1903,7 +1902,7 @@ func TestRangeFallbackForDetachCondAndBuildRangeForIndex(t *testing.T) { tk.MustExec("use test") tk.MustExec("drop table if exists t1") tk.MustExec("create table t1 (a int, b int, c int, d int, index idx(a, b, c))") - tbl, err := dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t1")) + tbl, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t1")) require.NoError(t, err) tblInfo := tbl.Meta() sctx := tk.Session() @@ -2032,7 +2031,7 @@ func TestRangeFallbackForDetachCondAndBuildRangeForIndex(t *testing.T) { // test prefix index tk.MustExec("drop table if exists t2") tk.MustExec("create table t2 (a varchar(10), b varchar(10), c varchar(10), d varchar(10), index idx(a(2), b(2), c(2)))") - tbl, err = dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t2")) + tbl, err = dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t2")) require.NoError(t, err) tblInfo = tbl.Meta() @@ -2146,7 +2145,7 @@ func TestRangeFallbackForBuildTableRange(t *testing.T) { tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t (a int primary key, b int)") - tbl, err := dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tblInfo := tbl.Meta() sctx := tk.Session() @@ -2182,7 +2181,7 @@ func TestRangeFallbackForBuildColumnRange(t *testing.T) { tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t (a varchar(20), b int not null)") - tbl, err := dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + tbl, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) tblInfo := tbl.Meta() sctx := tk.Session() diff --git a/pkg/util/rowDecoder/BUILD.bazel b/pkg/util/rowDecoder/BUILD.bazel index 563a3ffa4ef52..9a6aa73fc6e02 100644 --- a/pkg/util/rowDecoder/BUILD.bazel +++ b/pkg/util/rowDecoder/BUILD.bazel @@ -32,7 +32,7 @@ go_test( "//pkg/expression", "//pkg/kv", "//pkg/meta/model", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/parser/mysql", "//pkg/planner/core", "//pkg/sessionctx/stmtctx", diff --git a/pkg/util/rowDecoder/decoder_test.go b/pkg/util/rowDecoder/decoder_test.go index 152e8387a394d..00373b8dd3f98 100644 --- a/pkg/util/rowDecoder/decoder_test.go +++ b/pkg/util/rowDecoder/decoder_test.go @@ -21,7 +21,7 @@ import ( "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" _ "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" @@ -39,13 +39,13 @@ import ( func TestRowDecoder(t *testing.T) { defer view.Stop() - c1 := &model.ColumnInfo{ID: 1, Name: pmodel.NewCIStr("c1"), State: model.StatePublic, Offset: 0, FieldType: *types.NewFieldType(mysql.TypeLonglong)} - c2 := &model.ColumnInfo{ID: 2, Name: pmodel.NewCIStr("c2"), State: model.StatePublic, Offset: 1, FieldType: *types.NewFieldType(mysql.TypeVarchar)} - c3 := &model.ColumnInfo{ID: 3, Name: pmodel.NewCIStr("c3"), State: model.StatePublic, Offset: 2, FieldType: *types.NewFieldType(mysql.TypeNewDecimal)} - c4 := &model.ColumnInfo{ID: 4, Name: pmodel.NewCIStr("c4"), State: model.StatePublic, Offset: 3, FieldType: *types.NewFieldType(mysql.TypeTimestamp)} - c5 := &model.ColumnInfo{ID: 5, Name: pmodel.NewCIStr("c5"), State: model.StatePublic, Offset: 4, FieldType: *types.NewFieldType(mysql.TypeDuration), OriginDefaultValue: "02:00:02"} - c6 := &model.ColumnInfo{ID: 6, Name: pmodel.NewCIStr("c6"), State: model.StatePublic, Offset: 5, FieldType: *types.NewFieldType(mysql.TypeTimestamp), GeneratedExprString: "c4+c5"} - c7 := &model.ColumnInfo{ID: 7, Name: pmodel.NewCIStr("c7"), State: model.StatePublic, Offset: 6, FieldType: *types.NewFieldType(mysql.TypeLonglong)} + c1 := &model.ColumnInfo{ID: 1, Name: ast.NewCIStr("c1"), State: model.StatePublic, Offset: 0, FieldType: *types.NewFieldType(mysql.TypeLonglong)} + c2 := &model.ColumnInfo{ID: 2, Name: ast.NewCIStr("c2"), State: model.StatePublic, Offset: 1, FieldType: *types.NewFieldType(mysql.TypeVarchar)} + c3 := &model.ColumnInfo{ID: 3, Name: ast.NewCIStr("c3"), State: model.StatePublic, Offset: 2, FieldType: *types.NewFieldType(mysql.TypeNewDecimal)} + c4 := &model.ColumnInfo{ID: 4, Name: ast.NewCIStr("c4"), State: model.StatePublic, Offset: 3, FieldType: *types.NewFieldType(mysql.TypeTimestamp)} + c5 := &model.ColumnInfo{ID: 5, Name: ast.NewCIStr("c5"), State: model.StatePublic, Offset: 4, FieldType: *types.NewFieldType(mysql.TypeDuration), OriginDefaultValue: "02:00:02"} + c6 := &model.ColumnInfo{ID: 6, Name: ast.NewCIStr("c6"), State: model.StatePublic, Offset: 5, FieldType: *types.NewFieldType(mysql.TypeTimestamp), GeneratedExprString: "c4+c5"} + c7 := &model.ColumnInfo{ID: 7, Name: ast.NewCIStr("c7"), State: model.StatePublic, Offset: 6, FieldType: *types.NewFieldType(mysql.TypeLonglong)} c7.AddFlag(mysql.PriKeyFlag) cols := []*model.ColumnInfo{c1, c2, c3, c4, c5, c6, c7} @@ -149,14 +149,14 @@ func TestRowDecoder(t *testing.T) { func TestClusterIndexRowDecoder(t *testing.T) { defer view.Stop() - c1 := &model.ColumnInfo{ID: 1, Name: pmodel.NewCIStr("c1"), State: model.StatePublic, Offset: 0, FieldType: *types.NewFieldType(mysql.TypeLonglong)} - c2 := &model.ColumnInfo{ID: 2, Name: pmodel.NewCIStr("c2"), State: model.StatePublic, Offset: 1, FieldType: *types.NewFieldType(mysql.TypeVarchar)} - c3 := &model.ColumnInfo{ID: 3, Name: pmodel.NewCIStr("c3"), State: model.StatePublic, Offset: 2, FieldType: *types.NewFieldType(mysql.TypeNewDecimal)} + c1 := &model.ColumnInfo{ID: 1, Name: ast.NewCIStr("c1"), State: model.StatePublic, Offset: 0, FieldType: *types.NewFieldType(mysql.TypeLonglong)} + c2 := &model.ColumnInfo{ID: 2, Name: ast.NewCIStr("c2"), State: model.StatePublic, Offset: 1, FieldType: *types.NewFieldType(mysql.TypeVarchar)} + c3 := &model.ColumnInfo{ID: 3, Name: ast.NewCIStr("c3"), State: model.StatePublic, Offset: 2, FieldType: *types.NewFieldType(mysql.TypeNewDecimal)} c1.AddFlag(mysql.PriKeyFlag) c2.AddFlag(mysql.PriKeyFlag) - pk := &model.IndexInfo{ID: 1, Name: pmodel.NewCIStr("primary"), State: model.StatePublic, Primary: true, Columns: []*model.IndexColumn{ - {Name: pmodel.NewCIStr("c1"), Offset: 0}, - {Name: pmodel.NewCIStr("c2"), Offset: 1}, + pk := &model.IndexInfo{ID: 1, Name: ast.NewCIStr("primary"), State: model.StatePublic, Primary: true, Columns: []*model.IndexColumn{ + {Name: ast.NewCIStr("c1"), Offset: 0}, + {Name: ast.NewCIStr("c2"), Offset: 1}, }} cols := []*model.ColumnInfo{c1, c2, c3} diff --git a/pkg/util/schemacmp/BUILD.bazel b/pkg/util/schemacmp/BUILD.bazel index 3a3087abfa191..7013bdefa7309 100644 --- a/pkg/util/schemacmp/BUILD.bazel +++ b/pkg/util/schemacmp/BUILD.bazel @@ -12,9 +12,9 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/meta/model", + "//pkg/parser/ast", "//pkg/parser/charset", "//pkg/parser/format", - "//pkg/parser/model", "//pkg/parser/mysql", "//pkg/parser/types", "//pkg/types", diff --git a/pkg/util/schemacmp/table.go b/pkg/util/schemacmp/table.go index e0135c36305c8..056a0cf887b87 100644 --- a/pkg/util/schemacmp/table.go +++ b/pkg/util/schemacmp/table.go @@ -20,8 +20,8 @@ import ( "strings" "github.com/pingcap/tidb/pkg/meta/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/format" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/types" ) @@ -115,7 +115,7 @@ func encodeImplicitPrimaryKeyToLattice(ci *model.ColumnInfo) Tuple { EqualitySingleton(indexColumnSlice{indexColumn{colName: ci.Name.L, length: types.UnspecifiedLength}}), Bool(false), Bool(false), - Singleton(pmodel.IndexTypeBtree), + Singleton(ast.IndexTypeBtree), } } @@ -133,7 +133,7 @@ func restoreIndexInfoFromUnwrapped(ctx *format.RestoreCtx, index []any, keyName ctx.WriteName(keyName) } - if tp := index[indexInfoTupleIndexType].(pmodel.IndexType); tp != pmodel.IndexTypeBtree { + if tp := index[indexInfoTupleIndexType].(ast.IndexType); tp != ast.IndexTypeBtree { ctx.WriteKeyWord(" USING ") ctx.WriteKeyWord(tp.String()) } diff --git a/pkg/util/stmtsummary/BUILD.bazel b/pkg/util/stmtsummary/BUILD.bazel index 675df9fd991e5..6506fb5f0900b 100644 --- a/pkg/util/stmtsummary/BUILD.bazel +++ b/pkg/util/stmtsummary/BUILD.bazel @@ -43,8 +43,8 @@ go_test( shard_count = 24, deps = [ "//pkg/meta/model", + "//pkg/parser/ast", "//pkg/parser/auth", - "//pkg/parser/model", "//pkg/parser/mysql", "//pkg/sessionctx/stmtctx", "//pkg/testkit/testsetup", diff --git a/pkg/util/stmtsummary/statement_summary_test.go b/pkg/util/stmtsummary/statement_summary_test.go index d7d773befcdb1..bf7a546888c55 100644 --- a/pkg/util/stmtsummary/statement_summary_test.go +++ b/pkg/util/stmtsummary/statement_summary_test.go @@ -24,8 +24,8 @@ import ( "time" "github.com/pingcap/tidb/pkg/meta/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/auth" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" "github.com/pingcap/tidb/pkg/types" @@ -838,7 +838,7 @@ func newStmtSummaryReaderForTest(ssMap *stmtSummaryByDigestMap) *stmtSummaryRead for i := range columnNames { cols[i] = &model.ColumnInfo{ ID: int64(i), - Name: pmodel.NewCIStr(columnNames[i]), + Name: ast.NewCIStr(columnNames[i]), Offset: i, } } diff --git a/pkg/util/stmtsummary/v2/BUILD.bazel b/pkg/util/stmtsummary/v2/BUILD.bazel index 3a2d48bae7f0d..d2089efa40b77 100644 --- a/pkg/util/stmtsummary/v2/BUILD.bazel +++ b/pkg/util/stmtsummary/v2/BUILD.bazel @@ -50,8 +50,8 @@ go_test( shard_count = 13, deps = [ "//pkg/meta/model", + "//pkg/parser/ast", "//pkg/parser/auth", - "//pkg/parser/model", "//pkg/testkit/testsetup", "//pkg/types", "//pkg/util", diff --git a/pkg/util/stmtsummary/v2/column_test.go b/pkg/util/stmtsummary/v2/column_test.go index 6607f19d856a7..dd07fd5e22077 100644 --- a/pkg/util/stmtsummary/v2/column_test.go +++ b/pkg/util/stmtsummary/v2/column_test.go @@ -20,25 +20,25 @@ import ( "time" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/stretchr/testify/require" ) func TestColumn(t *testing.T) { columns := []*model.ColumnInfo{ - {Name: pmodel.NewCIStr(ClusterTableInstanceColumnNameStr)}, - {Name: pmodel.NewCIStr(StmtTypeStr)}, - {Name: pmodel.NewCIStr(SchemaNameStr)}, - {Name: pmodel.NewCIStr(DigestStr)}, - {Name: pmodel.NewCIStr(DigestTextStr)}, - {Name: pmodel.NewCIStr(TableNamesStr)}, - {Name: pmodel.NewCIStr(IndexNamesStr)}, - {Name: pmodel.NewCIStr(SampleUserStr)}, - {Name: pmodel.NewCIStr(ExecCountStr)}, - {Name: pmodel.NewCIStr(SumLatencyStr)}, - {Name: pmodel.NewCIStr(MaxLatencyStr)}, - {Name: pmodel.NewCIStr(AvgTidbCPUTimeStr)}, - {Name: pmodel.NewCIStr(AvgTikvCPUTimeStr)}, + {Name: ast.NewCIStr(ClusterTableInstanceColumnNameStr)}, + {Name: ast.NewCIStr(StmtTypeStr)}, + {Name: ast.NewCIStr(SchemaNameStr)}, + {Name: ast.NewCIStr(DigestStr)}, + {Name: ast.NewCIStr(DigestTextStr)}, + {Name: ast.NewCIStr(TableNamesStr)}, + {Name: ast.NewCIStr(IndexNamesStr)}, + {Name: ast.NewCIStr(SampleUserStr)}, + {Name: ast.NewCIStr(ExecCountStr)}, + {Name: ast.NewCIStr(SumLatencyStr)}, + {Name: ast.NewCIStr(MaxLatencyStr)}, + {Name: ast.NewCIStr(AvgTidbCPUTimeStr)}, + {Name: ast.NewCIStr(AvgTikvCPUTimeStr)}, } factories := makeColumnFactories(columns) info := GenerateStmtExecInfo4Test("digest") diff --git a/pkg/util/stmtsummary/v2/reader_test.go b/pkg/util/stmtsummary/v2/reader_test.go index cf5dded90c7ad..906fc69d2d2a0 100644 --- a/pkg/util/stmtsummary/v2/reader_test.go +++ b/pkg/util/stmtsummary/v2/reader_test.go @@ -23,8 +23,8 @@ import ( "time" "github.com/pingcap/tidb/pkg/meta/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/auth" - pmodel "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util" "github.com/pingcap/tidb/pkg/util/set" @@ -228,8 +228,8 @@ func TestMemReader(t *testing.T) { timeLocation, err := time.LoadLocation("Asia/Shanghai") require.NoError(t, err) columns := []*model.ColumnInfo{ - {Name: pmodel.NewCIStr(DigestStr)}, - {Name: pmodel.NewCIStr(ExecCountStr)}, + {Name: ast.NewCIStr(DigestStr)}, + {Name: ast.NewCIStr(ExecCountStr)}, } ss := NewStmtSummary4Test(3) @@ -282,8 +282,8 @@ func TestHistoryReader(t *testing.T) { timeLocation, err := time.LoadLocation("Asia/Shanghai") require.NoError(t, err) columns := []*model.ColumnInfo{ - {Name: pmodel.NewCIStr(DigestStr)}, - {Name: pmodel.NewCIStr(ExecCountStr)}, + {Name: ast.NewCIStr(DigestStr)}, + {Name: ast.NewCIStr(ExecCountStr)}, } func() { @@ -430,8 +430,8 @@ func TestHistoryReaderInvalidLine(t *testing.T) { timeLocation, err := time.LoadLocation("Asia/Shanghai") require.NoError(t, err) columns := []*model.ColumnInfo{ - {Name: pmodel.NewCIStr(DigestStr)}, - {Name: pmodel.NewCIStr(ExecCountStr)}, + {Name: ast.NewCIStr(DigestStr)}, + {Name: ast.NewCIStr(ExecCountStr)}, } reader, err := NewHistoryReader(context.Background(), columns, "", timeLocation, nil, false, nil, nil, 2) diff --git a/pkg/util/workloadrepo/BUILD.bazel b/pkg/util/workloadrepo/BUILD.bazel index b06c9826ee5f0..2edc49364790d 100644 --- a/pkg/util/workloadrepo/BUILD.bazel +++ b/pkg/util/workloadrepo/BUILD.bazel @@ -21,7 +21,7 @@ go_library( "//pkg/kv", "//pkg/meta/model", "//pkg/owner", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/sessionctx", "//pkg/sessionctx/variable", "//pkg/sessiontxn", @@ -49,7 +49,7 @@ go_test( "//pkg/domain", "//pkg/kv", "//pkg/owner", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/testkit", "@com_github_stretchr_testify//require", "@io_etcd_go_etcd_client_v3//:client", diff --git a/pkg/util/workloadrepo/const.go b/pkg/util/workloadrepo/const.go index 8a2b4d0eb0e24..a51ec10540ce5 100644 --- a/pkg/util/workloadrepo/const.go +++ b/pkg/util/workloadrepo/const.go @@ -17,7 +17,7 @@ package workloadrepo import ( "time" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" ) const ( @@ -41,6 +41,6 @@ const ( ) var ( - workloadSchemaCIStr = model.NewCIStr(WorkloadSchema) + workloadSchemaCIStr = ast.NewCIStr(WorkloadSchema) zeroTime = time.Time{} ) diff --git a/pkg/util/workloadrepo/housekeeper.go b/pkg/util/workloadrepo/housekeeper.go index 3ef613de25e2e..a6be07c835b93 100644 --- a/pkg/util/workloadrepo/housekeeper.go +++ b/pkg/util/workloadrepo/housekeeper.go @@ -21,7 +21,7 @@ import ( "time" "github.com/pingcap/tidb/pkg/infoschema" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessiontxn" "github.com/pingcap/tidb/pkg/util/logutil" @@ -41,7 +41,7 @@ func calcNextTick(now time.Time) time.Duration { func createAllPartitions(ctx context.Context, sess sessionctx.Context, is infoschema.InfoSchema) error { sb := &strings.Builder{} for _, tbl := range workloadTables { - tbSchema, err := is.TableByName(ctx, workloadSchemaCIStr, model.NewCIStr(tbl.destTable)) + tbSchema, err := is.TableByName(ctx, workloadSchemaCIStr, ast.NewCIStr(tbl.destTable)) if err != nil { logutil.BgLogger().Info("workload repository cannot get table", zap.String("tbl", tbl.destTable), zap.NamedError("err", err)) return err @@ -74,7 +74,7 @@ func (w *worker) dropOldPartitions(ctx context.Context, sess sessionctx.Context, sb := &strings.Builder{} for _, tbl := range workloadTables { - tbSchema, err := is.TableByName(ctx, workloadSchemaCIStr, model.NewCIStr(tbl.destTable)) + tbSchema, err := is.TableByName(ctx, workloadSchemaCIStr, ast.NewCIStr(tbl.destTable)) if err != nil { logutil.BgLogger().Info("workload repository cannot get table", zap.String("tbl", tbl.destTable), zap.NamedError("err", err)) continue diff --git a/pkg/util/workloadrepo/table.go b/pkg/util/workloadrepo/table.go index f2744b9b753b3..d3c88a84bb972 100644 --- a/pkg/util/workloadrepo/table.go +++ b/pkg/util/workloadrepo/table.go @@ -22,7 +22,7 @@ import ( "time" "github.com/pingcap/tidb/pkg/infoschema" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessiontxn" "github.com/pingcap/tidb/pkg/util/slice" @@ -31,7 +31,7 @@ import ( func buildCreateQuery(ctx context.Context, sess sessionctx.Context, rt *repositoryTable) (string, error) { is := sessiontxn.GetTxnManager(sess).GetTxnInfoSchema() - tbl, err := is.TableByName(ctx, model.NewCIStr(rt.schema), model.NewCIStr(rt.table)) + tbl, err := is.TableByName(ctx, ast.NewCIStr(rt.schema), ast.NewCIStr(rt.table)) if err != nil { return "", err } @@ -59,7 +59,7 @@ func buildCreateQuery(ctx context.Context, sess sessionctx.Context, rt *reposito func buildInsertQuery(ctx context.Context, sess sessionctx.Context, rt *repositoryTable) error { is := sessiontxn.GetTxnManager(sess).GetTxnInfoSchema() - tbl, err := is.TableByName(ctx, model.NewCIStr(rt.schema), model.NewCIStr(rt.table)) + tbl, err := is.TableByName(ctx, ast.NewCIStr(rt.schema), ast.NewCIStr(rt.table)) if err != nil { return err } @@ -159,11 +159,11 @@ func (w *worker) checkTablesExists(ctx context.Context) bool { func checkTableExistsByIS(ctx context.Context, is infoschema.InfoSchema, tblName string, now time.Time) bool { if now == zeroTime { - return is.TableExists(workloadSchemaCIStr, model.NewCIStr(tblName)) + return is.TableExists(workloadSchemaCIStr, ast.NewCIStr(tblName)) } // check for partitions, too - tbSchema, err := is.TableByName(ctx, workloadSchemaCIStr, model.NewCIStr(tblName)) + tbSchema, err := is.TableByName(ctx, workloadSchemaCIStr, ast.NewCIStr(tblName)) if err != nil { return false } diff --git a/pkg/util/workloadrepo/worker_test.go b/pkg/util/workloadrepo/worker_test.go index 578ae5c6f1b77..dc5e12e350dc5 100644 --- a/pkg/util/workloadrepo/worker_test.go +++ b/pkg/util/workloadrepo/worker_test.go @@ -23,7 +23,7 @@ import ( "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/owner" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/testkit" "github.com/stretchr/testify/require" clientv3 "go.etcd.io/etcd/client/v3" @@ -109,7 +109,7 @@ func setupWorker(ctx context.Context, t *testing.T, addr string, dom *domain.Dom func TestMultipleWorker(t *testing.T) { ctx, store, dom, addr := setupDomainAndContext(t) - _, ok := dom.InfoSchema().SchemaByName(model.NewCIStr("workload_schema")) + _, ok := dom.InfoSchema().SchemaByName(ast.NewCIStr("workload_schema")) require.False(t, ok) wrk1 := setupWorker(ctx, t, addr, dom, "worker1", true) @@ -153,7 +153,7 @@ func TestGlobalWorker(t *testing.T) { ctx, store, dom, addr := setupDomainAndContext(t) tk := testkit.NewTestKit(t, store) - _, ok := dom.InfoSchema().SchemaByName(model.NewCIStr("workload_schema")) + _, ok := dom.InfoSchema().SchemaByName(ast.NewCIStr("workload_schema")) require.False(t, ok) wrk := setupWorker(ctx, t, addr, dom, "worker", false) @@ -174,7 +174,7 @@ func TestAdminWorkloadRepo(t *testing.T) { ctx, store, dom, addr := setupDomainAndContext(t) tk := testkit.NewTestKit(t, store) - _, ok := dom.InfoSchema().SchemaByName(model.NewCIStr("workload_schema")) + _, ok := dom.InfoSchema().SchemaByName(ast.NewCIStr("workload_schema")) require.False(t, ok) wrk := setupWorker(ctx, t, addr, dom, "worker", false) diff --git a/tests/realtikvtest/addindextest3/BUILD.bazel b/tests/realtikvtest/addindextest3/BUILD.bazel index 93c92251ebc6e..6af6262846d00 100644 --- a/tests/realtikvtest/addindextest3/BUILD.bazel +++ b/tests/realtikvtest/addindextest3/BUILD.bazel @@ -22,7 +22,7 @@ go_test( "//pkg/kv", "//pkg/lightning/backend/local", "//pkg/meta/model", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/sessionctx", "//pkg/sessionctx/variable", "//pkg/table", diff --git a/tests/realtikvtest/addindextest3/functional_test.go b/tests/realtikvtest/addindextest3/functional_test.go index f23817713fd7a..ab1295499fd58 100644 --- a/tests/realtikvtest/addindextest3/functional_test.go +++ b/tests/realtikvtest/addindextest3/functional_test.go @@ -25,7 +25,7 @@ import ( "github.com/pingcap/tidb/pkg/ddl" "github.com/pingcap/tidb/pkg/ddl/ingest" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/table/tables" "github.com/pingcap/tidb/pkg/tablecodec" "github.com/pingcap/tidb/pkg/testkit" @@ -47,7 +47,7 @@ func TestDDLTestEstimateTableRowSize(t *testing.T) { ctx = util.WithInternalSourceType(ctx, "estimate_row_size") tkSess := tk.Session() exec := tkSess.GetRestrictedSQLExecutor() - tbl, err := dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) size := ddl.EstimateTableRowSizeForTest(ctx, store, exec, tbl) @@ -71,7 +71,7 @@ func TestDDLTestEstimateTableRowSize(t *testing.T) { tk.MustQuery("split table t between (0) and (1000000) regions 2;").Check(testkit.Rows("4 1")) tk.MustExec("set global tidb_analyze_skip_column_types=`json,blob,mediumblob,longblob`") tk.MustExec("analyze table t all columns;") - tbl, err = dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("test"), pmodel.NewCIStr("t")) + tbl, err = dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) size = ddl.EstimateTableRowSizeForTest(ctx, store, exec, tbl) require.Equal(t, 19, size) diff --git a/tests/realtikvtest/addindextest3/operator_test.go b/tests/realtikvtest/addindextest3/operator_test.go index a6b67ee79279a..9ff1b034549bb 100644 --- a/tests/realtikvtest/addindextest3/operator_test.go +++ b/tests/realtikvtest/addindextest3/operator_test.go @@ -33,7 +33,7 @@ import ( "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta/model" - pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/table/tables" @@ -336,7 +336,7 @@ func prepare(t *testing.T, tk *testkit.TestKit, dom *domain.Domain, regionCnt in tk.MustQuery("select count(*) from t;").Check(testkit.Rows(fmt.Sprintf("%d", regionCnt))) var err error - tbl, err = dom.InfoSchema().TableByName(context.Background(), pmodel.NewCIStr("op"), pmodel.NewCIStr("t")) + tbl, err = dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("op"), ast.NewCIStr("t")) require.NoError(t, err) start = tbl.RecordPrefix() end = tbl.RecordPrefix().PrefixNext() diff --git a/tests/realtikvtest/importintotest2/BUILD.bazel b/tests/realtikvtest/importintotest2/BUILD.bazel index 3d44979293394..c94e163bac71b 100644 --- a/tests/realtikvtest/importintotest2/BUILD.bazel +++ b/tests/realtikvtest/importintotest2/BUILD.bazel @@ -18,7 +18,7 @@ go_test( "//pkg/infoschema", "//pkg/kv", "//pkg/lightning/common", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/testkit", "//pkg/testkit/testfailpoint", "//pkg/util/dbterror/plannererrors", diff --git a/tests/realtikvtest/importintotest2/write_after_import_test.go b/tests/realtikvtest/importintotest2/write_after_import_test.go index aa3e8be81a372..29b6db3c296be 100644 --- a/tests/realtikvtest/importintotest2/write_after_import_test.go +++ b/tests/realtikvtest/importintotest2/write_after_import_test.go @@ -24,7 +24,7 @@ import ( "github.com/pingcap/tidb/pkg/executor/importer" "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/lightning/common" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/testkit" ) @@ -228,9 +228,9 @@ func (s *mockGCSSuite) testWriteAfterImport(importSQL string, sourceType importe s.tk.MustQuery(querySQL).Check(testkit.Rows(allData...)) is := s.tk.Session().GetDomainInfoSchema().(infoschema.InfoSchema) - dbInfo, ok := is.SchemaByName(model.NewCIStr("write_after_import")) + dbInfo, ok := is.SchemaByName(ast.NewCIStr("write_after_import")) s.True(ok) - tableObj, err := is.TableByName(context.Background(), model.NewCIStr("write_after_import"), model.NewCIStr("t")) + tableObj, err := is.TableByName(context.Background(), ast.NewCIStr("write_after_import"), ast.NewCIStr("t")) s.NoError(err) if common.TableHasAutoID(tableObj.Meta()) { allocators, err := common.GetGlobalAutoIDAlloc(domain.GetDomain(s.tk.Session()), dbInfo.ID, tableObj.Meta()) diff --git a/tests/realtikvtest/pessimistictest/BUILD.bazel b/tests/realtikvtest/pessimistictest/BUILD.bazel index 68f2261ad15b8..76a00174025a8 100644 --- a/tests/realtikvtest/pessimistictest/BUILD.bazel +++ b/tests/realtikvtest/pessimistictest/BUILD.bazel @@ -15,8 +15,8 @@ go_test( "//pkg/expression", "//pkg/kv", "//pkg/parser", + "//pkg/parser/ast", "//pkg/parser/auth", - "//pkg/parser/model", "//pkg/parser/mysql", "//pkg/parser/terror", "//pkg/session", diff --git a/tests/realtikvtest/pessimistictest/pessimistic_test.go b/tests/realtikvtest/pessimistictest/pessimistic_test.go index 3fb4120d85079..d4bb4bcb94b0a 100644 --- a/tests/realtikvtest/pessimistictest/pessimistic_test.go +++ b/tests/realtikvtest/pessimistictest/pessimistic_test.go @@ -32,8 +32,8 @@ import ( "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/auth" - "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/session" @@ -283,7 +283,7 @@ func TestSingleStatementRollback(t *testing.T) { dom := domain.GetDomain(tk1.Session()) is := dom.InfoSchema() - tbl, err := is.TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("single_statement")) + tbl, err := is.TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("single_statement")) require.NoError(t, err) tblID := tbl.Meta().ID diff --git a/tests/realtikvtest/sessiontest/BUILD.bazel b/tests/realtikvtest/sessiontest/BUILD.bazel index c733f8a372b29..e401af7c3a27d 100644 --- a/tests/realtikvtest/sessiontest/BUILD.bazel +++ b/tests/realtikvtest/sessiontest/BUILD.bazel @@ -15,7 +15,7 @@ go_test( "//pkg/config", "//pkg/infoschema", "//pkg/meta", - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/session", "//pkg/sessionctx/variable", "//pkg/store/helper", diff --git a/tests/realtikvtest/sessiontest/infoschema_v2_test.go b/tests/realtikvtest/sessiontest/infoschema_v2_test.go index 993dfc0c97f39..1b6d431a551e7 100644 --- a/tests/realtikvtest/sessiontest/infoschema_v2_test.go +++ b/tests/realtikvtest/sessiontest/infoschema_v2_test.go @@ -21,7 +21,7 @@ import ( "github.com/pingcap/tidb/pkg/config" "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/meta" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/session" "github.com/pingcap/tidb/pkg/store/helper" "github.com/pingcap/tidb/pkg/testkit" @@ -47,11 +47,11 @@ func TestGCOldVersion(t *testing.T) { require.NoError(t, err) oldIS := dom.InfoSchema() - t1, err := oldIS.TableInfoByName(model.NewCIStr("test"), model.NewCIStr("t1")) + t1, err := oldIS.TableInfoByName(ast.NewCIStr("test"), ast.NewCIStr("t1")) require.NoError(t, err) - t2, err := oldIS.TableInfoByName(model.NewCIStr("test"), model.NewCIStr("t2")) + t2, err := oldIS.TableInfoByName(ast.NewCIStr("test"), ast.NewCIStr("t2")) require.NoError(t, err) - t3, err := oldIS.TableInfoByName(model.NewCIStr("test"), model.NewCIStr("t3")) + t3, err := oldIS.TableInfoByName(ast.NewCIStr("test"), ast.NewCIStr("t3")) require.NoError(t, err) s := store.(helper.Storage) @@ -91,11 +91,11 @@ func TestGCOldVersion(t *testing.T) { require.False(t, ok) _, ok = oldIS.TableByID(context.Background(), t3.ID) require.False(t, ok) - _, err = oldIS.TableInfoByName(model.NewCIStr("test"), model.NewCIStr("t1")) + _, err = oldIS.TableInfoByName(ast.NewCIStr("test"), ast.NewCIStr("t1")) require.Error(t, err) - _, err = oldIS.TableInfoByName(model.NewCIStr("test"), model.NewCIStr("t2")) + _, err = oldIS.TableInfoByName(ast.NewCIStr("test"), ast.NewCIStr("t2")) require.Error(t, err) - _, err = oldIS.TableInfoByName(model.NewCIStr("test"), model.NewCIStr("t3")) + _, err = oldIS.TableInfoByName(ast.NewCIStr("test"), ast.NewCIStr("t3")) require.Error(t, err) // GC will not delete the current schema version. diff --git a/tests/realtikvtest/statisticstest/BUILD.bazel b/tests/realtikvtest/statisticstest/BUILD.bazel index 62459aff93f31..b3dcac1ca9cb2 100644 --- a/tests/realtikvtest/statisticstest/BUILD.bazel +++ b/tests/realtikvtest/statisticstest/BUILD.bazel @@ -10,7 +10,7 @@ go_test( flaky = True, race = "on", deps = [ - "//pkg/parser/model", + "//pkg/parser/ast", "//pkg/statistics/asyncload", "//pkg/testkit", "//tests/realtikvtest", diff --git a/tests/realtikvtest/statisticstest/statistics_test.go b/tests/realtikvtest/statisticstest/statistics_test.go index db4ae6afa8906..38957ba2497fb 100644 --- a/tests/realtikvtest/statisticstest/statistics_test.go +++ b/tests/realtikvtest/statisticstest/statistics_test.go @@ -20,7 +20,7 @@ import ( "testing" "time" - "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/statistics/asyncload" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/tests/realtikvtest" @@ -254,7 +254,7 @@ func TestNoNeedIndexStatsLoading(t *testing.T) { // 4. Try to select some data from this table by ID, it would trigger an async load. tk.MustExec("set tidb_opt_objective='determinate';") tk.MustQuery("select * from t where a = 1 and b = 1;").Check(testkit.Rows("1 1")) - table, err := dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr("t")) + table, err := dom.InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t")) require.NoError(t, err) checkTableIDInItems(t, table.Meta().ID) } From 67249cb2fe509bdae24d26bd6b6b6ba2400bd345 Mon Sep 17 00:00:00 2001 From: Rustin Date: Tue, 7 Jan 2025 21:00:58 +0800 Subject: [PATCH 03/27] statistics: refactor stats meta handling to use DeltaUpdate for multi-table support (#58657) ref pingcap/tidb#57869 --- pkg/statistics/handle/ddl/subscriber.go | 8 +- pkg/statistics/handle/storage/update.go | 97 +++++++++++++++---- .../handle/usage/session_stats_collect.go | 23 ++++- 3 files changed, 96 insertions(+), 32 deletions(-) diff --git a/pkg/statistics/handle/ddl/subscriber.go b/pkg/statistics/handle/ddl/subscriber.go index a4af15a1815d0..3578941a4f78d 100644 --- a/pkg/statistics/handle/ddl/subscriber.go +++ b/pkg/statistics/handle/ddl/subscriber.go @@ -413,9 +413,7 @@ func updateGlobalTableStats4DropPartition( ctx, sctx, startTS, - variable.TableDelta{Count: count, Delta: delta}, - globalTableInfo.ID, - isLocked, + storage.NewDeltaUpdate(globalTableInfo.ID, variable.TableDelta{Count: count, Delta: delta}, isLocked), )) } @@ -597,9 +595,7 @@ func updateGlobalTableStats4TruncatePartition( ctx, sctx, startTS, - variable.TableDelta{Count: count, Delta: delta}, - globalTableInfo.ID, - isLocked, + storage.NewDeltaUpdate(globalTableInfo.ID, variable.TableDelta{Count: count, Delta: delta}, isLocked), ) if err != nil { fields := truncatePartitionsLogFields( diff --git a/pkg/statistics/handle/storage/update.go b/pkg/statistics/handle/storage/update.go index fc312743f9c79..1ffb6314cb038 100644 --- a/pkg/statistics/handle/storage/update.go +++ b/pkg/statistics/handle/storage/update.go @@ -19,6 +19,7 @@ import ( "encoding/json" "fmt" "slices" + "strings" "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/parser/ast" @@ -55,36 +56,90 @@ func UpdateStatsVersion(ctx context.Context, sctx sessionctx.Context) error { return nil } -// UpdateStatsMeta update the stats meta stat for this Table. +// DeltaUpdate is the delta update for stats meta. +type DeltaUpdate struct { + Delta variable.TableDelta + TableID int64 + IsLocked bool +} + +// NewDeltaUpdate creates a new DeltaUpdate. +func NewDeltaUpdate(tableID int64, delta variable.TableDelta, isLocked bool) *DeltaUpdate { + return &DeltaUpdate{ + Delta: delta, + TableID: tableID, + IsLocked: isLocked, + } +} + +// UpdateStatsMeta updates the stats meta for multiple tables. +// It uses the INSERT INTO ... ON DUPLICATE KEY UPDATE syntax to fill the missing records. func UpdateStatsMeta( ctx context.Context, sctx sessionctx.Context, startTS uint64, - delta variable.TableDelta, - id int64, - isLocked bool, + updates ...*DeltaUpdate, ) (err error) { - if isLocked { - // use INSERT INTO ... ON DUPLICATE KEY UPDATE here to fill missing stats_table_locked. - // Note: For locked tables, it is possible that the record gets deleted. So it can be negative. - _, err = statsutil.ExecWithCtx(ctx, sctx, "insert into mysql.stats_table_locked (version, table_id, modify_count, count) values (%?, %?, %?, %?) on duplicate key "+ - "update version = values(version), modify_count = modify_count + values(modify_count), count = count + values(count)", - startTS, id, delta.Count, delta.Delta) - } else { - if delta.Delta < 0 { - // use INSERT INTO ... ON DUPLICATE KEY UPDATE here to fill missing stats_meta. - _, err = statsutil.ExecWithCtx(ctx, sctx, "insert into mysql.stats_meta (version, table_id, modify_count, count) values (%?, %?, %?, 0) on duplicate key "+ - "update version = values(version), modify_count = modify_count + values(modify_count), count = if(count > %?, count - %?, 0)", - startTS, id, delta.Count, -delta.Delta, -delta.Delta) + if len(updates) == 0 { + return nil + } + + // Separate locked and unlocked updates + var lockedValues, unlockedPosValues, unlockedNegValues []string + var cacheInvalidateIDs []int64 + + for _, update := range updates { + if update.IsLocked { + lockedValues = append(lockedValues, fmt.Sprintf("(%d, %d, %d, %d)", + startTS, update.TableID, update.Delta.Count, update.Delta.Delta)) } else { - // use INSERT INTO ... ON DUPLICATE KEY UPDATE here to fill missing stats_meta. - _, err = statsutil.ExecWithCtx(ctx, sctx, "insert into mysql.stats_meta (version, table_id, modify_count, count) values (%?, %?, %?, %?) on duplicate key "+ - "update version = values(version), modify_count = modify_count + values(modify_count), count = count + values(count)", startTS, - id, delta.Count, delta.Delta) + if update.Delta.Delta < 0 { + unlockedNegValues = append(unlockedNegValues, fmt.Sprintf("(%d, %d, %d, %d)", + startTS, update.TableID, update.Delta.Count, -update.Delta.Delta)) + } else { + unlockedPosValues = append(unlockedPosValues, fmt.Sprintf("(%d, %d, %d, %d)", + startTS, update.TableID, update.Delta.Count, update.Delta.Delta)) + } + cacheInvalidateIDs = append(cacheInvalidateIDs, update.TableID) + } + } + + // Execute locked updates + if len(lockedValues) > 0 { + sql := fmt.Sprintf("insert into mysql.stats_table_locked (version, table_id, modify_count, count) values %s "+ + "on duplicate key update version = values(version), modify_count = modify_count + values(modify_count), "+ + "count = count + values(count)", strings.Join(lockedValues, ",")) + if _, err = statsutil.ExecWithCtx(ctx, sctx, sql); err != nil { + return err + } + } + + // Execute unlocked updates with positive delta + if len(unlockedPosValues) > 0 { + sql := fmt.Sprintf("insert into mysql.stats_meta (version, table_id, modify_count, count) values %s "+ + "on duplicate key update version = values(version), modify_count = modify_count + values(modify_count), "+ + "count = count + values(count)", strings.Join(unlockedPosValues, ",")) + if _, err = statsutil.ExecWithCtx(ctx, sctx, sql); err != nil { + return err } + } + + // Execute unlocked updates with negative delta + if len(unlockedNegValues) > 0 { + sql := fmt.Sprintf("insert into mysql.stats_meta (version, table_id, modify_count, count) values %s "+ + "on duplicate key update version = values(version), modify_count = modify_count + values(modify_count), "+ + "count = if(count > values(count), count - values(count), 0)", strings.Join(unlockedNegValues, ",")) + if _, err = statsutil.ExecWithCtx(ctx, sctx, sql); err != nil { + return err + } + } + + // Invalidate cache for all unlocked tables + for _, id := range cacheInvalidateIDs { cache.TableRowStatsCache.Invalidate(id) } - return err + + return nil } // InsertExtendedStats inserts a record into mysql.stats_extended and update version in mysql.stats_meta. diff --git a/pkg/statistics/handle/usage/session_stats_collect.go b/pkg/statistics/handle/usage/session_stats_collect.go index fcabb207b1d5d..078dee2346f24 100644 --- a/pkg/statistics/handle/usage/session_stats_collect.go +++ b/pkg/statistics/handle/usage/session_stats_collect.go @@ -172,8 +172,12 @@ func (s *statsUsageImpl) dumpTableStatCountToKV(is infoschema.InfoSchema, physic } tableOrPartitionLocked := isTableLocked || isPartitionLocked isLocked = tableOrPartitionLocked - if err = storage.UpdateStatsMeta(utilstats.StatsCtx, sctx, statsVersion, delta, - physicalTableID, tableOrPartitionLocked); err != nil { + if err = storage.UpdateStatsMeta( + utilstats.StatsCtx, + sctx, + statsVersion, + storage.NewDeltaUpdate(physicalTableID, delta, tableOrPartitionLocked), + ); err != nil { return err } affectedRows += sctx.GetSessionVars().StmtCtx.AffectedRows() @@ -190,7 +194,12 @@ func (s *statsUsageImpl) dumpTableStatCountToKV(is infoschema.InfoSchema, physic // To sum up, we only need to update the global-stats when the table and the partition are not locked. if !isTableLocked && !isPartitionLocked { // If it's a partitioned table and its global-stats exists, update its count and modify_count as well. - if err = storage.UpdateStatsMeta(utilstats.StatsCtx, sctx, statsVersion, delta, tableID, isTableLocked); err != nil { + if err = storage.UpdateStatsMeta( + utilstats.StatsCtx, + sctx, + statsVersion, + storage.NewDeltaUpdate(tableID, delta, isTableLocked), + ); err != nil { return err } affectedRows += sctx.GetSessionVars().StmtCtx.AffectedRows() @@ -203,8 +212,12 @@ func (s *statsUsageImpl) dumpTableStatCountToKV(is infoschema.InfoSchema, physic isTableLocked = true } isLocked = isTableLocked - if err = storage.UpdateStatsMeta(utilstats.StatsCtx, sctx, statsVersion, delta, - physicalTableID, isTableLocked); err != nil { + if err = storage.UpdateStatsMeta( + utilstats.StatsCtx, + sctx, + statsVersion, + storage.NewDeltaUpdate(physicalTableID, delta, isTableLocked), + ); err != nil { return err } affectedRows += sctx.GetSessionVars().StmtCtx.AffectedRows() From 6c9ee36e8e7c86abbda374d841b019979ff3aaf4 Mon Sep 17 00:00:00 2001 From: okJiang <819421878@qq.com> Date: Wed, 8 Jan 2025 12:00:34 +0800 Subject: [PATCH 04/27] var: enable `pd_enable_follower_handle_region` as default (#58385) close pingcap/tidb#58384 --- pkg/executor/set_test.go | 4 +- pkg/metrics/grafana/tidb.json | 105 ++++++++++++++++++++ pkg/sessionctx/variable/tidb_vars.go | 2 +- tests/integrationtest/r/session/vars.result | 8 +- tests/integrationtest/t/session/vars.test | 4 +- 5 files changed, 114 insertions(+), 9 deletions(-) diff --git a/pkg/executor/set_test.go b/pkg/executor/set_test.go index f4111f6cd8129..44c3e0e224ec4 100644 --- a/pkg/executor/set_test.go +++ b/pkg/executor/set_test.go @@ -655,11 +655,11 @@ func TestSetVar(t *testing.T) { tk.MustExec("set global tidb_enable_tso_follower_proxy = 0") tk.MustQuery("select @@tidb_enable_tso_follower_proxy").Check(testkit.Rows("0")) require.Error(t, tk.ExecToErr("set tidb_enable_tso_follower_proxy = 1")) - tk.MustQuery("select @@pd_enable_follower_handle_region").Check(testkit.Rows("0")) - tk.MustExec("set global pd_enable_follower_handle_region = 1") tk.MustQuery("select @@pd_enable_follower_handle_region").Check(testkit.Rows("1")) tk.MustExec("set global pd_enable_follower_handle_region = 0") tk.MustQuery("select @@pd_enable_follower_handle_region").Check(testkit.Rows("0")) + tk.MustExec("set global pd_enable_follower_handle_region = 1") + tk.MustQuery("select @@pd_enable_follower_handle_region").Check(testkit.Rows("1")) require.Error(t, tk.ExecToErr("set pd_enable_follower_handle_region = 1")) tk.MustQuery("select @@tidb_enable_historical_stats").Check(testkit.Rows("0")) diff --git a/pkg/metrics/grafana/tidb.json b/pkg/metrics/grafana/tidb.json index 096e2facfd2e7..5d4368d8f7cf5 100644 --- a/pkg/metrics/grafana/tidb.json +++ b/pkg/metrics/grafana/tidb.json @@ -13336,6 +13336,111 @@ "alignLevel": null } }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The stale regions from PD per second.", + "editable": true, + "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 0, + "fillGradient": 0, + "grid": {}, + "gridPos": { + "h": 7, + "w": 8, + "x": 8, + "y": 37 + }, + "hiddenSeries": false, + "id": 339, + "legend": { + "alignAsTable": true, + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 2, + "links": [], + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.11", + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "sum(rate(tidb_tikvclient_stale_region_from_pd{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[30s]))", + "format": "time_series", + "interval": "", + "intervalFactor": 2, + "legendFormat": "all", + "refId": "A", + "step": 10 + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Stale Region From PD", + "tooltip": { + "msResolution": false, + "shared": true, + "sort": 0, + "value_type": "cumulative" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "ops", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, { "aliasColors": {}, "bars": false, diff --git a/pkg/sessionctx/variable/tidb_vars.go b/pkg/sessionctx/variable/tidb_vars.go index 09d18f75188b4..89bb6a47d79cc 100644 --- a/pkg/sessionctx/variable/tidb_vars.go +++ b/pkg/sessionctx/variable/tidb_vars.go @@ -1388,7 +1388,7 @@ const ( DefTiDBEnableLocalTxn = false DefTiDBTSOClientBatchMaxWaitTime = 0.0 // 0ms DefTiDBEnableTSOFollowerProxy = false - DefPDEnableFollowerHandleRegion = false + DefPDEnableFollowerHandleRegion = true DefTiDBEnableOrderedResultMode = false DefTiDBEnablePseudoForOutdatedStats = false DefTiDBRegardNULLAsPoint = true diff --git a/tests/integrationtest/r/session/vars.result b/tests/integrationtest/r/session/vars.result index 2f354c8baadcc..ca7114556ca00 100644 --- a/tests/integrationtest/r/session/vars.result +++ b/tests/integrationtest/r/session/vars.result @@ -64,15 +64,15 @@ select @@tidb_enable_tso_follower_proxy; 0 select @@pd_enable_follower_handle_region; @@pd_enable_follower_handle_region -0 -set global pd_enable_follower_handle_region = on; -select @@pd_enable_follower_handle_region; -@@pd_enable_follower_handle_region 1 set global pd_enable_follower_handle_region = off; select @@pd_enable_follower_handle_region; @@pd_enable_follower_handle_region 0 +set global pd_enable_follower_handle_region = on; +select @@pd_enable_follower_handle_region; +@@pd_enable_follower_handle_region +1 set tidb_tso_client_batch_max_wait_time = 0; Error 1229 (HY000): Variable 'tidb_tso_client_batch_max_wait_time' is a GLOBAL variable and should be set with SET GLOBAL set global tidb_enable_tso_follower_proxy = default; diff --git a/tests/integrationtest/t/session/vars.test b/tests/integrationtest/t/session/vars.test index 0f7d4fc12b760..ea648134cb67c 100644 --- a/tests/integrationtest/t/session/vars.test +++ b/tests/integrationtest/t/session/vars.test @@ -36,10 +36,10 @@ select @@tidb_enable_tso_follower_proxy; set global tidb_enable_tso_follower_proxy = off; select @@tidb_enable_tso_follower_proxy; select @@pd_enable_follower_handle_region; -set global pd_enable_follower_handle_region = on; -select @@pd_enable_follower_handle_region; set global pd_enable_follower_handle_region = off; select @@pd_enable_follower_handle_region; +set global pd_enable_follower_handle_region = on; +select @@pd_enable_follower_handle_region; -- error 1229 set tidb_tso_client_batch_max_wait_time = 0; set global tidb_enable_tso_follower_proxy = default; From d966219df57db3e007f8c361822add8b77f35104 Mon Sep 17 00:00:00 2001 From: ShuNing Date: Wed, 8 Jan 2025 13:14:28 +0800 Subject: [PATCH 05/27] util/stmtsummary: add the network traffic related fields (#58101) ref pingcap/tidb#57543 --- pkg/infoschema/tables.go | 8 + pkg/util/stmtsummary/reader.go | 247 ++++++++++-------- pkg/util/stmtsummary/statement_summary.go | 46 ++++ .../stmtsummary/statement_summary_test.go | 49 +++- pkg/util/stmtsummary/v2/column.go | 246 +++++++++-------- pkg/util/stmtsummary/v2/record.go | 5 + tests/integrationtest/r/index_merge.result | 4 +- 7 files changed, 382 insertions(+), 223 deletions(-) diff --git a/pkg/infoschema/tables.go b/pkg/infoschema/tables.go index 2b923f51250c1..06d9c69bb41c8 100644 --- a/pkg/infoschema/tables.go +++ b/pkg/infoschema/tables.go @@ -1393,6 +1393,14 @@ var tableStatementsSummaryCols = []columnInfo{ {name: stmtsummary.ResourceGroupName, tp: mysql.TypeVarchar, size: 64, comment: "Bind resource group name"}, {name: stmtsummary.PlanCacheUnqualifiedStr, tp: mysql.TypeLonglong, size: 20, flag: mysql.NotNullFlag, comment: "The number of times that these statements are not supported by the plan cache"}, {name: stmtsummary.PlanCacheUnqualifiedLastReasonStr, tp: mysql.TypeBlob, size: types.UnspecifiedLength, comment: "The last reason why the statement is not supported by the plan cache"}, + {name: stmtsummary.SumUnpackedBytesSentKVTotalStr, tp: mysql.TypeLonglong, size: 20, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Average bytes sent to KV"}, + {name: stmtsummary.SumUnpackedBytesReceivedKVTotalStr, tp: mysql.TypeLonglong, size: 20, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Average bytes received from KV"}, + {name: stmtsummary.SumUnpackedBytesSentKVCrossZoneStr, tp: mysql.TypeLonglong, size: 20, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Average bytes sent to KV cross zone"}, + {name: stmtsummary.SumUnpackedBytesReceivedKVCrossZoneStr, tp: mysql.TypeLonglong, size: 20, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Average bytes received from KV cross zone"}, + {name: stmtsummary.SumUnpackedBytesSentMPPTotalStr, tp: mysql.TypeLonglong, size: 20, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Average bytes sent to MPP"}, + {name: stmtsummary.SumUnpackedBytesReceivedMPPTotalStr, tp: mysql.TypeLonglong, size: 20, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Average bytes received from MPP"}, + {name: stmtsummary.SumUnpackedBytesSentMPPCrossZoneStr, tp: mysql.TypeLonglong, size: 20, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Average bytes sent to MPP cross zone"}, + {name: stmtsummary.SumUnpackedBytesReceiveMPPCrossZoneStr, tp: mysql.TypeLonglong, size: 20, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Average bytes received from MPP cross zone"}, } var tableTiDBStatementsStatsCols = []columnInfo{ diff --git a/pkg/util/stmtsummary/reader.go b/pkg/util/stmtsummary/reader.go index a537d3db4bae3..ee9bebd821c2f 100644 --- a/pkg/util/stmtsummary/reader.go +++ b/pkg/util/stmtsummary/reader.go @@ -256,113 +256,121 @@ func (ssc *stmtSummaryChecker) isDigestValid(digest string) bool { // Statements summary table column name. const ( - ClusterTableInstanceColumnNameStr = "INSTANCE" - SummaryBeginTimeStr = "SUMMARY_BEGIN_TIME" - SummaryEndTimeStr = "SUMMARY_END_TIME" - StmtTypeStr = "STMT_TYPE" - SchemaNameStr = "SCHEMA_NAME" - DigestStr = "DIGEST" - DigestTextStr = "DIGEST_TEXT" - TableNamesStr = "TABLE_NAMES" - IndexNamesStr = "INDEX_NAMES" - SampleUserStr = "SAMPLE_USER" - ExecCountStr = "EXEC_COUNT" - SumErrorsStr = "SUM_ERRORS" - SumWarningsStr = "SUM_WARNINGS" - SumLatencyStr = "SUM_LATENCY" - MaxLatencyStr = "MAX_LATENCY" - MinLatencyStr = "MIN_LATENCY" - AvgLatencyStr = "AVG_LATENCY" - AvgParseLatencyStr = "AVG_PARSE_LATENCY" - MaxParseLatencyStr = "MAX_PARSE_LATENCY" - AvgCompileLatencyStr = "AVG_COMPILE_LATENCY" - MaxCompileLatencyStr = "MAX_COMPILE_LATENCY" - SumCopTaskNumStr = "SUM_COP_TASK_NUM" - MaxCopProcessTimeStr = "MAX_COP_PROCESS_TIME" - MaxCopProcessAddressStr = "MAX_COP_PROCESS_ADDRESS" - MaxCopWaitTimeStr = "MAX_COP_WAIT_TIME" // #nosec G101 - MaxCopWaitAddressStr = "MAX_COP_WAIT_ADDRESS" // #nosec G101 - AvgProcessTimeStr = "AVG_PROCESS_TIME" - MaxProcessTimeStr = "MAX_PROCESS_TIME" - AvgWaitTimeStr = "AVG_WAIT_TIME" - MaxWaitTimeStr = "MAX_WAIT_TIME" - AvgBackoffTimeStr = "AVG_BACKOFF_TIME" - MaxBackoffTimeStr = "MAX_BACKOFF_TIME" - AvgTotalKeysStr = "AVG_TOTAL_KEYS" - MaxTotalKeysStr = "MAX_TOTAL_KEYS" - AvgProcessedKeysStr = "AVG_PROCESSED_KEYS" - MaxProcessedKeysStr = "MAX_PROCESSED_KEYS" - AvgRocksdbDeleteSkippedCountStr = "AVG_ROCKSDB_DELETE_SKIPPED_COUNT" - MaxRocksdbDeleteSkippedCountStr = "MAX_ROCKSDB_DELETE_SKIPPED_COUNT" - AvgRocksdbKeySkippedCountStr = "AVG_ROCKSDB_KEY_SKIPPED_COUNT" - MaxRocksdbKeySkippedCountStr = "MAX_ROCKSDB_KEY_SKIPPED_COUNT" - AvgRocksdbBlockCacheHitCountStr = "AVG_ROCKSDB_BLOCK_CACHE_HIT_COUNT" - MaxRocksdbBlockCacheHitCountStr = "MAX_ROCKSDB_BLOCK_CACHE_HIT_COUNT" - AvgRocksdbBlockReadCountStr = "AVG_ROCKSDB_BLOCK_READ_COUNT" - MaxRocksdbBlockReadCountStr = "MAX_ROCKSDB_BLOCK_READ_COUNT" - AvgRocksdbBlockReadByteStr = "AVG_ROCKSDB_BLOCK_READ_BYTE" - MaxRocksdbBlockReadByteStr = "MAX_ROCKSDB_BLOCK_READ_BYTE" - AvgPrewriteTimeStr = "AVG_PREWRITE_TIME" - MaxPrewriteTimeStr = "MAX_PREWRITE_TIME" - AvgCommitTimeStr = "AVG_COMMIT_TIME" - MaxCommitTimeStr = "MAX_COMMIT_TIME" - AvgGetCommitTsTimeStr = "AVG_GET_COMMIT_TS_TIME" - MaxGetCommitTsTimeStr = "MAX_GET_COMMIT_TS_TIME" - AvgCommitBackoffTimeStr = "AVG_COMMIT_BACKOFF_TIME" - MaxCommitBackoffTimeStr = "MAX_COMMIT_BACKOFF_TIME" - AvgResolveLockTimeStr = "AVG_RESOLVE_LOCK_TIME" - MaxResolveLockTimeStr = "MAX_RESOLVE_LOCK_TIME" - AvgLocalLatchWaitTimeStr = "AVG_LOCAL_LATCH_WAIT_TIME" - MaxLocalLatchWaitTimeStr = "MAX_LOCAL_LATCH_WAIT_TIME" - AvgWriteKeysStr = "AVG_WRITE_KEYS" - MaxWriteKeysStr = "MAX_WRITE_KEYS" - AvgWriteSizeStr = "AVG_WRITE_SIZE" - MaxWriteSizeStr = "MAX_WRITE_SIZE" - AvgPrewriteRegionsStr = "AVG_PREWRITE_REGIONS" - MaxPrewriteRegionsStr = "MAX_PREWRITE_REGIONS" - AvgTxnRetryStr = "AVG_TXN_RETRY" - MaxTxnRetryStr = "MAX_TXN_RETRY" - SumExecRetryStr = "SUM_EXEC_RETRY" - SumExecRetryTimeStr = "SUM_EXEC_RETRY_TIME" - SumBackoffTimesStr = "SUM_BACKOFF_TIMES" - BackoffTypesStr = "BACKOFF_TYPES" - AvgMemStr = "AVG_MEM" - MaxMemStr = "MAX_MEM" - AvgDiskStr = "AVG_DISK" - MaxDiskStr = "MAX_DISK" - AvgKvTimeStr = "AVG_KV_TIME" - AvgPdTimeStr = "AVG_PD_TIME" - AvgBackoffTotalTimeStr = "AVG_BACKOFF_TOTAL_TIME" - AvgWriteSQLRespTimeStr = "AVG_WRITE_SQL_RESP_TIME" - AvgTidbCPUTimeStr = "AVG_TIDB_CPU_TIME" - AvgTikvCPUTimeStr = "AVG_TIKV_CPU_TIME" - MaxResultRowsStr = "MAX_RESULT_ROWS" - MinResultRowsStr = "MIN_RESULT_ROWS" - AvgResultRowsStr = "AVG_RESULT_ROWS" - PreparedStr = "PREPARED" - AvgAffectedRowsStr = "AVG_AFFECTED_ROWS" - FirstSeenStr = "FIRST_SEEN" - LastSeenStr = "LAST_SEEN" - PlanInCacheStr = "PLAN_IN_CACHE" - PlanCacheHitsStr = "PLAN_CACHE_HITS" - PlanCacheUnqualifiedStr = "PLAN_CACHE_UNQUALIFIED" - PlanCacheUnqualifiedLastReasonStr = "PLAN_CACHE_UNQUALIFIED_LAST_REASON" - PlanInBindingStr = "PLAN_IN_BINDING" - QuerySampleTextStr = "QUERY_SAMPLE_TEXT" - PrevSampleTextStr = "PREV_SAMPLE_TEXT" - PlanDigestStr = "PLAN_DIGEST" - PlanStr = "PLAN" - BinaryPlan = "BINARY_PLAN" - Charset = "CHARSET" - Collation = "COLLATION" - PlanHint = "PLAN_HINT" - AvgRequestUnitReadStr = "AVG_REQUEST_UNIT_READ" - MaxRequestUnitReadStr = "MAX_REQUEST_UNIT_READ" - AvgRequestUnitWriteStr = "AVG_REQUEST_UNIT_WRITE" - MaxRequestUnitWriteStr = "MAX_REQUEST_UNIT_WRITE" - AvgQueuedRcTimeStr = "AVG_QUEUED_RC_TIME" - MaxQueuedRcTimeStr = "MAX_QUEUED_RC_TIME" - ResourceGroupName = "RESOURCE_GROUP" + ClusterTableInstanceColumnNameStr = "INSTANCE" + SummaryBeginTimeStr = "SUMMARY_BEGIN_TIME" + SummaryEndTimeStr = "SUMMARY_END_TIME" + StmtTypeStr = "STMT_TYPE" + SchemaNameStr = "SCHEMA_NAME" + DigestStr = "DIGEST" + DigestTextStr = "DIGEST_TEXT" + TableNamesStr = "TABLE_NAMES" + IndexNamesStr = "INDEX_NAMES" + SampleUserStr = "SAMPLE_USER" + ExecCountStr = "EXEC_COUNT" + SumErrorsStr = "SUM_ERRORS" + SumWarningsStr = "SUM_WARNINGS" + SumLatencyStr = "SUM_LATENCY" + MaxLatencyStr = "MAX_LATENCY" + MinLatencyStr = "MIN_LATENCY" + AvgLatencyStr = "AVG_LATENCY" + AvgParseLatencyStr = "AVG_PARSE_LATENCY" + MaxParseLatencyStr = "MAX_PARSE_LATENCY" + AvgCompileLatencyStr = "AVG_COMPILE_LATENCY" + MaxCompileLatencyStr = "MAX_COMPILE_LATENCY" + SumCopTaskNumStr = "SUM_COP_TASK_NUM" + MaxCopProcessTimeStr = "MAX_COP_PROCESS_TIME" + MaxCopProcessAddressStr = "MAX_COP_PROCESS_ADDRESS" + MaxCopWaitTimeStr = "MAX_COP_WAIT_TIME" // #nosec G101 + MaxCopWaitAddressStr = "MAX_COP_WAIT_ADDRESS" // #nosec G101 + AvgProcessTimeStr = "AVG_PROCESS_TIME" + MaxProcessTimeStr = "MAX_PROCESS_TIME" + AvgWaitTimeStr = "AVG_WAIT_TIME" + MaxWaitTimeStr = "MAX_WAIT_TIME" + AvgBackoffTimeStr = "AVG_BACKOFF_TIME" + MaxBackoffTimeStr = "MAX_BACKOFF_TIME" + AvgTotalKeysStr = "AVG_TOTAL_KEYS" + MaxTotalKeysStr = "MAX_TOTAL_KEYS" + AvgProcessedKeysStr = "AVG_PROCESSED_KEYS" + MaxProcessedKeysStr = "MAX_PROCESSED_KEYS" + AvgRocksdbDeleteSkippedCountStr = "AVG_ROCKSDB_DELETE_SKIPPED_COUNT" + MaxRocksdbDeleteSkippedCountStr = "MAX_ROCKSDB_DELETE_SKIPPED_COUNT" + AvgRocksdbKeySkippedCountStr = "AVG_ROCKSDB_KEY_SKIPPED_COUNT" + MaxRocksdbKeySkippedCountStr = "MAX_ROCKSDB_KEY_SKIPPED_COUNT" + AvgRocksdbBlockCacheHitCountStr = "AVG_ROCKSDB_BLOCK_CACHE_HIT_COUNT" + MaxRocksdbBlockCacheHitCountStr = "MAX_ROCKSDB_BLOCK_CACHE_HIT_COUNT" + AvgRocksdbBlockReadCountStr = "AVG_ROCKSDB_BLOCK_READ_COUNT" + MaxRocksdbBlockReadCountStr = "MAX_ROCKSDB_BLOCK_READ_COUNT" + AvgRocksdbBlockReadByteStr = "AVG_ROCKSDB_BLOCK_READ_BYTE" + MaxRocksdbBlockReadByteStr = "MAX_ROCKSDB_BLOCK_READ_BYTE" + AvgPrewriteTimeStr = "AVG_PREWRITE_TIME" + MaxPrewriteTimeStr = "MAX_PREWRITE_TIME" + AvgCommitTimeStr = "AVG_COMMIT_TIME" + MaxCommitTimeStr = "MAX_COMMIT_TIME" + AvgGetCommitTsTimeStr = "AVG_GET_COMMIT_TS_TIME" + MaxGetCommitTsTimeStr = "MAX_GET_COMMIT_TS_TIME" + AvgCommitBackoffTimeStr = "AVG_COMMIT_BACKOFF_TIME" + MaxCommitBackoffTimeStr = "MAX_COMMIT_BACKOFF_TIME" + AvgResolveLockTimeStr = "AVG_RESOLVE_LOCK_TIME" + MaxResolveLockTimeStr = "MAX_RESOLVE_LOCK_TIME" + AvgLocalLatchWaitTimeStr = "AVG_LOCAL_LATCH_WAIT_TIME" + MaxLocalLatchWaitTimeStr = "MAX_LOCAL_LATCH_WAIT_TIME" + AvgWriteKeysStr = "AVG_WRITE_KEYS" + MaxWriteKeysStr = "MAX_WRITE_KEYS" + AvgWriteSizeStr = "AVG_WRITE_SIZE" + MaxWriteSizeStr = "MAX_WRITE_SIZE" + AvgPrewriteRegionsStr = "AVG_PREWRITE_REGIONS" + MaxPrewriteRegionsStr = "MAX_PREWRITE_REGIONS" + AvgTxnRetryStr = "AVG_TXN_RETRY" + MaxTxnRetryStr = "MAX_TXN_RETRY" + SumExecRetryStr = "SUM_EXEC_RETRY" + SumExecRetryTimeStr = "SUM_EXEC_RETRY_TIME" + SumBackoffTimesStr = "SUM_BACKOFF_TIMES" + BackoffTypesStr = "BACKOFF_TYPES" + AvgMemStr = "AVG_MEM" + MaxMemStr = "MAX_MEM" + AvgDiskStr = "AVG_DISK" + MaxDiskStr = "MAX_DISK" + AvgKvTimeStr = "AVG_KV_TIME" + AvgPdTimeStr = "AVG_PD_TIME" + AvgBackoffTotalTimeStr = "AVG_BACKOFF_TOTAL_TIME" + AvgWriteSQLRespTimeStr = "AVG_WRITE_SQL_RESP_TIME" + AvgTidbCPUTimeStr = "AVG_TIDB_CPU_TIME" + AvgTikvCPUTimeStr = "AVG_TIKV_CPU_TIME" + MaxResultRowsStr = "MAX_RESULT_ROWS" + MinResultRowsStr = "MIN_RESULT_ROWS" + AvgResultRowsStr = "AVG_RESULT_ROWS" + PreparedStr = "PREPARED" + AvgAffectedRowsStr = "AVG_AFFECTED_ROWS" + FirstSeenStr = "FIRST_SEEN" + LastSeenStr = "LAST_SEEN" + PlanInCacheStr = "PLAN_IN_CACHE" + PlanCacheHitsStr = "PLAN_CACHE_HITS" + PlanCacheUnqualifiedStr = "PLAN_CACHE_UNQUALIFIED" + PlanCacheUnqualifiedLastReasonStr = "PLAN_CACHE_UNQUALIFIED_LAST_REASON" + PlanInBindingStr = "PLAN_IN_BINDING" + QuerySampleTextStr = "QUERY_SAMPLE_TEXT" + PrevSampleTextStr = "PREV_SAMPLE_TEXT" + PlanDigestStr = "PLAN_DIGEST" + PlanStr = "PLAN" + BinaryPlan = "BINARY_PLAN" + Charset = "CHARSET" + Collation = "COLLATION" + PlanHint = "PLAN_HINT" + AvgRequestUnitReadStr = "AVG_REQUEST_UNIT_READ" + MaxRequestUnitReadStr = "MAX_REQUEST_UNIT_READ" + AvgRequestUnitWriteStr = "AVG_REQUEST_UNIT_WRITE" + MaxRequestUnitWriteStr = "MAX_REQUEST_UNIT_WRITE" + AvgQueuedRcTimeStr = "AVG_QUEUED_RC_TIME" + MaxQueuedRcTimeStr = "MAX_QUEUED_RC_TIME" + ResourceGroupName = "RESOURCE_GROUP" + SumUnpackedBytesSentKVTotalStr = "SUM_UNPACKED_BYTES_SENT_KV_TOTAL" + SumUnpackedBytesReceivedKVTotalStr = "SUM_UNPACKED_BYTES_RECEIVED_KV_TOTAL" + SumUnpackedBytesSentKVCrossZoneStr = "SUM_UNPACKED_BYTES_SENT_KV_CROSS_ZONE" + SumUnpackedBytesReceivedKVCrossZoneStr = "SUM_UNPACKED_BYTES_RECEIVED_KV_CROSS_ZONE" + SumUnpackedBytesSentMPPTotalStr = "SUM_UNPACKED_BYTES_SENT_MPP_TOTAL" + SumUnpackedBytesReceivedMPPTotalStr = "SUM_UNPACKED_BYTES_RECEIVED_MPP_TOTAL" + SumUnpackedBytesSentMPPCrossZoneStr = "SUM_UNPACKED_BYTES_SENT_MPP_CROSS_ZONE" + SumUnpackedBytesReceiveMPPCrossZoneStr = "SUM_UNPACKED_BYTES_RECEIVED_MPP_CROSS_ZONE" ) // Column names for the statement stats table, including columns that have been @@ -888,4 +896,29 @@ var columnValueFactoryMap = map[string]columnValueFactory{ PlanCacheUnqualifiedLastReasonStr: func(_ *stmtSummaryReader, _ *stmtSummaryByDigestElement, _ *stmtSummaryByDigest, ssStats *stmtSummaryStats) any { return ssStats.lastPlanCacheUnqualified }, + SumUnpackedBytesSentKVTotalStr: func(_ *stmtSummaryReader, _ *stmtSummaryByDigestElement, _ *stmtSummaryByDigest, ssStats *stmtSummaryStats) any { + return ssStats.UnpackedBytesSentKVTotal + }, + + SumUnpackedBytesReceivedKVTotalStr: func(_ *stmtSummaryReader, _ *stmtSummaryByDigestElement, _ *stmtSummaryByDigest, ssStats *stmtSummaryStats) any { + return ssStats.UnpackedBytesReceivedKVTotal + }, + SumUnpackedBytesSentKVCrossZoneStr: func(_ *stmtSummaryReader, _ *stmtSummaryByDigestElement, _ *stmtSummaryByDigest, ssStats *stmtSummaryStats) any { + return ssStats.UnpackedBytesSentKVCrossZone + }, + SumUnpackedBytesReceivedKVCrossZoneStr: func(_ *stmtSummaryReader, _ *stmtSummaryByDigestElement, _ *stmtSummaryByDigest, ssStats *stmtSummaryStats) any { + return ssStats.UnpackedBytesReceivedKVCrossZone + }, + SumUnpackedBytesSentMPPTotalStr: func(_ *stmtSummaryReader, _ *stmtSummaryByDigestElement, _ *stmtSummaryByDigest, ssStats *stmtSummaryStats) any { + return ssStats.UnpackedBytesSentMPPTotal + }, + SumUnpackedBytesReceivedMPPTotalStr: func(_ *stmtSummaryReader, _ *stmtSummaryByDigestElement, _ *stmtSummaryByDigest, ssStats *stmtSummaryStats) any { + return ssStats.UnpackedBytesReceivedMPPTotal + }, + SumUnpackedBytesSentMPPCrossZoneStr: func(_ *stmtSummaryReader, _ *stmtSummaryByDigestElement, _ *stmtSummaryByDigest, ssStats *stmtSummaryStats) any { + return ssStats.UnpackedBytesSentMPPCrossZone + }, + SumUnpackedBytesReceiveMPPCrossZoneStr: func(_ *stmtSummaryReader, _ *stmtSummaryByDigestElement, _ *stmtSummaryByDigest, ssStats *stmtSummaryStats) any { + return ssStats.UnpackedBytesReceivedMPPCrossZone + }, } diff --git a/pkg/util/stmtsummary/statement_summary.go b/pkg/util/stmtsummary/statement_summary.go index f711d15bfc176..bf1d6f943e785 100644 --- a/pkg/util/stmtsummary/statement_summary.go +++ b/pkg/util/stmtsummary/statement_summary.go @@ -235,9 +235,11 @@ type stmtSummaryStats struct { // request-units resourceGroupName string StmtRUSummary + StmtNetworkTrafficSummary planCacheUnqualifiedCount int64 lastPlanCacheUnqualified string // the reason why this query is unqualified for the plan cache + } // StmtExecInfo records execution information of each statement. @@ -978,6 +980,9 @@ func (ssStats *stmtSummaryStats) add(sei *StmtExecInfo) { ssStats.sumTidbCPU += sei.CPUUsages.TidbCPUTime ssStats.sumTikvCPU += sei.CPUUsages.TikvCPUTime + // network traffic + ssStats.StmtNetworkTrafficSummary.Add(&sei.TiKVExecDetails) + // request-units ssStats.StmtRUSummary.Add(sei.RUDetail) } @@ -1110,3 +1115,44 @@ func (s *StmtRUSummary) Merge(other *StmtRUSummary) { s.MaxRUWaitDuration = other.MaxRUWaitDuration } } + +// StmtNetworkTrafficSummary is the network traffic summary for each type of statements. +type StmtNetworkTrafficSummary struct { + UnpackedBytesSentKVTotal int64 `json:"unpacked_bytes_send_kv_total"` + UnpackedBytesReceivedKVTotal int64 `json:"unpacked_bytes_received_kv_total"` + UnpackedBytesSentKVCrossZone int64 `json:"unpacked_bytes_send_kv_cross_zone"` + UnpackedBytesReceivedKVCrossZone int64 `json:"unpacked_bytes_received_kv_cross_zone"` + UnpackedBytesSentMPPTotal int64 `json:"unpacked_bytes_send_mpp_total"` + UnpackedBytesReceivedMPPTotal int64 `json:"unpacked_bytes_received_mpp_total"` + UnpackedBytesSentMPPCrossZone int64 `json:"unpacked_bytes_send_mpp_cross_zone"` + UnpackedBytesReceivedMPPCrossZone int64 `json:"unpacked_bytes_received_mpp_cross_zone"` +} + +// Merge merges the value of 2 network traffic summary records. +func (s *StmtNetworkTrafficSummary) Merge(other *StmtNetworkTrafficSummary) { + if other == nil { + return + } + s.UnpackedBytesSentKVTotal += other.UnpackedBytesSentKVTotal + s.UnpackedBytesReceivedKVTotal += other.UnpackedBytesReceivedKVTotal + s.UnpackedBytesSentKVCrossZone += other.UnpackedBytesSentKVCrossZone + s.UnpackedBytesReceivedKVCrossZone += other.UnpackedBytesReceivedKVCrossZone + s.UnpackedBytesSentMPPTotal += other.UnpackedBytesSentMPPTotal + s.UnpackedBytesReceivedMPPTotal += other.UnpackedBytesReceivedMPPTotal + s.UnpackedBytesSentMPPCrossZone += other.UnpackedBytesSentMPPCrossZone + s.UnpackedBytesReceivedMPPCrossZone += other.UnpackedBytesReceivedMPPCrossZone +} + +// Add add a new sample value to the ru summary record. +func (s *StmtNetworkTrafficSummary) Add(info *util.ExecDetails) { + if info != nil { + s.UnpackedBytesSentKVTotal += info.UnpackedBytesSentKVTotal + s.UnpackedBytesReceivedKVTotal += info.UnpackedBytesReceivedKVTotal + s.UnpackedBytesSentKVCrossZone += info.UnpackedBytesSentKVCrossZone + s.UnpackedBytesReceivedKVCrossZone += info.UnpackedBytesReceivedKVCrossZone + s.UnpackedBytesSentMPPTotal += info.UnpackedBytesSentMPPTotal + s.UnpackedBytesReceivedMPPTotal += info.UnpackedBytesReceivedMPPTotal + s.UnpackedBytesSentMPPCrossZone += info.UnpackedBytesSentMPPCrossZone + s.UnpackedBytesReceivedMPPCrossZone += info.UnpackedBytesReceivedMPPCrossZone + } +} diff --git a/pkg/util/stmtsummary/statement_summary_test.go b/pkg/util/stmtsummary/statement_summary_test.go index bf7a546888c55..3ee938c90f807 100644 --- a/pkg/util/stmtsummary/statement_summary_test.go +++ b/pkg/util/stmtsummary/statement_summary_test.go @@ -150,6 +150,16 @@ func TestAddStatement(t *testing.T) { MaxRUWaitDuration: stmtExecInfo1.RUDetail.RUWaitDuration(), }, resourceGroupName: stmtExecInfo1.ResourceGroupName, + StmtNetworkTrafficSummary: StmtNetworkTrafficSummary{ + UnpackedBytesSentKVTotal: stmtExecInfo1.TiKVExecDetails.UnpackedBytesSentKVTotal, + UnpackedBytesReceivedKVTotal: stmtExecInfo1.TiKVExecDetails.UnpackedBytesReceivedKVTotal, + UnpackedBytesSentKVCrossZone: stmtExecInfo1.TiKVExecDetails.UnpackedBytesSentKVCrossZone, + UnpackedBytesReceivedKVCrossZone: stmtExecInfo1.TiKVExecDetails.UnpackedBytesReceivedKVCrossZone, + UnpackedBytesSentMPPTotal: stmtExecInfo1.TiKVExecDetails.UnpackedBytesSentMPPTotal, + UnpackedBytesReceivedMPPTotal: stmtExecInfo1.TiKVExecDetails.UnpackedBytesReceivedMPPTotal, + UnpackedBytesSentMPPCrossZone: stmtExecInfo1.TiKVExecDetails.UnpackedBytesSentMPPCrossZone, + UnpackedBytesReceivedMPPCrossZone: stmtExecInfo1.TiKVExecDetails.UnpackedBytesReceivedMPPCrossZone, + }, }, } stmtExecInfo1.ExecDetail.CommitDetail.Mu.Unlock() @@ -235,12 +245,18 @@ func TestAddStatement(t *testing.T) { }, CalleeAddress: "202", }, }, - StmtCtx: sc, - MemMax: 20000, - DiskMax: 20000, - StartTime: time.Date(2019, 1, 1, 10, 10, 20, 10, time.UTC), - Succeed: true, - RUDetail: util.NewRUDetailsWith(123.0, 45.6, 2*time.Second), + StmtCtx: sc, + MemMax: 20000, + DiskMax: 20000, + StartTime: time.Date(2019, 1, 1, 10, 10, 20, 10, time.UTC), + Succeed: true, + RUDetail: util.NewRUDetailsWith(123.0, 45.6, 2*time.Second), + TiKVExecDetails: util.ExecDetails{ + TrafficDetails: util.TrafficDetails{ + UnpackedBytesSentKVTotal: 100, + UnpackedBytesReceivedKVTotal: 200, + }, + }, ResourceGroupName: "rg1", LazyInfo: &mockLazyInfo{ originalSQL: "original_sql2", @@ -311,6 +327,7 @@ func TestAddStatement(t *testing.T) { expectedSummaryElement.MaxWRU = stmtExecInfo2.RUDetail.WRU() expectedSummaryElement.SumRUWaitDuration += stmtExecInfo2.RUDetail.RUWaitDuration() expectedSummaryElement.MaxRUWaitDuration = stmtExecInfo2.RUDetail.RUWaitDuration() + expectedSummaryElement.StmtNetworkTrafficSummary.Add(&stmtExecInfo2.TiKVExecDetails) ssMap.AddStatement(stmtExecInfo2) summary, ok = ssMap.summaryMap.Get(key) @@ -391,6 +408,14 @@ func TestAddStatement(t *testing.T) { Succeed: true, RUDetail: util.NewRUDetailsWith(0.12, 0.34, 5*time.Microsecond), ResourceGroupName: "rg1", + TiKVExecDetails: util.ExecDetails{ + TrafficDetails: util.TrafficDetails{ + UnpackedBytesSentKVTotal: 1, + UnpackedBytesReceivedKVTotal: 300, + UnpackedBytesSentMPPTotal: 1, + UnpackedBytesReceivedMPPTotal: 300, + }, + }, LazyInfo: &mockLazyInfo{ originalSQL: "original_sql3", plan: "", @@ -434,6 +459,7 @@ func TestAddStatement(t *testing.T) { expectedSummaryElement.SumRRU += stmtExecInfo3.RUDetail.RRU() expectedSummaryElement.SumWRU += stmtExecInfo3.RUDetail.WRU() expectedSummaryElement.SumRUWaitDuration += stmtExecInfo3.RUDetail.RUWaitDuration() + expectedSummaryElement.StmtNetworkTrafficSummary.Add(&stmtExecInfo3.TiKVExecDetails) ssMap.AddStatement(stmtExecInfo3) summary, ok = ssMap.summaryMap.Get(key) @@ -579,7 +605,8 @@ func matchStmtSummaryByDigest(first, second *stmtSummaryByDigest) bool { !ssElement1.firstSeen.Equal(ssElement2.firstSeen) || !ssElement1.lastSeen.Equal(ssElement2.lastSeen) || ssElement1.resourceGroupName != ssElement2.resourceGroupName || - ssElement1.StmtRUSummary != ssElement2.StmtRUSummary { + ssElement1.StmtRUSummary != ssElement2.StmtRUSummary || + ssElement1.StmtNetworkTrafficSummary != ssElement2.StmtNetworkTrafficSummary { return false } if len(ssElement1.backoffTypes) != len(ssElement2.backoffTypes) { @@ -695,6 +722,14 @@ func generateAnyExecInfo() *StmtExecInfo { ResourceGroupName: "rg1", RUDetail: util.NewRUDetailsWith(1.1, 2.5, 2*time.Millisecond), CPUUsages: ppcpuusage.CPUUsages{TidbCPUTime: time.Duration(20), TikvCPUTime: time.Duration(100)}, + TiKVExecDetails: util.ExecDetails{ + TrafficDetails: util.TrafficDetails{ + UnpackedBytesSentKVTotal: 10, + UnpackedBytesReceivedKVTotal: 1000, + UnpackedBytesReceivedKVCrossZone: 1, + UnpackedBytesSentKVCrossZone: 100, + }, + }, LazyInfo: &mockLazyInfo{ originalSQL: "original_sql1", plan: "", diff --git a/pkg/util/stmtsummary/v2/column.go b/pkg/util/stmtsummary/v2/column.go index 9694cd8e80521..08e964640e9f9 100644 --- a/pkg/util/stmtsummary/v2/column.go +++ b/pkg/util/stmtsummary/v2/column.go @@ -32,113 +32,121 @@ import ( // Statements summary table column name. const ( - ClusterTableInstanceColumnNameStr = "INSTANCE" - SummaryBeginTimeStr = "SUMMARY_BEGIN_TIME" - SummaryEndTimeStr = "SUMMARY_END_TIME" - StmtTypeStr = "STMT_TYPE" - SchemaNameStr = "SCHEMA_NAME" - DigestStr = "DIGEST" - DigestTextStr = "DIGEST_TEXT" - TableNamesStr = "TABLE_NAMES" - IndexNamesStr = "INDEX_NAMES" - SampleUserStr = "SAMPLE_USER" - ExecCountStr = "EXEC_COUNT" - SumErrorsStr = "SUM_ERRORS" - SumWarningsStr = "SUM_WARNINGS" - SumLatencyStr = "SUM_LATENCY" - MaxLatencyStr = "MAX_LATENCY" - MinLatencyStr = "MIN_LATENCY" - AvgLatencyStr = "AVG_LATENCY" - AvgParseLatencyStr = "AVG_PARSE_LATENCY" - MaxParseLatencyStr = "MAX_PARSE_LATENCY" - AvgCompileLatencyStr = "AVG_COMPILE_LATENCY" - MaxCompileLatencyStr = "MAX_COMPILE_LATENCY" - SumCopTaskNumStr = "SUM_COP_TASK_NUM" - MaxCopProcessTimeStr = "MAX_COP_PROCESS_TIME" - MaxCopProcessAddressStr = "MAX_COP_PROCESS_ADDRESS" - MaxCopWaitTimeStr = "MAX_COP_WAIT_TIME" // #nosec G101 - MaxCopWaitAddressStr = "MAX_COP_WAIT_ADDRESS" // #nosec G101 - AvgProcessTimeStr = "AVG_PROCESS_TIME" - MaxProcessTimeStr = "MAX_PROCESS_TIME" - AvgWaitTimeStr = "AVG_WAIT_TIME" - MaxWaitTimeStr = "MAX_WAIT_TIME" - AvgBackoffTimeStr = "AVG_BACKOFF_TIME" - MaxBackoffTimeStr = "MAX_BACKOFF_TIME" - AvgTotalKeysStr = "AVG_TOTAL_KEYS" - MaxTotalKeysStr = "MAX_TOTAL_KEYS" - AvgProcessedKeysStr = "AVG_PROCESSED_KEYS" - MaxProcessedKeysStr = "MAX_PROCESSED_KEYS" - AvgRocksdbDeleteSkippedCountStr = "AVG_ROCKSDB_DELETE_SKIPPED_COUNT" - MaxRocksdbDeleteSkippedCountStr = "MAX_ROCKSDB_DELETE_SKIPPED_COUNT" - AvgRocksdbKeySkippedCountStr = "AVG_ROCKSDB_KEY_SKIPPED_COUNT" - MaxRocksdbKeySkippedCountStr = "MAX_ROCKSDB_KEY_SKIPPED_COUNT" - AvgRocksdbBlockCacheHitCountStr = "AVG_ROCKSDB_BLOCK_CACHE_HIT_COUNT" - MaxRocksdbBlockCacheHitCountStr = "MAX_ROCKSDB_BLOCK_CACHE_HIT_COUNT" - AvgRocksdbBlockReadCountStr = "AVG_ROCKSDB_BLOCK_READ_COUNT" - MaxRocksdbBlockReadCountStr = "MAX_ROCKSDB_BLOCK_READ_COUNT" - AvgRocksdbBlockReadByteStr = "AVG_ROCKSDB_BLOCK_READ_BYTE" - MaxRocksdbBlockReadByteStr = "MAX_ROCKSDB_BLOCK_READ_BYTE" - AvgPrewriteTimeStr = "AVG_PREWRITE_TIME" - MaxPrewriteTimeStr = "MAX_PREWRITE_TIME" - AvgCommitTimeStr = "AVG_COMMIT_TIME" - MaxCommitTimeStr = "MAX_COMMIT_TIME" - AvgGetCommitTsTimeStr = "AVG_GET_COMMIT_TS_TIME" - MaxGetCommitTsTimeStr = "MAX_GET_COMMIT_TS_TIME" - AvgCommitBackoffTimeStr = "AVG_COMMIT_BACKOFF_TIME" - MaxCommitBackoffTimeStr = "MAX_COMMIT_BACKOFF_TIME" - AvgResolveLockTimeStr = "AVG_RESOLVE_LOCK_TIME" - MaxResolveLockTimeStr = "MAX_RESOLVE_LOCK_TIME" - AvgLocalLatchWaitTimeStr = "AVG_LOCAL_LATCH_WAIT_TIME" - MaxLocalLatchWaitTimeStr = "MAX_LOCAL_LATCH_WAIT_TIME" - AvgWriteKeysStr = "AVG_WRITE_KEYS" - MaxWriteKeysStr = "MAX_WRITE_KEYS" - AvgWriteSizeStr = "AVG_WRITE_SIZE" - MaxWriteSizeStr = "MAX_WRITE_SIZE" - AvgPrewriteRegionsStr = "AVG_PREWRITE_REGIONS" - MaxPrewriteRegionsStr = "MAX_PREWRITE_REGIONS" - AvgTxnRetryStr = "AVG_TXN_RETRY" - MaxTxnRetryStr = "MAX_TXN_RETRY" - SumExecRetryStr = "SUM_EXEC_RETRY" - SumExecRetryTimeStr = "SUM_EXEC_RETRY_TIME" - SumBackoffTimesStr = "SUM_BACKOFF_TIMES" - BackoffTypesStr = "BACKOFF_TYPES" - AvgMemStr = "AVG_MEM" - MaxMemStr = "MAX_MEM" - AvgDiskStr = "AVG_DISK" - MaxDiskStr = "MAX_DISK" - AvgKvTimeStr = "AVG_KV_TIME" - AvgPdTimeStr = "AVG_PD_TIME" - AvgBackoffTotalTimeStr = "AVG_BACKOFF_TOTAL_TIME" - AvgWriteSQLRespTimeStr = "AVG_WRITE_SQL_RESP_TIME" - AvgTidbCPUTimeStr = "AVG_TIDB_CPU_TIME" - AvgTikvCPUTimeStr = "AVG_TIKV_CPU_TIME" - MaxResultRowsStr = "MAX_RESULT_ROWS" - MinResultRowsStr = "MIN_RESULT_ROWS" - AvgResultRowsStr = "AVG_RESULT_ROWS" - PreparedStr = "PREPARED" - AvgAffectedRowsStr = "AVG_AFFECTED_ROWS" - FirstSeenStr = "FIRST_SEEN" - LastSeenStr = "LAST_SEEN" - PlanInCacheStr = "PLAN_IN_CACHE" - PlanCacheHitsStr = "PLAN_CACHE_HITS" - PlanCacheUnqualifiedStr = "PLAN_CACHE_UNQUALIFIED" - PlanCacheUnqualifiedLastReasonStr = "PLAN_CACHE_UNQUALIFIED_LAST_REASON" - PlanInBindingStr = "PLAN_IN_BINDING" - QuerySampleTextStr = "QUERY_SAMPLE_TEXT" - PrevSampleTextStr = "PREV_SAMPLE_TEXT" - PlanDigestStr = "PLAN_DIGEST" - PlanStr = "PLAN" - BinaryPlan = "BINARY_PLAN" - Charset = "CHARSET" - Collation = "COLLATION" - PlanHint = "PLAN_HINT" - AvgRequestUnitRead = "AVG_REQUEST_UNIT_READ" - MaxRequestUnitRead = "MAX_REQUEST_UNIT_READ" - AvgRequestUnitWrite = "AVG_REQUEST_UNIT_WRITE" - MaxRequestUnitWrite = "MAX_REQUEST_UNIT_WRITE" - AvgQueuedRcTimeStr = "AVG_QUEUED_RC_TIME" - MaxQueuedRcTimeStr = "MAX_QUEUED_RC_TIME" - ResourceGroupName = "RESOURCE_GROUP" + ClusterTableInstanceColumnNameStr = "INSTANCE" + SummaryBeginTimeStr = "SUMMARY_BEGIN_TIME" + SummaryEndTimeStr = "SUMMARY_END_TIME" + StmtTypeStr = "STMT_TYPE" + SchemaNameStr = "SCHEMA_NAME" + DigestStr = "DIGEST" + DigestTextStr = "DIGEST_TEXT" + TableNamesStr = "TABLE_NAMES" + IndexNamesStr = "INDEX_NAMES" + SampleUserStr = "SAMPLE_USER" + ExecCountStr = "EXEC_COUNT" + SumErrorsStr = "SUM_ERRORS" + SumWarningsStr = "SUM_WARNINGS" + SumLatencyStr = "SUM_LATENCY" + MaxLatencyStr = "MAX_LATENCY" + MinLatencyStr = "MIN_LATENCY" + AvgLatencyStr = "AVG_LATENCY" + AvgParseLatencyStr = "AVG_PARSE_LATENCY" + MaxParseLatencyStr = "MAX_PARSE_LATENCY" + AvgCompileLatencyStr = "AVG_COMPILE_LATENCY" + MaxCompileLatencyStr = "MAX_COMPILE_LATENCY" + SumCopTaskNumStr = "SUM_COP_TASK_NUM" + MaxCopProcessTimeStr = "MAX_COP_PROCESS_TIME" + MaxCopProcessAddressStr = "MAX_COP_PROCESS_ADDRESS" + MaxCopWaitTimeStr = "MAX_COP_WAIT_TIME" // #nosec G101 + MaxCopWaitAddressStr = "MAX_COP_WAIT_ADDRESS" // #nosec G101 + AvgProcessTimeStr = "AVG_PROCESS_TIME" + MaxProcessTimeStr = "MAX_PROCESS_TIME" + AvgWaitTimeStr = "AVG_WAIT_TIME" + MaxWaitTimeStr = "MAX_WAIT_TIME" + AvgBackoffTimeStr = "AVG_BACKOFF_TIME" + MaxBackoffTimeStr = "MAX_BACKOFF_TIME" + AvgTotalKeysStr = "AVG_TOTAL_KEYS" + MaxTotalKeysStr = "MAX_TOTAL_KEYS" + AvgProcessedKeysStr = "AVG_PROCESSED_KEYS" + MaxProcessedKeysStr = "MAX_PROCESSED_KEYS" + AvgRocksdbDeleteSkippedCountStr = "AVG_ROCKSDB_DELETE_SKIPPED_COUNT" + MaxRocksdbDeleteSkippedCountStr = "MAX_ROCKSDB_DELETE_SKIPPED_COUNT" + AvgRocksdbKeySkippedCountStr = "AVG_ROCKSDB_KEY_SKIPPED_COUNT" + MaxRocksdbKeySkippedCountStr = "MAX_ROCKSDB_KEY_SKIPPED_COUNT" + AvgRocksdbBlockCacheHitCountStr = "AVG_ROCKSDB_BLOCK_CACHE_HIT_COUNT" + MaxRocksdbBlockCacheHitCountStr = "MAX_ROCKSDB_BLOCK_CACHE_HIT_COUNT" + AvgRocksdbBlockReadCountStr = "AVG_ROCKSDB_BLOCK_READ_COUNT" + MaxRocksdbBlockReadCountStr = "MAX_ROCKSDB_BLOCK_READ_COUNT" + AvgRocksdbBlockReadByteStr = "AVG_ROCKSDB_BLOCK_READ_BYTE" + MaxRocksdbBlockReadByteStr = "MAX_ROCKSDB_BLOCK_READ_BYTE" + AvgPrewriteTimeStr = "AVG_PREWRITE_TIME" + MaxPrewriteTimeStr = "MAX_PREWRITE_TIME" + AvgCommitTimeStr = "AVG_COMMIT_TIME" + MaxCommitTimeStr = "MAX_COMMIT_TIME" + AvgGetCommitTsTimeStr = "AVG_GET_COMMIT_TS_TIME" + MaxGetCommitTsTimeStr = "MAX_GET_COMMIT_TS_TIME" + AvgCommitBackoffTimeStr = "AVG_COMMIT_BACKOFF_TIME" + MaxCommitBackoffTimeStr = "MAX_COMMIT_BACKOFF_TIME" + AvgResolveLockTimeStr = "AVG_RESOLVE_LOCK_TIME" + MaxResolveLockTimeStr = "MAX_RESOLVE_LOCK_TIME" + AvgLocalLatchWaitTimeStr = "AVG_LOCAL_LATCH_WAIT_TIME" + MaxLocalLatchWaitTimeStr = "MAX_LOCAL_LATCH_WAIT_TIME" + AvgWriteKeysStr = "AVG_WRITE_KEYS" + MaxWriteKeysStr = "MAX_WRITE_KEYS" + AvgWriteSizeStr = "AVG_WRITE_SIZE" + MaxWriteSizeStr = "MAX_WRITE_SIZE" + AvgPrewriteRegionsStr = "AVG_PREWRITE_REGIONS" + MaxPrewriteRegionsStr = "MAX_PREWRITE_REGIONS" + AvgTxnRetryStr = "AVG_TXN_RETRY" + MaxTxnRetryStr = "MAX_TXN_RETRY" + SumExecRetryStr = "SUM_EXEC_RETRY" + SumExecRetryTimeStr = "SUM_EXEC_RETRY_TIME" + SumBackoffTimesStr = "SUM_BACKOFF_TIMES" + BackoffTypesStr = "BACKOFF_TYPES" + AvgMemStr = "AVG_MEM" + MaxMemStr = "MAX_MEM" + AvgDiskStr = "AVG_DISK" + MaxDiskStr = "MAX_DISK" + AvgKvTimeStr = "AVG_KV_TIME" + AvgPdTimeStr = "AVG_PD_TIME" + AvgBackoffTotalTimeStr = "AVG_BACKOFF_TOTAL_TIME" + AvgWriteSQLRespTimeStr = "AVG_WRITE_SQL_RESP_TIME" + AvgTidbCPUTimeStr = "AVG_TIDB_CPU_TIME" + AvgTikvCPUTimeStr = "AVG_TIKV_CPU_TIME" + MaxResultRowsStr = "MAX_RESULT_ROWS" + MinResultRowsStr = "MIN_RESULT_ROWS" + AvgResultRowsStr = "AVG_RESULT_ROWS" + PreparedStr = "PREPARED" + AvgAffectedRowsStr = "AVG_AFFECTED_ROWS" + FirstSeenStr = "FIRST_SEEN" + LastSeenStr = "LAST_SEEN" + PlanInCacheStr = "PLAN_IN_CACHE" + PlanCacheHitsStr = "PLAN_CACHE_HITS" + PlanCacheUnqualifiedStr = "PLAN_CACHE_UNQUALIFIED" + PlanCacheUnqualifiedLastReasonStr = "PLAN_CACHE_UNQUALIFIED_LAST_REASON" + PlanInBindingStr = "PLAN_IN_BINDING" + QuerySampleTextStr = "QUERY_SAMPLE_TEXT" + PrevSampleTextStr = "PREV_SAMPLE_TEXT" + PlanDigestStr = "PLAN_DIGEST" + PlanStr = "PLAN" + BinaryPlan = "BINARY_PLAN" + Charset = "CHARSET" + Collation = "COLLATION" + PlanHint = "PLAN_HINT" + AvgRequestUnitRead = "AVG_REQUEST_UNIT_READ" + MaxRequestUnitRead = "MAX_REQUEST_UNIT_READ" + AvgRequestUnitWrite = "AVG_REQUEST_UNIT_WRITE" + MaxRequestUnitWrite = "MAX_REQUEST_UNIT_WRITE" + AvgQueuedRcTimeStr = "AVG_QUEUED_RC_TIME" + MaxQueuedRcTimeStr = "MAX_QUEUED_RC_TIME" + ResourceGroupName = "RESOURCE_GROUP" + SumUnpackedBytesSentKVTotalStr = "SUM_UNPACKED_BYTES_SENT_KV_TOTAL" + SumUnpackedBytesReceivedKVTotalStr = "SUM_UNPACKED_BYTES_RECEIVED_KV_TOTAL" + SumUnpackedBytesSentKVCrossZoneStr = "SUM_UNPACKED_BYTES_SENT_KV_CROSS_ZONE" + SumUnpackedBytesReceivedKVCrossZoneStr = "SUM_UNPACKED_BYTES_RECEIVED_KV_CROSS_ZONE" + SumUnpackedBytesSentMPPTotalStr = "SUM_UNPACKED_BYTES_SENT_MPP_TOTAL" + SumUnpackedBytesReceivedMPPTotalStr = "SUM_UNPACKED_BYTES_RECEIVED_MPP_TOTAL" + SumUnpackedBytesSentMPPCrossZoneStr = "SUM_UNPACKED_BYTES_SENT_MPP_CROSS_ZONE" + SumUnpackedBytesReceiveMPPCrossZoneStr = "SUM_UNPACKED_BYTES_RECEIVED_MPP_CROSS_ZONE" ) type columnInfo interface { @@ -498,6 +506,30 @@ var columnFactoryMap = map[string]columnFactory{ PlanCacheUnqualifiedLastReasonStr: func(_ columnInfo, record *StmtRecord) any { return record.PlanCacheUnqualifiedLastReason }, + SumUnpackedBytesSentKVTotalStr: func(_ columnInfo, record *StmtRecord) any { + return record.UnpackedBytesSentKVTotal + }, + SumUnpackedBytesReceivedKVTotalStr: func(_ columnInfo, record *StmtRecord) any { + return record.UnpackedBytesReceivedKVTotal + }, + SumUnpackedBytesSentKVCrossZoneStr: func(_ columnInfo, record *StmtRecord) any { + return record.UnpackedBytesSentKVCrossZone + }, + SumUnpackedBytesReceivedKVCrossZoneStr: func(_ columnInfo, record *StmtRecord) any { + return record.UnpackedBytesReceivedKVCrossZone + }, + SumUnpackedBytesSentMPPTotalStr: func(_ columnInfo, record *StmtRecord) any { + return record.UnpackedBytesSentMPPTotal + }, + SumUnpackedBytesReceivedMPPTotalStr: func(_ columnInfo, record *StmtRecord) any { + return record.UnpackedBytesReceivedMPPTotal + }, + SumUnpackedBytesSentMPPCrossZoneStr: func(_ columnInfo, record *StmtRecord) any { + return record.UnpackedBytesSentMPPCrossZone + }, + SumUnpackedBytesReceiveMPPCrossZoneStr: func(_ columnInfo, record *StmtRecord) any { + return record.UnpackedBytesReceivedMPPCrossZone + }, } func makeColumnFactories(columns []*model.ColumnInfo) []columnFactory { diff --git a/pkg/util/stmtsummary/v2/record.go b/pkg/util/stmtsummary/v2/record.go index 62e011874f384..9de4b19958c19 100644 --- a/pkg/util/stmtsummary/v2/record.go +++ b/pkg/util/stmtsummary/v2/record.go @@ -156,6 +156,8 @@ type StmtRecord struct { PlanCacheUnqualifiedCount int64 `json:"plan_cache_unqualified_count"` PlanCacheUnqualifiedLastReason string `json:"plan_cache_unqualified_last_reason"` // the reason why this query is unqualified for the plan cache + + stmtsummary.StmtNetworkTrafficSummary } // NewStmtRecord creates a new StmtRecord from StmtExecInfo. @@ -420,6 +422,9 @@ func (r *StmtRecord) Add(info *stmtsummary.StmtExecInfo) { r.SumWriteSQLRespTotal += info.StmtExecDetails.WriteSQLRespDuration r.SumTidbCPU += info.CPUUsages.TidbCPUTime r.SumTikvCPU += info.CPUUsages.TikvCPUTime + + // Newroks + r.StmtNetworkTrafficSummary.Add(&info.TiKVExecDetails) // RU r.StmtRUSummary.Add(info.RUDetail) } diff --git a/tests/integrationtest/r/index_merge.result b/tests/integrationtest/r/index_merge.result index f2e94be8a354f..1110414a2a790 100644 --- a/tests/integrationtest/r/index_merge.result +++ b/tests/integrationtest/r/index_merge.result @@ -453,8 +453,8 @@ c1 c2 c3 ///// MEMORY Table explain select count(c1) from (select /*+ use_index_merge(t_alias), stream_agg() */ count(1) c1 from information_schema.statements_summary where sum_latency >= 0 or max_latency >= 0 order by 1) dt; id estRows task access object operator info -StreamAgg_9 1.00 root funcs:count(Column#107)->Column#108 -└─StreamAgg_11 1.00 root funcs:count(1)->Column#107 +StreamAgg_9 1.00 root funcs:count(Column#115)->Column#116 +└─StreamAgg_11 1.00 root funcs:count(1)->Column#115 └─MemTableScan_15 10000.00 root table:STATEMENTS_SUMMARY show warnings; Level Code Message From d1ed962f95f862696b0837cd1eb67c1ab3ce5173 Mon Sep 17 00:00:00 2001 From: Hangjie Mo Date: Wed, 8 Jan 2025 14:01:28 +0800 Subject: [PATCH 06/27] ddl: supports non-unique global index (#58678) close pingcap/tidb#58650 --- pkg/ddl/executor.go | 16 +-- pkg/ddl/index_modify_test.go | 45 +++++- pkg/ddl/ingest/integration_test.go | 9 ++ pkg/ddl/partition.go | 27 +--- .../integrationtest/r/globalindex/ddl.result | 128 +++++++++++++----- .../r/globalindex/expression_index.result | 10 ++ .../r/globalindex/insert.result | 17 +++ .../r/globalindex/update.result | 11 ++ tests/integrationtest/t/globalindex/ddl.test | 71 +++++----- .../t/globalindex/expression_index.test | 12 ++ .../integrationtest/t/globalindex/insert.test | 9 ++ .../integrationtest/t/globalindex/update.test | 11 ++ 12 files changed, 265 insertions(+), 101 deletions(-) diff --git a/pkg/ddl/executor.go b/pkg/ddl/executor.go index b35f577dbdba9..8b45f286483b0 100644 --- a/pkg/ddl/executor.go +++ b/pkg/ddl/executor.go @@ -958,10 +958,6 @@ func checkGlobalIndex(ec errctx.Context, tblInfo *model.TableInfo, indexInfo *mo // partitioning an index differently from the table partitioning. return dbterror.ErrGeneralUnsupportedDDL.GenWithStackByArgs("Global Index on non-partitioned table") } - // TODO: remove limitation - if !indexInfo.Unique { - return dbterror.ErrGeneralUnsupportedDDL.GenWithStackByArgs("GLOBAL IndexOption on non-unique index") - } validateGlobalIndexWithGeneratedColumns(ec, tblInfo, indexInfo.Name.O, indexInfo.Columns) } return nil @@ -4534,15 +4530,11 @@ func GetName4AnonymousIndex(t table.Table, colName ast.CIStr, idxName ast.CIStr) return indexName } -func checkCreateUniqueGlobalIndex(ec errctx.Context, tblInfo *model.TableInfo, indexName string, indexColumns []*model.IndexColumn, isUnique bool, isGlobal bool) error { +func checkCreateGlobalIndex(ec errctx.Context, tblInfo *model.TableInfo, indexName string, indexColumns []*model.IndexColumn, isUnique bool, isGlobal bool) error { pi := tblInfo.GetPartitionInfo() if isGlobal && pi == nil { return dbterror.ErrGeneralUnsupportedDDL.GenWithStackByArgs("Global Index on non-partitioned table") } - if isGlobal && !isUnique { - // TODO: remove this limitation - return dbterror.ErrGeneralUnsupportedDDL.GenWithStackByArgs("Global IndexOption on non-unique index") - } if isUnique && pi != nil { ck, err := checkPartitionKeysConstraint(tblInfo.GetPartitionInfo(), indexColumns, tblInfo) if err != nil { @@ -4552,6 +4544,8 @@ func checkCreateUniqueGlobalIndex(ec errctx.Context, tblInfo *model.TableInfo, i // index columns does not contain all partition columns, must be global return dbterror.ErrGlobalIndexNotExplicitlySet.GenWithStackByArgs(indexName) } + } + if isGlobal { validateGlobalIndexWithGeneratedColumns(ec, tblInfo, indexName, indexColumns) } return nil @@ -4602,7 +4596,7 @@ func (e *executor) CreatePrimaryKey(ctx sessionctx.Context, ti ast.Ident, indexN return err } - if err = checkCreateUniqueGlobalIndex(ctx.GetSessionVars().StmtCtx.ErrCtx(), tblInfo, "PRIMARY", indexColumns, true, indexOption != nil && indexOption.Global); err != nil { + if err = checkCreateGlobalIndex(ctx.GetSessionVars().StmtCtx.ErrCtx(), tblInfo, "PRIMARY", indexColumns, true, indexOption != nil && indexOption.Global); err != nil { return err } @@ -4871,7 +4865,7 @@ func (e *executor) createIndex(ctx sessionctx.Context, ti ast.Ident, keyType ast return errors.Trace(err) } - if err = checkCreateUniqueGlobalIndex(ctx.GetSessionVars().StmtCtx.ErrCtx(), tblInfo, indexName.O, indexColumns, unique, indexOption != nil && indexOption.Global); err != nil { + if err = checkCreateGlobalIndex(ctx.GetSessionVars().StmtCtx.ErrCtx(), tblInfo, indexName.O, indexColumns, unique, indexOption != nil && indexOption.Global); err != nil { return err } diff --git a/pkg/ddl/index_modify_test.go b/pkg/ddl/index_modify_test.go index 999e4fe2b6994..6ad42279fe3f9 100644 --- a/pkg/ddl/index_modify_test.go +++ b/pkg/ddl/index_modify_test.go @@ -749,6 +749,39 @@ func TestAddGlobalIndex(t *testing.T) { require.NoError(t, txn.Commit(context.Background())) + // Test add non-unqiue global index + tk.MustExec("drop table if exists test_t2") + tk.MustExec("create table test_t2 (a int, b int) partition by range (b)" + + " (partition p0 values less than (10), " + + " partition p1 values less than (maxvalue));") + tk.MustExec("insert test_t2 values (2, 1)") + tk.MustExec("alter table test_t2 add key p_a (a) global") + tk.MustExec("insert test_t2 values (1, 11)") + tbl = external.GetTableByName(t, tk, "test", "test_t2") + tblInfo = tbl.Meta() + indexInfo = tblInfo.FindIndexByName("p_a") + require.NotNil(t, indexInfo) + require.True(t, indexInfo.Global) + require.False(t, indexInfo.Unique) + + require.NoError(t, sessiontxn.NewTxn(context.Background(), tk.Session())) + txn, err = tk.Session().Txn(true) + require.NoError(t, err) + + // check row 1 + pid = tblInfo.Partition.Definitions[0].ID + idxVals = []types.Datum{types.NewDatum(2)} + rowVals = []types.Datum{types.NewDatum(2), types.NewDatum(1)} + checkGlobalIndexRow(t, tk.Session(), tblInfo, indexInfo, pid, idxVals, rowVals) + + // check row 2 + pid = tblInfo.Partition.Definitions[1].ID + idxVals = []types.Datum{types.NewDatum(1)} + rowVals = []types.Datum{types.NewDatum(1), types.NewDatum(11)} + checkGlobalIndexRow(t, tk.Session(), tblInfo, indexInfo, pid, idxVals, rowVals) + + require.NoError(t, txn.Commit(context.Background())) + // `sanity_check.go` will check the del_range numbers are correct or not. // normal index tk.MustExec("drop table if exists t") @@ -801,7 +834,17 @@ func checkGlobalIndexRow( require.NoError(t, err) key := tablecodec.EncodeIndexSeekKey(tblInfo.ID, indexInfo.ID, encodedValue) require.NoError(t, err) - value, err := txn.Get(context.Background(), key) + var value []byte + if indexInfo.Unique { + value, err = txn.Get(context.Background(), key) + } else { + var iter kv.Iterator + iter, err = txn.Iter(key, key.PrefixNext()) + require.NoError(t, err) + require.True(t, iter.Valid()) + key = iter.Key() + value = iter.Value() + } require.NoError(t, err) idxColInfos := tables.BuildRowcodecColInfoForIndexColumns(indexInfo, tblInfo) colVals, err := tablecodec.DecodeIndexKV(key, value, len(indexInfo.Columns), tablecodec.HandleDefault, idxColInfos) diff --git a/pkg/ddl/ingest/integration_test.go b/pkg/ddl/ingest/integration_test.go index 5f75579d450c3..e96eaae5ef585 100644 --- a/pkg/ddl/ingest/integration_test.go +++ b/pkg/ddl/ingest/integration_test.go @@ -494,6 +494,15 @@ func TestAddGlobalIndexInIngest(t *testing.T) { require.Greater(t, len(rsGlobalIndex1.Rows()), len(rsGlobalIndex.Rows())) require.Equal(t, rsGlobalIndex1.String(), rsTable.String()) require.Equal(t, rsGlobalIndex1.String(), rsGlobalIndex2.String()) + + // for non-unique global idnexes + tk.MustExec("alter table t add index idx_7(b) global, add index idx_8(b) global") + rsNonUniqueGlobalIndex1 := tk.MustQuery("select * from t use index(idx_7)").Sort() + rsTable = tk.MustQuery("select * from t use index()").Sort() + rsNonUniqueGlobalIndex2 := tk.MustQuery("select * from t use index(idx_8)").Sort() + require.Greater(t, len(rsNonUniqueGlobalIndex1.Rows()), len(rsGlobalIndex.Rows())) + require.Equal(t, rsNonUniqueGlobalIndex1.String(), rsTable.String()) + require.Equal(t, rsNonUniqueGlobalIndex1.String(), rsNonUniqueGlobalIndex2.String()) } func TestAddGlobalIndexInIngestWithUpdate(t *testing.T) { diff --git a/pkg/ddl/partition.go b/pkg/ddl/partition.go index 0600caee75c7b..9d5a4d4914fd1 100644 --- a/pkg/ddl/partition.go +++ b/pkg/ddl/partition.go @@ -3233,28 +3233,18 @@ func (w *worker) onReorganizePartition(jobCtx *jobContext, job *model.Job) (ver // When removing partitioning, set all indexes to 'local' since it will become a non-partitioned table! newGlobal = false } - if !index.Unique { - // for now, only unique index can be global, non-unique indexes are 'local' - // TODO: For the future loosen this restriction and allow non-unique global indexes - if newGlobal { - job.State = model.JobStateCancelled - return ver, dbterror.ErrGeneralUnsupportedDDL.GenWithStackByArgs(fmt.Sprintf("PARTITION BY, index '%v' is not unique, but has Global Index set", index.Name.O)) - } + if !index.Global && !newGlobal { continue } inAllPartitionColumns, err := checkPartitionKeysConstraint(partInfo, index.Columns, tblInfo) if err != nil { return ver, errors.Trace(err) } - // Currently only support Explicit Global indexes. - if !inAllPartitionColumns && !newGlobal { + // Currently only support Explicit Global indexes for unique index. + if !inAllPartitionColumns && !newGlobal && index.Unique { job.State = model.JobStateCancelled return ver, dbterror.ErrGlobalIndexNotExplicitlySet.GenWithStackByArgs(index.Name.O) } - if !index.Global && !newGlobal { - // still local index, no need to duplicate index. - continue - } if tblInfo.Partition.DDLChangedIndex == nil { tblInfo.Partition.DDLChangedIndex = make(map[int64]bool) } @@ -3389,7 +3379,7 @@ func (w *worker) onReorganizePartition(jobCtx *jobContext, job *model.Job) (ver } for i := range tblInfo.Indices { - if tblInfo.Indices[i].Unique && tblInfo.Indices[i].State == model.StateDeleteOnly { + if tblInfo.Indices[i].State == model.StateDeleteOnly { tblInfo.Indices[i].State = model.StateWriteOnly } } @@ -3409,7 +3399,7 @@ func (w *worker) onReorganizePartition(jobCtx *jobContext, job *model.Job) (ver // so that new data will be updated in both old and new partitions when reorganizing. job.SnapshotVer = 0 for i := range tblInfo.Indices { - if tblInfo.Indices[i].Unique && tblInfo.Indices[i].State == model.StateWriteOnly { + if tblInfo.Indices[i].State == model.StateWriteOnly { tblInfo.Indices[i].State = model.StateWriteReorganization } } @@ -3451,9 +3441,6 @@ func (w *worker) onReorganizePartition(jobCtx *jobContext, job *model.Job) (ver }) for _, index := range tblInfo.Indices { - if !index.Unique { - continue - } isNew, ok := tblInfo.Partition.DDLChangedIndex[index.ID] if !ok { continue @@ -3553,8 +3540,8 @@ func (w *worker) onReorganizePartition(jobCtx *jobContext, job *model.Job) (ver var dropIndices []*model.IndexInfo for _, indexInfo := range tblInfo.Indices { - if indexInfo.Unique && indexInfo.State == model.StateDeleteOnly { - // Drop the old unique (possible global) index, see onDropIndex + if indexInfo.State == model.StateDeleteOnly { + // Drop the old indexes, see onDropIndex indexInfo.State = model.StateNone DropIndexColumnFlag(tblInfo, indexInfo) RemoveDependentHiddenColumns(tblInfo, indexInfo) diff --git a/tests/integrationtest/r/globalindex/ddl.result b/tests/integrationtest/r/globalindex/ddl.result index 735d7856b1a03..0b4bbdac32471 100644 --- a/tests/integrationtest/r/globalindex/ddl.result +++ b/tests/integrationtest/r/globalindex/ddl.result @@ -5,39 +5,11 @@ Error 8200 (HY000): Unsupported Global Index on non-partitioned table create table t (a int, b int, unique index idx(a) global) partition by hash(b) partitions 3; drop table t; create table t (a int, b int, index idx(a) global) partition by hash(b) partitions 3; -Error 8200 (HY000): Unsupported GLOBAL IndexOption on non-unique index +drop table t; create table t3(a int not null, b int, primary key(a) nonclustered, unique idx_b(b) global) partition by hash(a) partitions 3; drop table t3; create table t (a int primary key nonclustered, b int) partition by hash(b) partitions 3; Error 8264 (HY000): Global Index is needed for index 'PRIMARY', since the unique index is not including all partitioning columns, and GLOBAL is not given as IndexOption -create table t (a int, b int, unique key (a)) partition by hash(a) partitions 3; -alter table t partition by hash(b) partitions 3; -Error 8264 (HY000): Global Index is needed for index 'a', since the unique index is not including all partitioning columns, and GLOBAL is not given as IndexOption -alter table t partition by hash(b) partitions 3 update indexes (a global); -alter table t add index idxErr (b) global; -Error 8200 (HY000): Unsupported Global IndexOption on non-unique index -alter table t add unique index idxOK (b) global; -create index idxErr on t (b) global; -Error 8200 (HY000): Unsupported Global IndexOption on non-unique index -create unique index idxOK2 on t (b) global; -alter table t remove partitioning; -alter table t add index idxErr (b) global; -Error 8200 (HY000): Unsupported Global Index on non-partitioned table -alter table t add unique index idxErr (b) global; -Error 8200 (HY000): Unsupported Global Index on non-partitioned table -create index idxErr on t (b) global; -Error 8200 (HY000): Unsupported Global Index on non-partitioned table -create unique index idxErr on t (b) global; -Error 8200 (HY000): Unsupported Global Index on non-partitioned table -drop table t; -create table t (a int, b int, unique index idx(a) global); -Error 8200 (HY000): Unsupported Global Index on non-partitioned table -create table t (a int, b int, index idx(a) global); -Error 8200 (HY000): Unsupported Global Index on non-partitioned table -create table t (a int, b int, index idx(a) global) partition by hash(b) partitions 3; -Error 8200 (HY000): Unsupported GLOBAL IndexOption on non-unique index -create table t (a int not null, b int, primary key(a) nonclustered, unique idx_b(b) global) partition by hash(a) partitions 3; -drop table t; create table t (a int key global, b int) partition by hash(b) partitions 3; Error 8200 (HY000): Unsupported create an index that is both a global index and a clustered index create table t (a int unique, b int) partition by hash(b) partitions 3; @@ -65,14 +37,34 @@ create table t (a int, b int, unique key (a)) partition by hash(a) partitions 3; alter table t partition by hash(b) partitions 3; Error 8264 (HY000): Global Index is needed for index 'a', since the unique index is not including all partitioning columns, and GLOBAL is not given as IndexOption alter table t partition by hash(b) partitions 3 UPDATE INDEXES (a GLOBAL); +alter table t add index idxOK (b) global; +alter table t add unique index idxOK2 (a) global; +alter table t add unique index idxOK3 (b) global; +create index idxOK4 on t (b) global; +create unique index idxOK5 on t (a) global; +create unique index idxOK6 on t (b) global; +alter table t remove partitioning; +show create table t; +Table Create Table +t CREATE TABLE `t` ( + `a` int DEFAULT NULL, + `b` int DEFAULT NULL, + UNIQUE KEY `a` (`a`), + KEY `idxOK` (`b`), + UNIQUE KEY `idxOK2` (`a`), + UNIQUE KEY `idxOK3` (`b`), + KEY `idxOK4` (`b`), + UNIQUE KEY `idxOK5` (`a`), + UNIQUE KEY `idxOK6` (`b`) +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin alter table t add index idxErr (b) global; -Error 8200 (HY000): Unsupported Global IndexOption on non-unique index -alter table t add unique index idxOK (a) global; -alter table t add unique index idxOK2 (b) global; +Error 8200 (HY000): Unsupported Global Index on non-partitioned table +alter table t add unique index idxErr (b) global; +Error 8200 (HY000): Unsupported Global Index on non-partitioned table create index idxErr on t (b) global; -Error 8200 (HY000): Unsupported Global IndexOption on non-unique index -create unique index idxOK3 on t (a) global; -create unique index idxOK4 on t (b) global; +Error 8200 (HY000): Unsupported Global Index on non-partitioned table +create unique index idxErr on t (b) global; +Error 8200 (HY000): Unsupported Global Index on non-partitioned table drop table t; create table t(a int, b int, primary key (a) nonclustered global); Error 8200 (HY000): Unsupported Global Index on non-partitioned table @@ -82,6 +74,72 @@ create table t(a int, b int, primary key (a) global) partition by hash(a) partit Error 8200 (HY000): Unsupported create an index that is both a global index and a clustered index create table t(a int, b int, primary key (b) global) partition by hash(a) partitions 5; Error 8200 (HY000): Unsupported create an index that is both a global index and a clustered index +create table t(a int, b int, key(a), key(b)) partition by hash(a) partitions 4; +alter table t partition by hash(b) partitions 3 UPDATE INDEXES (a GLOBAL, b LOCAL); +show create table t; +Table Create Table +t CREATE TABLE `t` ( + `a` int DEFAULT NULL, + `b` int DEFAULT NULL, + KEY `b` (`b`), + KEY `a` (`a`) /*T![global_index] GLOBAL */ +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin +PARTITION BY HASH (`b`) PARTITIONS 3 +alter table t partition by hash(b) partitions 3 UPDATE INDEXES (a GLOBAL, b GLOBAL); +show create table t; +Table Create Table +t CREATE TABLE `t` ( + `a` int DEFAULT NULL, + `b` int DEFAULT NULL, + KEY `b` (`b`) /*T![global_index] GLOBAL */, + KEY `a` (`a`) /*T![global_index] GLOBAL */ +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin +PARTITION BY HASH (`b`) PARTITIONS 3 +alter table t partition by hash(b) partitions 3 UPDATE INDEXES (a LOCAL); +show create table t; +Table Create Table +t CREATE TABLE `t` ( + `a` int DEFAULT NULL, + `b` int DEFAULT NULL, + KEY `b` (`b`) /*T![global_index] GLOBAL */, + KEY `a` (`a`) +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin +PARTITION BY HASH (`b`) PARTITIONS 3 +drop table t; +create table t(a int, b int, unique key(a), unique key(b) global) partition by hash(a) partitions 4; +alter table t partition by hash(b) partitions 3 UPDATE INDEXES (a GLOBAL, b LOCAL); +show create table t; +Table Create Table +t CREATE TABLE `t` ( + `a` int DEFAULT NULL, + `b` int DEFAULT NULL, + UNIQUE KEY `a` (`a`) /*T![global_index] GLOBAL */, + UNIQUE KEY `b` (`b`) +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin +PARTITION BY HASH (`b`) PARTITIONS 3 +alter table t partition by hash(b) partitions 3 UPDATE INDEXES (a GLOBAL, b GLOBAL); +show create table t; +Table Create Table +t CREATE TABLE `t` ( + `a` int DEFAULT NULL, + `b` int DEFAULT NULL, + UNIQUE KEY `a` (`a`) /*T![global_index] GLOBAL */, + UNIQUE KEY `b` (`b`) /*T![global_index] GLOBAL */ +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin +PARTITION BY HASH (`b`) PARTITIONS 3 +alter table t partition by hash(b) partitions 3 UPDATE INDEXES (a LOCAL); +Error 8264 (HY000): Global Index is needed for index 'a', since the unique index is not including all partitioning columns, and GLOBAL is not given as IndexOption +alter table t partition by hash(b) partitions 3 UPDATE INDEXES (b LOCAL); +show create table t; +Table Create Table +t CREATE TABLE `t` ( + `a` int DEFAULT NULL, + `b` int DEFAULT NULL, + UNIQUE KEY `a` (`a`) /*T![global_index] GLOBAL */, + UNIQUE KEY `b` (`b`) +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin +PARTITION BY HASH (`b`) PARTITIONS 3 +drop table t; create table t(a int, b int); alter table t add primary key (a) global; Error 8200 (HY000): Unsupported Global Index on non-partitioned table diff --git a/tests/integrationtest/r/globalindex/expression_index.result b/tests/integrationtest/r/globalindex/expression_index.result index d8460fd4d771b..ec2a175682e79 100644 --- a/tests/integrationtest/r/globalindex/expression_index.result +++ b/tests/integrationtest/r/globalindex/expression_index.result @@ -40,3 +40,13 @@ Projection 3333.33 root NULL globalindex__expression_index.t.a, globalindex__exp select * from t partition(p0) use index(idx) where lower(b) > 'c'; a b 5 x +drop table if exists t; +CREATE TABLE `t` ( +`a` int DEFAULT NULL, +`b` char DEFAULT NULL, +KEY `idx` ((lower(`b`))) global +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin +PARTITION BY HASH (`a`) PARTITIONS 5; +show warnings; +Level Code Message +Warning 8265 Auto analyze is not effective for index 'idx', need analyze manually diff --git a/tests/integrationtest/r/globalindex/insert.result b/tests/integrationtest/r/globalindex/insert.result index e873815ea20ff..94bf8a12a5359 100644 --- a/tests/integrationtest/r/globalindex/insert.result +++ b/tests/integrationtest/r/globalindex/insert.result @@ -10,3 +10,20 @@ select * from t use index (idx1) order by a desc; a b 2 5 1 3 +drop table if exists t; +create table t(a int, b int, index idx(a) global) partition by hash(b) partitions 5; +insert into t values (1, 1), (1, 2), (2, 2); +select * from t use index (idx); +a b +1 1 +1 2 +2 2 +alter table t add index idx1(b) global; +insert into t values (2, 4), (3, 4); +select * from t use index (idx1) order by a desc, b; +a b +3 4 +2 2 +2 4 +1 1 +1 2 diff --git a/tests/integrationtest/r/globalindex/update.result b/tests/integrationtest/r/globalindex/update.result index 6a138c912af1c..8ae79d047ba9f 100644 --- a/tests/integrationtest/r/globalindex/update.result +++ b/tests/integrationtest/r/globalindex/update.result @@ -60,3 +60,14 @@ replace into t values ('',826536 ); select * from t; a b 826536 +drop table t; +create table t(a int, b int, index idx(a) global) partition by hash(b) partitions 5; +insert into t values (1, 2), (1, 3), (1, 4); +replace into t values (2, 3); +update t set a = 3, b = 4 where a = 1; +select * from t; +a b +2 3 +3 4 +3 4 +3 4 diff --git a/tests/integrationtest/t/globalindex/ddl.test b/tests/integrationtest/t/globalindex/ddl.test index 3fcbb99f503aa..6807370d228d8 100644 --- a/tests/integrationtest/t/globalindex/ddl.test +++ b/tests/integrationtest/t/globalindex/ddl.test @@ -4,41 +4,13 @@ create table t (a int, b int, unique index idx(a) global); create table t (a int, b int, index idx(a) global); create table t (a int, b int, unique index idx(a) global) partition by hash(b) partitions 3; drop table t; --- error 8200 create table t (a int, b int, index idx(a) global) partition by hash(b) partitions 3; +drop table t; create table t3(a int not null, b int, primary key(a) nonclustered, unique idx_b(b) global) partition by hash(a) partitions 3; drop table t3; -- error 8264 create table t (a int primary key nonclustered, b int) partition by hash(b) partitions 3; -create table t (a int, b int, unique key (a)) partition by hash(a) partitions 3; --- error 8264 -alter table t partition by hash(b) partitions 3; -alter table t partition by hash(b) partitions 3 update indexes (a global); --- error 8200 -alter table t add index idxErr (b) global; -alter table t add unique index idxOK (b) global; --- error 8200 -create index idxErr on t (b) global; -create unique index idxOK2 on t (b) global; -alter table t remove partitioning; --- error 8200 -alter table t add index idxErr (b) global; --- error 8200 -alter table t add unique index idxErr (b) global; --- error 8200 -create index idxErr on t (b) global; --- error 8200 -create unique index idxErr on t (b) global; -drop table t; --- error 8200 -create table t (a int, b int, unique index idx(a) global); --- error 8200 -create table t (a int, b int, index idx(a) global); --- error 8200 -create table t (a int, b int, index idx(a) global) partition by hash(b) partitions 3; -create table t (a int not null, b int, primary key(a) nonclustered, unique idx_b(b) global) partition by hash(a) partitions 3; -drop table t; -- error 8200 create table t (a int key global, b int) partition by hash(b) partitions 3; -- error 8264 @@ -47,6 +19,7 @@ create table t (a int unique, b int) partition by hash(b) partitions 3; create table t (a int unique key, b int) partition by hash(b) partitions 3; -- error 8264 create table t (a int primary key nonclustered, b int) partition by hash(b) partitions 3; + CREATE TABLE `t` ( `a` int(11) NOT NULL, `b` int(11) DEFAULT NULL, @@ -55,20 +28,30 @@ CREATE TABLE `t` ( PARTITION BY HASH (`b`) PARTITIONS 3; show create table t; drop table t; + create table t (a int, b int, unique key (a)) partition by hash(a) partitions 3; -- error 8264 alter table t partition by hash(b) partitions 3; alter table t partition by hash(b) partitions 3 UPDATE INDEXES (a GLOBAL); +alter table t add index idxOK (b) global; +alter table t add unique index idxOK2 (a) global; +alter table t add unique index idxOK3 (b) global; +create index idxOK4 on t (b) global; +create unique index idxOK5 on t (a) global; +create unique index idxOK6 on t (b) global; + +alter table t remove partitioning; +show create table t; -- error 8200 alter table t add index idxErr (b) global; -alter table t add unique index idxOK (a) global; -alter table t add unique index idxOK2 (b) global; +-- error 8200 +alter table t add unique index idxErr (b) global; -- error 8200 create index idxErr on t (b) global; -create unique index idxOK3 on t (a) global; -create unique index idxOK4 on t (b) global; - +-- error 8200 +create unique index idxErr on t (b) global; drop table t; + --error 8200 create table t(a int, b int, primary key (a) nonclustered global); --error 8200 @@ -78,6 +61,26 @@ create table t(a int, b int, primary key (a) global) partition by hash(a) partit --error 8200 create table t(a int, b int, primary key (b) global) partition by hash(a) partitions 5; +create table t(a int, b int, key(a), key(b)) partition by hash(a) partitions 4; +alter table t partition by hash(b) partitions 3 UPDATE INDEXES (a GLOBAL, b LOCAL); +show create table t; +alter table t partition by hash(b) partitions 3 UPDATE INDEXES (a GLOBAL, b GLOBAL); +show create table t; +alter table t partition by hash(b) partitions 3 UPDATE INDEXES (a LOCAL); +show create table t; +drop table t; + +create table t(a int, b int, unique key(a), unique key(b) global) partition by hash(a) partitions 4; +alter table t partition by hash(b) partitions 3 UPDATE INDEXES (a GLOBAL, b LOCAL); +show create table t; +alter table t partition by hash(b) partitions 3 UPDATE INDEXES (a GLOBAL, b GLOBAL); +show create table t; +--error 8264 +alter table t partition by hash(b) partitions 3 UPDATE INDEXES (a LOCAL); +alter table t partition by hash(b) partitions 3 UPDATE INDEXES (b LOCAL); +show create table t; +drop table t; + create table t(a int, b int); --error 8200 alter table t add primary key (a) global; diff --git a/tests/integrationtest/t/globalindex/expression_index.test b/tests/integrationtest/t/globalindex/expression_index.test index de5d36dc1ab2f..52ec1761502ba 100644 --- a/tests/integrationtest/t/globalindex/expression_index.test +++ b/tests/integrationtest/t/globalindex/expression_index.test @@ -23,3 +23,15 @@ select * from t use index(idx) where lower(b) > 'c' order by lower(b); explain format='brief' select * from t partition(p0) use index(idx) where lower(b) > 'c'; select * from t partition(p0) use index(idx) where lower(b) > 'c'; + +# For global index + non-unique index +drop table if exists t; +CREATE TABLE `t` ( + `a` int DEFAULT NULL, + `b` char DEFAULT NULL, + KEY `idx` ((lower(`b`))) global +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin +PARTITION BY HASH (`a`) PARTITIONS 5; + +show warnings; + diff --git a/tests/integrationtest/t/globalindex/insert.test b/tests/integrationtest/t/globalindex/insert.test index 9e641354a341c..cb7457680fee7 100644 --- a/tests/integrationtest/t/globalindex/insert.test +++ b/tests/integrationtest/t/globalindex/insert.test @@ -8,3 +8,12 @@ insert into t values (2, 4), (3, 4) on duplicate key update a=2, b=5; select * from t use index (idx1) order by a desc; +# Test global index + non-unique index +drop table if exists t; +create table t(a int, b int, index idx(a) global) partition by hash(b) partitions 5; +insert into t values (1, 1), (1, 2), (2, 2); +select * from t use index (idx); + +alter table t add index idx1(b) global; +insert into t values (2, 4), (3, 4); +select * from t use index (idx1) order by a desc, b; diff --git a/tests/integrationtest/t/globalindex/update.test b/tests/integrationtest/t/globalindex/update.test index e383f37f589d6..8a026ba854316 100644 --- a/tests/integrationtest/t/globalindex/update.test +++ b/tests/integrationtest/t/globalindex/update.test @@ -43,3 +43,14 @@ create table t(a varchar(70), b mediumint(9), unique index idx_a(a) global, uniq insert into t values ('',826534 ); replace into t values ('',826536 ); select * from t; + +# For global index + non-unique index +drop table t; +create table t(a int, b int, index idx(a) global) partition by hash(b) partitions 5; +insert into t values (1, 2), (1, 3), (1, 4); + +# Replace will not affects, `idx` is a non-unique index. +replace into t values (2, 3); +update t set a = 3, b = 4 where a = 1; +--sorted_result +select * from t; From 74798043c8c1dfedcbe9ddee83ccaf45c4a414b0 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Wed, 8 Jan 2025 15:30:13 +0800 Subject: [PATCH 07/27] planner: prealloc the slices in the SplitCorColAccessCondFromFilters (#58785) close pingcap/tidb#58784 --- pkg/planner/util/path.go | 3 +++ 1 file changed, 3 insertions(+) diff --git a/pkg/planner/util/path.go b/pkg/planner/util/path.go index a8ccf7df379f2..2aba097c4086a 100644 --- a/pkg/planner/util/path.go +++ b/pkg/planner/util/path.go @@ -194,6 +194,7 @@ func (path *AccessPath) SplitCorColAccessCondFromFilters(ctx planctx.PlanContext } access = make([]expression.Expression, len(path.IdxCols)-eqOrInCount) used := make([]bool, len(path.TableFilters)) + usedCnt := 0 for i := eqOrInCount; i < len(path.IdxCols); i++ { matched := false for j, filter := range path.TableFilters { @@ -217,6 +218,7 @@ func (path *AccessPath) SplitCorColAccessCondFromFilters(ctx planctx.PlanContext access[i-eqOrInCount] = filter if path.IdxColLens[i] == types.UnspecifiedLength { used[j] = true + usedCnt++ } break } @@ -225,6 +227,7 @@ func (path *AccessPath) SplitCorColAccessCondFromFilters(ctx planctx.PlanContext break } } + remained = make([]expression.Expression, 0, len(used)-usedCnt) for i, ok := range used { if !ok { remained = append(remained, path.TableFilters[i]) // nozero From 7bee5a71087d4e7f354a79c10d49ea3e3b3f29f7 Mon Sep 17 00:00:00 2001 From: Rustin Date: Wed, 8 Jan 2025 17:06:37 +0800 Subject: [PATCH 08/27] planner: improve warning messages for unsupported HASH_JOIN hints (#58646) close pingcap/tidb#58647 --- .../testdata/enforce_mpp_suite_out.json | 4 +-- pkg/planner/core/casetest/join/join_test.go | 8 ++--- .../testdata/plan_suite_out.json | 8 ++--- pkg/planner/core/exhaust_physical_plans.go | 15 ++++++--- .../physicalplantest/physical_plan.result | 32 +++++++++---------- 5 files changed, 36 insertions(+), 31 deletions(-) diff --git a/pkg/planner/core/casetest/enforcempp/testdata/enforce_mpp_suite_out.json b/pkg/planner/core/casetest/enforcempp/testdata/enforce_mpp_suite_out.json index 1e8cf370aaea0..08cd555488cca 100644 --- a/pkg/planner/core/casetest/enforcempp/testdata/enforce_mpp_suite_out.json +++ b/pkg/planner/core/casetest/enforcempp/testdata/enforce_mpp_suite_out.json @@ -1667,7 +1667,7 @@ " └─TableFullScan_12(Probe) 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ], "Warn": [ - "[planner:1815]We can't use the HASH_JOIN_BUILD or HASH_JOIN_PROBE hint for anti semi join, please check the hint" + "[planner:1815]The HASH_JOIN_BUILD and HASH_JOIN_PROBE hints are not supported for anti semi join with hash join version 1. Please remove these hints" ] }, { @@ -1683,7 +1683,7 @@ ], "Warn": [ "[planner:1815]Some HASH_JOIN_BUILD and HASH_JOIN_PROBE hints cannot be utilized for MPP joins, please check the hints", - "[planner:1815]We can't use the HASH_JOIN_BUILD or HASH_JOIN_PROBE hint for anti semi join, please check the hint", + "[planner:1815]The HASH_JOIN_BUILD and HASH_JOIN_PROBE hints are not supported for anti semi join with hash join version 1. Please remove these hints", "[planner:1815]Some HASH_JOIN_BUILD and HASH_JOIN_PROBE hints cannot be utilized for MPP joins, please check the hints" ] } diff --git a/pkg/planner/core/casetest/join/join_test.go b/pkg/planner/core/casetest/join/join_test.go index 9cf244b75e8f0..a5fb0dcea4e83 100644 --- a/pkg/planner/core/casetest/join/join_test.go +++ b/pkg/planner/core/casetest/join/join_test.go @@ -73,8 +73,8 @@ func TestSemiJoinOrder(t *testing.T) { " └─Selection 9990.00 cop[tikv] not(isnull(test.t1.col0))", " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo")) tk.MustQuery("show warnings").Check(testkit.Rows( - "Warning 1815 We can't use the HASH_JOIN_BUILD or HASH_JOIN_PROBE hint for semi join, please check the hint", - "Warning 1815 We can't use the HASH_JOIN_BUILD or HASH_JOIN_PROBE hint for semi join, please check the hint")) + "Warning 1815 The HASH_JOIN_BUILD and HASH_JOIN_PROBE hints are not supported for semi join with hash join version 1. Please remove these hints", + "Warning 1815 The HASH_JOIN_BUILD and HASH_JOIN_PROBE hints are not supported for semi join with hash join version 1. Please remove these hints")) tk.MustQuery("explain format = 'brief' select /*+ HASH_JOIN_BUILD(t2@sel_2) */ * from t1 where exists (select 1 from t2 where t1.col0 = t2.col0) order by t1.col0, t1.col1;").Check(testkit.Rows( "Sort 7992.00 root test.t1.col0, test.t1.col1", "└─HashJoin 7992.00 root semi join, left side:TableReader, equal:[eq(test.t1.col0, test.t2.col0)]", @@ -85,6 +85,6 @@ func TestSemiJoinOrder(t *testing.T) { " └─Selection 9990.00 cop[tikv] not(isnull(test.t1.col0))", " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo")) tk.MustQuery("show warnings").Check(testkit.Rows( - "Warning 1815 We can't use the HASH_JOIN_BUILD or HASH_JOIN_PROBE hint for semi join, please check the hint", - "Warning 1815 We can't use the HASH_JOIN_BUILD or HASH_JOIN_PROBE hint for semi join, please check the hint")) + "Warning 1815 The HASH_JOIN_BUILD and HASH_JOIN_PROBE hints are not supported for semi join with hash join version 1. Please remove these hints", + "Warning 1815 The HASH_JOIN_BUILD and HASH_JOIN_PROBE hints are not supported for semi join with hash join version 1. Please remove these hints")) } diff --git a/pkg/planner/core/casetest/physicalplantest/testdata/plan_suite_out.json b/pkg/planner/core/casetest/physicalplantest/testdata/plan_suite_out.json index 5ab80ad00909f..16bf9bc7f0ba3 100644 --- a/pkg/planner/core/casetest/physicalplantest/testdata/plan_suite_out.json +++ b/pkg/planner/core/casetest/physicalplantest/testdata/plan_suite_out.json @@ -1097,7 +1097,7 @@ " └─TableFullScan(Probe) 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" ], "Warn": [ - "[planner:1815]We can't use the HASH_JOIN_BUILD or HASH_JOIN_PROBE hint for left outer semi join, please check the hint" + "[planner:1815]HASH_JOIN_BUILD and HASH_JOIN_PROBE hints are not supported for left outer semi join because the build side is fixed. Please remove these hints" ] }, { @@ -1115,7 +1115,7 @@ ], "Warn": [ "[planner:1815]Some HASH_JOIN_BUILD and HASH_JOIN_PROBE hints cannot be utilized for MPP joins, please check the hints", - "[planner:1815]We can't use the HASH_JOIN_BUILD or HASH_JOIN_PROBE hint for anti left outer semi join, please check the hint", + "[planner:1815]HASH_JOIN_BUILD and HASH_JOIN_PROBE hints are not supported for anti left outer semi join because the build side is fixed. Please remove these hints", "[planner:1815]Some HASH_JOIN_BUILD and HASH_JOIN_PROBE hints cannot be utilized for MPP joins, please check the hints" ] }, @@ -1134,7 +1134,7 @@ ], "Warn": [ "[planner:1815]Some HASH_JOIN_BUILD and HASH_JOIN_PROBE hints cannot be utilized for MPP joins, please check the hints", - "[planner:1815]We can't use the HASH_JOIN_BUILD or HASH_JOIN_PROBE hint for left outer semi join, please check the hint", + "[planner:1815]HASH_JOIN_BUILD and HASH_JOIN_PROBE hints are not supported for left outer semi join because the build side is fixed. Please remove these hints", "[planner:1815]Some HASH_JOIN_BUILD and HASH_JOIN_PROBE hints cannot be utilized for MPP joins, please check the hints" ] }, @@ -1152,7 +1152,7 @@ " └─TableFullScan(Probe) 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" ], "Warn": [ - "[planner:1815]We can't use the HASH_JOIN_BUILD or HASH_JOIN_PROBE hint for anti left outer semi join, please check the hint" + "[planner:1815]HASH_JOIN_BUILD and HASH_JOIN_PROBE hints are not supported for anti left outer semi join because the build side is fixed. Please remove these hints" ] } ] diff --git a/pkg/planner/core/exhaust_physical_plans.go b/pkg/planner/core/exhaust_physical_plans.go index 3d0e4088e2a44..6c87bd71d8132 100644 --- a/pkg/planner/core/exhaust_physical_plans.go +++ b/pkg/planner/core/exhaust_physical_plans.go @@ -431,7 +431,8 @@ func getHashJoins(p *logicalop.LogicalJoin, prop *property.PhysicalProperty) (jo forceLeftToBuild := ((p.PreferJoinType & h.PreferLeftAsHJBuild) > 0) || ((p.PreferJoinType & h.PreferRightAsHJProbe) > 0) forceRightToBuild := ((p.PreferJoinType & h.PreferRightAsHJBuild) > 0) || ((p.PreferJoinType & h.PreferLeftAsHJProbe) > 0) if forceLeftToBuild && forceRightToBuild { - p.SCtx().GetSessionVars().StmtCtx.SetHintWarning("Some HASH_JOIN_BUILD and HASH_JOIN_PROBE hints are conflicts, please check the hints") + p.SCtx().GetSessionVars().StmtCtx.SetHintWarning("Conflicting HASH_JOIN_BUILD and HASH_JOIN_PROBE hints detected. " + + "Both sides cannot be specified to use the same table. Please review the hints") forceLeftToBuild = false forceRightToBuild = false } @@ -450,8 +451,10 @@ func getHashJoins(p *logicalop.LogicalJoin, prop *property.PhysicalProperty) (jo } else { joins = append(joins, getHashJoin(p, prop, 1, false)) if forceLeftToBuild || forceRightToBuild { - // Do not support specifying the build and probe side for semi join. - p.SCtx().GetSessionVars().StmtCtx.SetHintWarning(fmt.Sprintf("We can't use the HASH_JOIN_BUILD or HASH_JOIN_PROBE hint for %s, please check the hint", p.JoinType)) + p.SCtx().GetSessionVars().StmtCtx.SetHintWarning(fmt.Sprintf( + "The HASH_JOIN_BUILD and HASH_JOIN_PROBE hints are not supported for %s with hash join version 1. "+ + "Please remove these hints", + p.JoinType)) forceLeftToBuild = false forceRightToBuild = false } @@ -459,8 +462,10 @@ func getHashJoins(p *logicalop.LogicalJoin, prop *property.PhysicalProperty) (jo case logicalop.LeftOuterSemiJoin, logicalop.AntiLeftOuterSemiJoin: joins = append(joins, getHashJoin(p, prop, 1, false)) if forceLeftToBuild || forceRightToBuild { - // Do not support specifying the build and probe side for semi join. - p.SCtx().GetSessionVars().StmtCtx.SetHintWarning(fmt.Sprintf("We can't use the HASH_JOIN_BUILD or HASH_JOIN_PROBE hint for %s, please check the hint", p.JoinType)) + p.SCtx().GetSessionVars().StmtCtx.SetHintWarning(fmt.Sprintf( + "HASH_JOIN_BUILD and HASH_JOIN_PROBE hints are not supported for %s because the build side is fixed. "+ + "Please remove these hints", + p.JoinType)) forceLeftToBuild = false forceRightToBuild = false } diff --git a/tests/integrationtest/r/planner/core/casetest/physicalplantest/physical_plan.result b/tests/integrationtest/r/planner/core/casetest/physicalplantest/physical_plan.result index 4438e56437820..084ebac7c5171 100644 --- a/tests/integrationtest/r/planner/core/casetest/physicalplantest/physical_plan.result +++ b/tests/integrationtest/r/planner/core/casetest/physicalplantest/physical_plan.result @@ -1771,8 +1771,8 @@ sum(t1.a in (select a from t2)) 2 show warnings; Level Code Message -Warning 1815 We can't use the HASH_JOIN_BUILD or HASH_JOIN_PROBE hint for left outer semi join, please check the hint -Warning 1815 We can't use the HASH_JOIN_BUILD or HASH_JOIN_PROBE hint for left outer semi join, please check the hint +Warning 1815 HASH_JOIN_BUILD and HASH_JOIN_PROBE hints are not supported for left outer semi join because the build side is fixed. Please remove these hints +Warning 1815 HASH_JOIN_BUILD and HASH_JOIN_PROBE hints are not supported for left outer semi join because the build side is fixed. Please remove these hints explain format = 'brief' select /*+ hash_join_probe(t1) */ sum(t1.a in (select a from t2)) from t1; id estRows task access object operator info HashAgg 1.00 root funcs:sum(Column#9)->Column#8 @@ -1787,8 +1787,8 @@ sum(t1.a in (select a from t2)) 2 show warnings; Level Code Message -Warning 1815 We can't use the HASH_JOIN_BUILD or HASH_JOIN_PROBE hint for left outer semi join, please check the hint -Warning 1815 We can't use the HASH_JOIN_BUILD or HASH_JOIN_PROBE hint for left outer semi join, please check the hint +Warning 1815 HASH_JOIN_BUILD and HASH_JOIN_PROBE hints are not supported for left outer semi join because the build side is fixed. Please remove these hints +Warning 1815 HASH_JOIN_BUILD and HASH_JOIN_PROBE hints are not supported for left outer semi join because the build side is fixed. Please remove these hints explain format = 'brief' select /*+ hash_join_build(t2@sel_2) */ sum(t1.a in (select a from t2)) from t1; id estRows task access object operator info HashAgg 1.00 root funcs:sum(Column#9)->Column#8 @@ -1803,8 +1803,8 @@ sum(t1.a in (select a from t2)) 2 show warnings; Level Code Message -Warning 1815 We can't use the HASH_JOIN_BUILD or HASH_JOIN_PROBE hint for left outer semi join, please check the hint -Warning 1815 We can't use the HASH_JOIN_BUILD or HASH_JOIN_PROBE hint for left outer semi join, please check the hint +Warning 1815 HASH_JOIN_BUILD and HASH_JOIN_PROBE hints are not supported for left outer semi join because the build side is fixed. Please remove these hints +Warning 1815 HASH_JOIN_BUILD and HASH_JOIN_PROBE hints are not supported for left outer semi join because the build side is fixed. Please remove these hints explain format = 'brief' select /*+ hash_join_probe(t2@sel_2) */ sum(t1.a in (select a from t2)) from t1; id estRows task access object operator info HashAgg 1.00 root funcs:sum(Column#9)->Column#8 @@ -1819,8 +1819,8 @@ sum(t1.a in (select a from t2)) 2 show warnings; Level Code Message -Warning 1815 We can't use the HASH_JOIN_BUILD or HASH_JOIN_PROBE hint for left outer semi join, please check the hint -Warning 1815 We can't use the HASH_JOIN_BUILD or HASH_JOIN_PROBE hint for left outer semi join, please check the hint +Warning 1815 HASH_JOIN_BUILD and HASH_JOIN_PROBE hints are not supported for left outer semi join because the build side is fixed. Please remove these hints +Warning 1815 HASH_JOIN_BUILD and HASH_JOIN_PROBE hints are not supported for left outer semi join because the build side is fixed. Please remove these hints explain format = 'brief' select /*+ hash_join_build(t1) */ sum(t1.a not in (select a from t2)) from t1; id estRows task access object operator info HashAgg 1.00 root funcs:sum(Column#9)->Column#8 @@ -1835,8 +1835,8 @@ sum(t1.a not in (select a from t2)) 0 show warnings; Level Code Message -Warning 1815 We can't use the HASH_JOIN_BUILD or HASH_JOIN_PROBE hint for anti left outer semi join, please check the hint -Warning 1815 We can't use the HASH_JOIN_BUILD or HASH_JOIN_PROBE hint for anti left outer semi join, please check the hint +Warning 1815 HASH_JOIN_BUILD and HASH_JOIN_PROBE hints are not supported for anti left outer semi join because the build side is fixed. Please remove these hints +Warning 1815 HASH_JOIN_BUILD and HASH_JOIN_PROBE hints are not supported for anti left outer semi join because the build side is fixed. Please remove these hints explain format = 'brief' select /*+ hash_join_probe(t1) */ sum(t1.a not in (select a from t2)) from t1; id estRows task access object operator info HashAgg 1.00 root funcs:sum(Column#9)->Column#8 @@ -1851,8 +1851,8 @@ sum(t1.a not in (select a from t2)) 0 show warnings; Level Code Message -Warning 1815 We can't use the HASH_JOIN_BUILD or HASH_JOIN_PROBE hint for anti left outer semi join, please check the hint -Warning 1815 We can't use the HASH_JOIN_BUILD or HASH_JOIN_PROBE hint for anti left outer semi join, please check the hint +Warning 1815 HASH_JOIN_BUILD and HASH_JOIN_PROBE hints are not supported for anti left outer semi join because the build side is fixed. Please remove these hints +Warning 1815 HASH_JOIN_BUILD and HASH_JOIN_PROBE hints are not supported for anti left outer semi join because the build side is fixed. Please remove these hints explain format = 'brief' select /*+ hash_join_build(t2@sel_2) */ sum(t1.a not in (select a from t2)) from t1; id estRows task access object operator info HashAgg 1.00 root funcs:sum(Column#9)->Column#8 @@ -1867,8 +1867,8 @@ sum(t1.a not in (select a from t2)) 0 show warnings; Level Code Message -Warning 1815 We can't use the HASH_JOIN_BUILD or HASH_JOIN_PROBE hint for anti left outer semi join, please check the hint -Warning 1815 We can't use the HASH_JOIN_BUILD or HASH_JOIN_PROBE hint for anti left outer semi join, please check the hint +Warning 1815 HASH_JOIN_BUILD and HASH_JOIN_PROBE hints are not supported for anti left outer semi join because the build side is fixed. Please remove these hints +Warning 1815 HASH_JOIN_BUILD and HASH_JOIN_PROBE hints are not supported for anti left outer semi join because the build side is fixed. Please remove these hints explain format = 'brief' select /*+ hash_join_probe(t2@sel_2) */ sum(t1.a not in (select a from t2)) from t1; id estRows task access object operator info HashAgg 1.00 root funcs:sum(Column#9)->Column#8 @@ -1883,8 +1883,8 @@ sum(t1.a not in (select a from t2)) 0 show warnings; Level Code Message -Warning 1815 We can't use the HASH_JOIN_BUILD or HASH_JOIN_PROBE hint for anti left outer semi join, please check the hint -Warning 1815 We can't use the HASH_JOIN_BUILD or HASH_JOIN_PROBE hint for anti left outer semi join, please check the hint +Warning 1815 HASH_JOIN_BUILD and HASH_JOIN_PROBE hints are not supported for anti left outer semi join because the build side is fixed. Please remove these hints +Warning 1815 HASH_JOIN_BUILD and HASH_JOIN_PROBE hints are not supported for anti left outer semi join because the build side is fixed. Please remove these hints explain format = 'brief' select /*+ hash_join_probe(t2, t2) */ t1.a, t2.a from t1 join t2 on t1.a=t2.a and t1.b=t2.b; id estRows task access object operator info HashJoin 12500.00 root inner join, equal:[eq(planner__core__casetest__physicalplantest__physical_plan.t1.a, planner__core__casetest__physicalplantest__physical_plan.t2.a) eq(planner__core__casetest__physicalplantest__physical_plan.t1.b, planner__core__casetest__physicalplantest__physical_plan.t2.b)] From d73e58449eba168ee20a2d540a0054f59bdec94f Mon Sep 17 00:00:00 2001 From: YangKeao Date: Wed, 8 Jan 2025 17:06:44 +0800 Subject: [PATCH 09/27] ttl, test: scale TTL workers during the fault tests (#58750) close pingcap/tidb#58745 --- .../ttlworker/job_manager_integration_test.go | 39 +++++++++++++++++-- 1 file changed, 36 insertions(+), 3 deletions(-) diff --git a/pkg/ttl/ttlworker/job_manager_integration_test.go b/pkg/ttl/ttlworker/job_manager_integration_test.go index efe12c646ec2c..0cb045bc1098a 100644 --- a/pkg/ttl/ttlworker/job_manager_integration_test.go +++ b/pkg/ttl/ttlworker/job_manager_integration_test.go @@ -1727,8 +1727,6 @@ func TestJobManagerWithFault(t *testing.T) { } stopTestCh := make(chan struct{}) - wg := &sync.WaitGroup{} - wg.Add(1) fault := newFaultWithFilter(func(sql string) bool { // skip some local only sql, ref `getSession()` in `session.go` @@ -1739,6 +1737,10 @@ func TestJobManagerWithFault(t *testing.T) { return true }, newFaultWithProbability(faultPercent)) + + wg := &sync.WaitGroup{} + // start the goroutine to inject fault to managers randomly + wg.Add(1) go func() { defer wg.Done() @@ -1775,6 +1777,38 @@ func TestJobManagerWithFault(t *testing.T) { } }() + // start the goroutine to randomly scale the worker count + wg.Add(1) + go func() { + defer wg.Done() + + maxScanWorkerCount := variable.DefTiDBTTLScanWorkerCount * 2 + minScanWorkerCount := variable.DefTiDBTTLScanWorkerCount / 2 + + maxDelWorkerCount := variable.DefTiDBTTLDeleteWorkerCount * 2 + minDelWorkerCount := variable.DefTiDBTTLDeleteWorkerCount / 2 + faultTicker := time.NewTicker(time.Second) + + tk := testkit.NewTestKit(t, store) + for { + 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) + + return + case <-faultTicker.C: + scanWorkerCount := rand.Int()%(maxScanWorkerCount-minScanWorkerCount) + minScanWorkerCount + delWorkerCount := rand.Int()%(maxDelWorkerCount-minDelWorkerCount) + minDelWorkerCount + + logutil.BgLogger().Info("scale worker count", zap.Int("scanWorkerCount", scanWorkerCount), zap.Int("delWorkerCount", delWorkerCount)) + tk.MustExec("set @@global.tidb_ttl_scan_worker_count = ?", scanWorkerCount) + tk.MustExec("set @@global.tidb_ttl_delete_worker_count = ?", delWorkerCount) + } + } + }() + // run the workload goroutine testStart := time.Now() for time.Since(testStart) < testDuration { @@ -1825,7 +1859,6 @@ func TestJobManagerWithFault(t *testing.T) { } logutil.BgLogger().Info("test finished") - stopTestCh <- struct{}{} close(stopTestCh) wg.Wait() From 510d0037b18f258f505abc6cf13a8128563e9359 Mon Sep 17 00:00:00 2001 From: okJiang <819421878@qq.com> Date: Wed, 8 Jan 2025 18:13:55 +0800 Subject: [PATCH 10/27] *: upgrade pd client to make sure tso client initiate successfully (#58752) close pingcap/tidb#55248, close pingcap/tidb#58239 --- DEPS.bzl | 12 ++++++------ go.mod | 2 +- go.sum | 4 ++-- 3 files changed, 9 insertions(+), 9 deletions(-) diff --git a/DEPS.bzl b/DEPS.bzl index bf4ee0e36d695..c0e6c310b416b 100644 --- a/DEPS.bzl +++ b/DEPS.bzl @@ -7050,13 +7050,13 @@ def go_deps(): name = "com_github_tikv_pd_client", build_file_proto_mode = "disable_global", importpath = "github.com/tikv/pd/client", - sha256 = "e46a55f684df2acf06f1081a8a32750ef99bc05e9a1d6508f5c8e605bf954642", - strip_prefix = "github.com/tikv/pd/client@v0.0.0-20241220053006-461b86adc78d", + sha256 = "a704b691889a87001a40bbcf40b69df8b0d801f063bf07bc15542088df15682a", + strip_prefix = "github.com/tikv/pd/client@v0.0.0-20250107032658-5c4ab57d68de", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/github.com/tikv/pd/client/com_github_tikv_pd_client-v0.0.0-20241220053006-461b86adc78d.zip", - "http://ats.apps.svc/gomod/github.com/tikv/pd/client/com_github_tikv_pd_client-v0.0.0-20241220053006-461b86adc78d.zip", - "https://cache.hawkingrei.com/gomod/github.com/tikv/pd/client/com_github_tikv_pd_client-v0.0.0-20241220053006-461b86adc78d.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/github.com/tikv/pd/client/com_github_tikv_pd_client-v0.0.0-20241220053006-461b86adc78d.zip", + "http://bazel-cache.pingcap.net:8080/gomod/github.com/tikv/pd/client/com_github_tikv_pd_client-v0.0.0-20250107032658-5c4ab57d68de.zip", + "http://ats.apps.svc/gomod/github.com/tikv/pd/client/com_github_tikv_pd_client-v0.0.0-20250107032658-5c4ab57d68de.zip", + "https://cache.hawkingrei.com/gomod/github.com/tikv/pd/client/com_github_tikv_pd_client-v0.0.0-20250107032658-5c4ab57d68de.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/github.com/tikv/pd/client/com_github_tikv_pd_client-v0.0.0-20250107032658-5c4ab57d68de.zip", ], ) go_repository( diff --git a/go.mod b/go.mod index 7e9a0ac1a907d..703c543cc7c01 100644 --- a/go.mod +++ b/go.mod @@ -110,7 +110,7 @@ require ( github.com/tdakkota/asciicheck v0.3.0 github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2 github.com/tikv/client-go/v2 v2.0.8-0.20241225040645-f2266d6bf259 - github.com/tikv/pd/client v0.0.0-20241220053006-461b86adc78d + github.com/tikv/pd/client v0.0.0-20250107032658-5c4ab57d68de github.com/timakin/bodyclose v0.0.0-20241017074812-ed6a65f985e3 github.com/twmb/murmur3 v1.1.6 github.com/uber/jaeger-client-go v2.22.1+incompatible diff --git a/go.sum b/go.sum index 0e36eb7a11528..b4c5f48cd5720 100644 --- a/go.sum +++ b/go.sum @@ -832,8 +832,8 @@ github.com/tiancaiamao/gp v0.0.0-20221230034425-4025bc8a4d4a h1:J/YdBZ46WKpXsxsW github.com/tiancaiamao/gp v0.0.0-20221230034425-4025bc8a4d4a/go.mod h1:h4xBhSNtOeEosLJ4P7JyKXX7Cabg7AVkWCK5gV2vOrM= github.com/tikv/client-go/v2 v2.0.8-0.20241225040645-f2266d6bf259 h1:L+IptTYGAYldntJTyU/VIEcomRoeVDTHZnA9HMDrGF4= github.com/tikv/client-go/v2 v2.0.8-0.20241225040645-f2266d6bf259/go.mod h1:H+EqIN/3fOXhVgzZT91Bk1XF1kfvlBZCYcpMIC8FNbg= -github.com/tikv/pd/client v0.0.0-20241220053006-461b86adc78d h1:V/LfOrlfS4Ct+AHvjbxVXAZZgqjcpuc53m0wxk0xYF4= -github.com/tikv/pd/client v0.0.0-20241220053006-461b86adc78d/go.mod h1:Dsiy6gxwRqsqgJPfDSxDbdG+qFvk0cNzvHax0TOPj2Q= +github.com/tikv/pd/client v0.0.0-20250107032658-5c4ab57d68de h1:fefo6U56UMca1CfwY/FVhfVcinyDgXgffQQp9lfMeLg= +github.com/tikv/pd/client v0.0.0-20250107032658-5c4ab57d68de/go.mod h1:6xcCIcECSowarZ9+AqdURngHzS+t5w2x3qhwg1moj4o= github.com/timakin/bodyclose v0.0.0-20241017074812-ed6a65f985e3 h1:y4mJRFlM6fUyPhoXuFg/Yu02fg/nIPFMOY8tOqppoFg= github.com/timakin/bodyclose v0.0.0-20241017074812-ed6a65f985e3/go.mod h1:mkjARE7Yr8qU23YcGMSALbIxTQ9r9QBVahQOBRfU460= github.com/tklauser/go-sysconf v0.3.9/go.mod h1:11DU/5sG7UexIrp/O6g35hrWzu0JxlwQ3LSFUzyeuhs= From 2a3542cc2f4c7efdf1e045c423179f43e9824ded Mon Sep 17 00:00:00 2001 From: Zequan <110292382+EricZequan@users.noreply.github.com> Date: Wed, 8 Jan 2025 21:00:46 +0800 Subject: [PATCH 11/27] planner: fix an unexpected result of the inlineprojection of TopN (#58771) ref pingcap/tidb#58743 --- .../core/operator/logicalop/logical_top_n.go | 12 ++- .../logicalop/logicalop_test/BUILD.bazel | 3 +- .../logicalop_test/plan_execute_test.go | 84 +++++++++++++++++++ 3 files changed, 97 insertions(+), 2 deletions(-) create mode 100644 pkg/planner/core/operator/logicalop/logicalop_test/plan_execute_test.go diff --git a/pkg/planner/core/operator/logicalop/logical_top_n.go b/pkg/planner/core/operator/logicalop/logical_top_n.go index 303b40f363dcd..18af3912e8354 100644 --- a/pkg/planner/core/operator/logicalop/logical_top_n.go +++ b/pkg/planner/core/operator/logicalop/logical_top_n.go @@ -83,7 +83,8 @@ func (lt *LogicalTopN) PruneColumns(parentUsedCols []*expression.Column, opt *op child := lt.Children()[0] var cols []*expression.Column - lt.InlineProjection(parentUsedCols, opt) + snapParentUsedCols := make([]*expression.Column, 0, len(parentUsedCols)) + snapParentUsedCols = append(snapParentUsedCols, parentUsedCols...) lt.ByItems, cols = pruneByItems(lt, lt.ByItems, opt) parentUsedCols = append(parentUsedCols, cols...) @@ -92,6 +93,15 @@ func (lt *LogicalTopN) PruneColumns(parentUsedCols []*expression.Column, opt *op if err != nil { return nil, err } + // If the length of parentUsedCols is 0, it means that the parent plan does not need this plan to output related + // results, such as: select count(*) from t + // So we set the schema of topN to 0. After inlineprojection, the schema of topN will be set to the shortest column + // in its child plan, and this column will not be used later. + if len(snapParentUsedCols) == 0 { + lt.SetSchema(nil) + } + lt.InlineProjection(snapParentUsedCols, opt) + return lt, nil } diff --git a/pkg/planner/core/operator/logicalop/logicalop_test/BUILD.bazel b/pkg/planner/core/operator/logicalop/logicalop_test/BUILD.bazel index 8b24e7c700c8d..d085cef75ebb1 100644 --- a/pkg/planner/core/operator/logicalop/logicalop_test/BUILD.bazel +++ b/pkg/planner/core/operator/logicalop/logicalop_test/BUILD.bazel @@ -6,9 +6,10 @@ go_test( srcs = [ "hash64_equals_test.go", "logical_mem_table_predicate_extractor_test.go", + "plan_execute_test.go", ], flaky = True, - shard_count = 33, + shard_count = 34, deps = [ "//pkg/domain", "//pkg/expression", diff --git a/pkg/planner/core/operator/logicalop/logicalop_test/plan_execute_test.go b/pkg/planner/core/operator/logicalop/logicalop_test/plan_execute_test.go new file mode 100644 index 0000000000000..e17994dd1b5b6 --- /dev/null +++ b/pkg/planner/core/operator/logicalop/logicalop_test/plan_execute_test.go @@ -0,0 +1,84 @@ +// Copyright 2019 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 logicalop_test + +import ( + "testing" + + "github.com/pingcap/tidb/pkg/testkit" +) + +func TestIssue58743(t *testing.T) { + store := testkit.CreateMockStore(t) + + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec(` + CREATE TABLE tlf5d55361 ( + col_9 time NOT NULL, + col_10 float NOT NULL, + col_11 json NOT NULL, + col_12 date NOT NULL, + col_13 json NOT NULL, + col_14 tinyint NOT NULL, + col_15 date DEFAULT NULL, + col_16 tinyblob NOT NULL, + col_17 time DEFAULT '03:51:26', + PRIMARY KEY (col_14, col_9, col_10) /*T![clustered_index] CLUSTERED */ + ) ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_general_ci + PARTITION BY RANGE (col_14) + ( + PARTITION p0 VALUES LESS THAN (-128), + PARTITION p1 VALUES LESS THAN (-84), + PARTITION p2 VALUES LESS THAN (-41), + PARTITION p3 VALUES LESS THAN (-30) + ); + `) + tk.MustExec(` + CREATE TABLE td8d55878 ( + col_26 datetime DEFAULT NULL, + col_27 time DEFAULT NULL, + col_28 json DEFAULT NULL, + col_29 char(186) CHARACTER SET utf8 COLLATE utf8_unicode_ci DEFAULT '4-BJKi', + col_30 date NOT NULL DEFAULT '1998-07-28', + col_31 datetime NOT NULL + ) ENGINE=InnoDB DEFAULT CHARSET=gbk COLLATE=gbk_chinese_ci; + `) + + tk.MustQuery(` + WITH cte_3 (col_128) AS ( + SELECT /*+ USE_INDEX_MERGE(td8d55878 tlf5d55361)*/ + MIN(td8d55878.col_26) AS r0 + FROM tlf5d55361 + JOIN td8d55878 + GROUP BY tlf5d55361.col_17 + HAVING tlf5d55361.col_17 <= '20:22:14.00' OR tlf5d55361.col_17 BETWEEN '21:56:23.00' AND '19:42:43.00' + ORDER BY r0 + LIMIT 772780933 + ), + cte_4 (col_129) AS ( + SELECT /*+ HASH_AGG()*/ + SUM(tlf5d55361.col_14) AS r0 + FROM td8d55878 + JOIN tlf5d55361 ON tlf5d55361.col_17 = td8d55878.col_27 + GROUP BY td8d55878.col_30 + HAVING ISNULL(td8d55878.col_30) OR td8d55878.col_30 BETWEEN '2009-09-08' AND '1980-11-17' + ) + SELECT SUM((cte_4.col_129 IN (0.65, 564617.3335, 45, 0.319, 0.4427)) IS TRUE) + FROM cte_4 + JOIN cte_3; + `).Equal(nil) +} From 8390fc4557d0b903f9befd9fb9cc7936cf64379f Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Wed, 8 Jan 2025 21:59:27 +0800 Subject: [PATCH 12/27] planner: check binding validation when creating bindings (#58760) ref pingcap/tidb#51347 --- pkg/bindinfo/BUILD.bazel | 1 + pkg/bindinfo/binding.go | 70 ++++++++++++++++++- pkg/bindinfo/tests/BUILD.bazel | 2 +- pkg/bindinfo/tests/bind_test.go | 34 ++++++++- .../sessionstates/session_states_test.go | 12 +--- 5 files changed, 106 insertions(+), 13 deletions(-) diff --git a/pkg/bindinfo/BUILD.bazel b/pkg/bindinfo/BUILD.bazel index eeeecc8ce5b9f..504cc88a10250 100644 --- a/pkg/bindinfo/BUILD.bazel +++ b/pkg/bindinfo/BUILD.bazel @@ -23,6 +23,7 @@ go_library( "//pkg/sessionctx/sessionstates", "//pkg/sessionctx/variable", "//pkg/types", + "//pkg/types/parser_driver", "//pkg/util", "//pkg/util/chunk", "//pkg/util/hack", diff --git a/pkg/bindinfo/binding.go b/pkg/bindinfo/binding.go index a2ced548d3c56..f09b17fbbff0b 100644 --- a/pkg/bindinfo/binding.go +++ b/pkg/bindinfo/binding.go @@ -15,6 +15,8 @@ package bindinfo import ( + "context" + "fmt" "strings" "sync" "unsafe" @@ -24,9 +26,11 @@ import ( "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/types" + driver "github.com/pingcap/tidb/pkg/types/parser_driver" "github.com/pingcap/tidb/pkg/util/hint" utilparser "github.com/pingcap/tidb/pkg/util/parser" "github.com/pkg/errors" + "go.uber.org/zap" ) const ( @@ -264,7 +268,7 @@ func (*tableNameCollector) Leave(in ast.Node) (out ast.Node, ok bool) { // prepareHints builds ID and Hint for Bindings. If sctx is not nil, we check if // the BindSQL is still valid. -func prepareHints(_ sessionctx.Context, binding *Binding) (rerr error) { +func prepareHints(sctx sessionctx.Context, binding *Binding) (rerr error) { defer func() { if r := recover(); r != nil { rerr = errors.Errorf("panic when preparing hints for binding %v, panic: %v", binding.BindSQL, r) @@ -286,10 +290,16 @@ func prepareHints(_ sessionctx.Context, binding *Binding) (rerr error) { dbName = "*" // ues '*' for universal bindings } - hintsSet, _, warns, err := hint.ParseHintsSet(p, binding.BindSQL, binding.Charset, binding.Collation, dbName) + hintsSet, stmt, warns, err := hint.ParseHintsSet(p, binding.BindSQL, binding.Charset, binding.Collation, dbName) if err != nil { return err } + if !isCrossDB && !hasParam(stmt) { + // TODO: how to check cross-db binding and bindings with parameters? + if err = checkBindingValidation(sctx, binding.BindSQL); err != nil { + return err + } + } hintsStr, err := hintsSet.Restore() if err != nil { return err @@ -460,3 +470,59 @@ func eraseLastSemicolon(stmt ast.StmtNode) { stmt.SetText(nil, sql[:len(sql)-1]) } } + +type paramChecker struct { + hasParam bool +} + +func (e *paramChecker) Enter(in ast.Node) (ast.Node, bool) { + if _, ok := in.(*driver.ParamMarkerExpr); ok { + e.hasParam = true + return in, true + } + return in, false +} + +func (*paramChecker) Leave(in ast.Node) (ast.Node, bool) { + return in, true +} + +// hasParam checks whether the statement contains any parameters. +// For example, `create binding using select * from t where a=?` contains a parameter '?'. +func hasParam(stmt ast.Node) bool { + p := new(paramChecker) + stmt.Accept(p) + return p.hasParam +} + +// CheckBindingStmt checks whether the statement is valid. +func checkBindingValidation(sctx sessionctx.Context, bindingSQL string) error { + origVals := sctx.GetSessionVars().UsePlanBaselines + sctx.GetSessionVars().UsePlanBaselines = false + + // Usually passing a sprintf to ExecuteInternal is not recommended, but in this case + // it is safe because ExecuteInternal does not permit MultiStatement execution. Thus, + // the statement won't be able to "break out" from EXPLAIN. + rs, err := exec(sctx, fmt.Sprintf("EXPLAIN FORMAT='hint' %s", bindingSQL)) + sctx.GetSessionVars().UsePlanBaselines = origVals + if rs != nil { + defer func() { + // Audit log is collected in Close(), set InRestrictedSQL to avoid 'create sql binding' been recorded as 'explain'. + origin := sctx.GetSessionVars().InRestrictedSQL + sctx.GetSessionVars().InRestrictedSQL = true + if rerr := rs.Close(); rerr != nil { + bindingLogger().Error("close result set failed", zap.Error(rerr), zap.String("binding_sql", bindingSQL)) + } + sctx.GetSessionVars().InRestrictedSQL = origin + }() + } + if err != nil { + return err + } + chk := rs.NewChunk(nil) + err = rs.Next(context.TODO(), chk) + if err != nil { + return err + } + return nil +} diff --git a/pkg/bindinfo/tests/BUILD.bazel b/pkg/bindinfo/tests/BUILD.bazel index 8db1e1252f208..a3d61ded64e3a 100644 --- a/pkg/bindinfo/tests/BUILD.bazel +++ b/pkg/bindinfo/tests/BUILD.bazel @@ -10,7 +10,7 @@ go_test( ], flaky = True, race = "on", - shard_count = 21, + shard_count = 22, deps = [ "//pkg/bindinfo", "//pkg/domain", diff --git a/pkg/bindinfo/tests/bind_test.go b/pkg/bindinfo/tests/bind_test.go index 98c4a11a4cbce..76a3e8c33a62a 100644 --- a/pkg/bindinfo/tests/bind_test.go +++ b/pkg/bindinfo/tests/bind_test.go @@ -348,9 +348,10 @@ func TestInvisibleIndex(t *testing.T) { tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int, b int, unique idx_a(a), index idx_b(b) invisible)") + tk.MustContainErrMsg("create global binding for select * from t using select * from t use index(idx_b)", + "[planner:1176]Key 'idx_b' doesn't exist in table 't'") // Create bind using index - tk.MustExec("create global binding for select * from t using select * from t use index(idx_b)") tk.MustExec("create global binding for select * from t using select * from t use index(idx_a)") tk.MustQuery("select * from t") @@ -851,3 +852,34 @@ func TestBatchDropBindings(t *testing.T) { removeAllBindings(tk, true) removeAllBindings(tk, false) } + +func TestInvalidBindingCheck(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec(`use test`) + tk.MustExec(`create table t (a int, b int)`) + + cases := []struct { + SQL string + Err string + }{ + {"select * from t where c=1", "[planner:1054]Unknown column 'c' in 'where clause'"}, + {"select * from t where a=1 and c=1", "[planner:1054]Unknown column 'c' in 'where clause'"}, + {"select * from dbx.t", "[schema:1146]Table 'dbx.t' doesn't exist"}, + {"select * from t1", "[schema:1146]Table 'test.t1' doesn't exist"}, + {"select * from t1, t", "[schema:1146]Table 'test.t1' doesn't exist"}, + {"select * from t use index(c)", "[planner:1176]Key 'c' doesn't exist in table 't'"}, + } + + for _, c := range cases { + for _, scope := range []string{"session", "global"} { + sql := fmt.Sprintf("create %v binding using %v", scope, c.SQL) + tk.MustGetErrMsg(sql, c.Err) + } + } + + // cross-db bindings or bindings with parameters can bypass the check, which is expected. + // We'll optimize this check further in the future. + tk.MustExec("create binding using select * from *.t where c=1") + tk.MustExec("create binding using select * from t where c=?") +} diff --git a/pkg/sessionctx/sessionstates/session_states_test.go b/pkg/sessionctx/sessionstates/session_states_test.go index 474293a48ccf0..2caea78e5575b 100644 --- a/pkg/sessionctx/sessionstates/session_states_test.go +++ b/pkg/sessionctx/sessionstates/session_states_test.go @@ -1293,9 +1293,7 @@ func TestSQLBinding(t *testing.T) { tk.MustExec("drop table test.t1") return nil }, - checkFunc: func(tk *testkit.TestKit, param any) { - require.Equal(t, 1, len(tk.MustQuery("show session bindings").Rows())) - }, + restoreErr: errno.ErrNoSuchTable, cleanFunc: func(tk *testkit.TestKit) { tk.MustExec("create table test.t1(id int primary key, name varchar(10), key(name))") }, @@ -1310,9 +1308,7 @@ func TestSQLBinding(t *testing.T) { tk.MustExec("drop database test1") return nil }, - checkFunc: func(tk *testkit.TestKit, param any) { - require.Equal(t, 1, len(tk.MustQuery("show session bindings").Rows())) - }, + restoreErr: errno.ErrNoSuchTable, }, { // alter the table @@ -1321,9 +1317,7 @@ func TestSQLBinding(t *testing.T) { tk.MustExec("alter table test.t1 drop index name") return nil }, - checkFunc: func(tk *testkit.TestKit, param any) { - require.Equal(t, 1, len(tk.MustQuery("show session bindings").Rows())) - }, + restoreErr: errno.ErrKeyDoesNotExist, cleanFunc: func(tk *testkit.TestKit) { tk.MustExec("alter table test.t1 add index name(name)") }, From 2ae0b0abfbe341520437275a4f871bc16971fdc8 Mon Sep 17 00:00:00 2001 From: Will DeVries <1624341+wddevries@users.noreply.github.com> Date: Wed, 8 Jan 2025 19:10:58 -0800 Subject: [PATCH 13/27] workloadrepo: Add unit testing for the Workload Repository. (#58266) ref pingcap/tidb#58247 --- pkg/util/workloadrepo/BUILD.bazel | 4 +- pkg/util/workloadrepo/housekeeper.go | 136 +++--- pkg/util/workloadrepo/sampling.go | 16 +- pkg/util/workloadrepo/snapshot.go | 57 ++- pkg/util/workloadrepo/table.go | 44 +- pkg/util/workloadrepo/utils.go | 72 ++- pkg/util/workloadrepo/worker.go | 65 +-- pkg/util/workloadrepo/worker_test.go | 682 ++++++++++++++++++++++++++- 8 files changed, 909 insertions(+), 167 deletions(-) diff --git a/pkg/util/workloadrepo/BUILD.bazel b/pkg/util/workloadrepo/BUILD.bazel index 2edc49364790d..9b910c112a453 100644 --- a/pkg/util/workloadrepo/BUILD.bazel +++ b/pkg/util/workloadrepo/BUILD.bazel @@ -41,15 +41,17 @@ go_library( go_test( name = "workloadrepo_test", - timeout = "short", + timeout = "long", srcs = ["worker_test.go"], embed = [":workloadrepo"], flaky = True, deps = [ "//pkg/domain", + "//pkg/infoschema", "//pkg/kv", "//pkg/owner", "//pkg/parser/ast", + "//pkg/sessionctx", "//pkg/testkit", "@com_github_stretchr_testify//require", "@io_etcd_go_etcd_client_v3//:client", diff --git a/pkg/util/workloadrepo/housekeeper.go b/pkg/util/workloadrepo/housekeeper.go index a6be07c835b93..829c2cea7c221 100644 --- a/pkg/util/workloadrepo/housekeeper.go +++ b/pkg/util/workloadrepo/housekeeper.go @@ -16,6 +16,7 @@ package workloadrepo import ( "context" + "errors" "fmt" "strings" "time" @@ -38,79 +39,102 @@ func calcNextTick(now time.Time) time.Duration { return next.Sub(now) } -func createAllPartitions(ctx context.Context, sess sessionctx.Context, is infoschema.InfoSchema) error { +func createPartition(ctx context.Context, is infoschema.InfoSchema, tbl *repositoryTable, sess sessionctx.Context, now time.Time) error { + tbSchema, err := is.TableByName(ctx, workloadSchemaCIStr, ast.NewCIStr(tbl.destTable)) + if err != nil { + logutil.BgLogger().Info("workload repository cannot get table", zap.String("tbl", tbl.destTable), zap.NamedError("err", err)) + return err + } + tbInfo := tbSchema.Meta() + sb := &strings.Builder{} - for _, tbl := range workloadTables { - tbSchema, err := is.TableByName(ctx, workloadSchemaCIStr, ast.NewCIStr(tbl.destTable)) + sqlescape.MustFormatSQL(sb, "ALTER TABLE %n.%n ADD PARTITION (", WorkloadSchema, tbl.destTable) + skip, err := generatePartitionRanges(sb, tbInfo, now) + if err != nil { + return err + } + if !skip { + fmt.Fprintf(sb, ")") + _, err = execRetry(ctx, sess, sb.String()) if err != nil { - logutil.BgLogger().Info("workload repository cannot get table", zap.String("tbl", tbl.destTable), zap.NamedError("err", err)) + logutil.BgLogger().Info("workload repository cannot add partitions", zap.String("parts", sb.String()), zap.NamedError("err", err)) return err } - tbInfo := tbSchema.Meta() - - sb.Reset() - sqlescape.MustFormatSQL(sb, "ALTER TABLE %n.%n ADD PARTITION (", WorkloadSchema, tbl.destTable) - if !generatePartitionRanges(sb, tbInfo) { - fmt.Fprintf(sb, ")") - _, err = execRetry(ctx, sess, sb.String()) - if err != nil { - logutil.BgLogger().Info("workload repository cannot add partitions", zap.String("parts", sb.String()), zap.NamedError("err", err)) - return err - } + } + return nil +} + +func (w *worker) createAllPartitions(ctx context.Context, sess sessionctx.Context, is infoschema.InfoSchema, now time.Time) error { + for _, tbl := range w.workloadTables { + if err := createPartition(ctx, is, &tbl, sess, now); err != nil { + return err } } return nil } -func (w *worker) dropOldPartitions(ctx context.Context, sess sessionctx.Context, is infoschema.InfoSchema, now time.Time) error { - w.Lock() - retention := int(w.retentionDays) - w.Unlock() +func dropOldPartition(ctx context.Context, is infoschema.InfoSchema, + tbl *repositoryTable, now time.Time, retention int, sess sessionctx.Context) error { + tbSchema, err := is.TableByName(ctx, workloadSchemaCIStr, ast.NewCIStr(tbl.destTable)) + if err != nil { + return fmt.Errorf("workload repository could not find table `%s`: %v", tbl.destTable, err) + } + tbInfo := tbSchema.Meta() + if tbInfo == nil { + return fmt.Errorf("workload repository could not load information for '%s'", tbl.destTable) + } + pi := tbInfo.GetPartitionInfo() + if pi == nil || pi.Definitions == nil { + return fmt.Errorf("workload repository could not load partition information for '%s'", tbl.destTable) + } + for _, pt := range pi.Definitions { + ot, err := parsePartitionName(pt.Name.L) + if err != nil { + return fmt.Errorf("workload repository could not cannot parse partition name (%s) for '%s': %v", pt.Name.L, tbl.destTable, err) + } + if int(now.Sub(ot).Hours()/24) < retention { + continue + } + sb := &strings.Builder{} + sqlescape.MustFormatSQL(sb, "ALTER TABLE %n.%n DROP PARTITION %n", + WorkloadSchema, tbl.destTable, pt.Name.L) + _, err = execRetry(ctx, sess, sb.String()) + if err != nil { + return fmt.Errorf("workload repository cannot drop partition (%s) on '%s': %v", pt.Name.L, tbl.destTable, err) + } + } + + return nil +} +func (w *worker) dropOldPartitions(ctx context.Context, sess sessionctx.Context, is infoschema.InfoSchema, now time.Time, retention int) error { if retention == 0 { // disabled housekeeping return nil } - sb := &strings.Builder{} - for _, tbl := range workloadTables { - tbSchema, err := is.TableByName(ctx, workloadSchemaCIStr, ast.NewCIStr(tbl.destTable)) - if err != nil { - logutil.BgLogger().Info("workload repository cannot get table", zap.String("tbl", tbl.destTable), zap.NamedError("err", err)) - continue - } - tbInfo := tbSchema.Meta() - for _, pt := range tbInfo.GetPartitionInfo().Definitions { - ot, err := time.Parse("p20060102", pt.Name.L) - if err != nil { - logutil.BgLogger().Info("workload repository cannot parse partition name", zap.String("part", pt.Name.L), zap.NamedError("err", err)) - break - } - if int(now.Sub(ot).Hours()/24) < retention { - continue - } - sb.Reset() - sqlescape.MustFormatSQL(sb, "ALTER TABLE %s.%s DROP PARTITION %s", - WorkloadSchema, tbl.destTable, pt.Name.L) - _, err = execRetry(ctx, sess, sb.String()) - if err != nil { - logutil.BgLogger().Info("workload repository cannot drop partition", zap.String("part", pt.Name.L), zap.NamedError("err", err)) - break - } + var err error + for _, tbl := range w.workloadTables { + err2 := dropOldPartition(ctx, is, &tbl, now, retention, sess) + if err2 != nil { + logutil.BgLogger().Warn("workload repository could not drop partitions", zap.NamedError("err", err2)) + err = errors.Join(err, err2) } } - return nil + + return err } -func (w *worker) startHouseKeeper(ctx context.Context) func() { +func (w *worker) getHouseKeeper(ctx context.Context, fn func(time.Time) time.Duration) func() { return func() { now := time.Now() - timer := time.NewTimer(calcNextTick(now)) + timer := time.NewTimer(fn(now)) defer timer.Stop() _sessctx := w.getSessionWithRetry() defer w.sesspool.Put(_sessctx) sess := _sessctx.(sessionctx.Context) + for { select { case <-ctx.Done(): @@ -125,21 +149,25 @@ func (w *worker) startHouseKeeper(ctx context.Context) func() { is := sessiontxn.GetTxnManager(sess).GetTxnInfoSchema() // create new partitions - if err := createAllPartitions(ctx, sess, is); err != nil { + if err := w.createAllPartitions(ctx, sess, is, now); err != nil { continue } + w.Lock() + retention := int(w.retentionDays) + w.Unlock() + // drop old partitions - if err := w.dropOldPartitions(ctx, sess, is, now); err != nil { + if err := w.dropOldPartitions(ctx, sess, is, now, retention); err != nil { continue } - // reschedule, drain channel first - if !timer.Stop() { - <-timer.C - } - timer.Reset(calcNextTick(now)) + timer.Reset(fn(now)) } } } } + +func (w *worker) startHouseKeeper(ctx context.Context) func() { + return w.getHouseKeeper(ctx, calcNextTick) +} diff --git a/pkg/util/workloadrepo/sampling.go b/pkg/util/workloadrepo/sampling.go index a4a86b71f869b..c6b9c34751d69 100644 --- a/pkg/util/workloadrepo/sampling.go +++ b/pkg/util/workloadrepo/sampling.go @@ -44,9 +44,7 @@ func (w *worker) samplingTable(ctx context.Context, rt *repositoryTable) { func (w *worker) startSample(ctx context.Context) func() { return func() { - w.Lock() - w.samplingTicker = time.NewTicker(time.Duration(w.samplingInterval) * time.Second) - w.Unlock() + w.resetSamplingInterval(w.samplingInterval) for { select { @@ -56,8 +54,8 @@ func (w *worker) startSample(ctx context.Context) func() { // sample thread var wg util.WaitGroupWrapper - for rtIdx := range workloadTables { - rt := &workloadTables[rtIdx] + for rtIdx := range w.workloadTables { + rt := &w.workloadTables[rtIdx] if rt.tableType != samplingTable { continue } @@ -73,10 +71,6 @@ func (w *worker) startSample(ctx context.Context) func() { } func (w *worker) resetSamplingInterval(newRate int32) { - if w.samplingTicker == nil { - return - } - if newRate == 0 { w.samplingTicker.Stop() } else { @@ -95,7 +89,9 @@ func (w *worker) changeSamplingInterval(_ context.Context, d string) error { if int32(n) != w.samplingInterval { w.samplingInterval = int32(n) - w.resetSamplingInterval(w.samplingInterval) + if w.samplingTicker != nil { + w.resetSamplingInterval(w.samplingInterval) + } } return nil diff --git a/pkg/util/workloadrepo/snapshot.go b/pkg/util/workloadrepo/snapshot.go index 933e4df83ecb1..8fd71a62ff596 100644 --- a/pkg/util/workloadrepo/snapshot.go +++ b/pkg/util/workloadrepo/snapshot.go @@ -46,7 +46,7 @@ func (w *worker) etcdCreate(ctx context.Context, key, val string) error { return nil } -func (w *worker) etcdGet(_ctx context.Context, key, defval string) (string, error) { +func (w *worker) etcdGet(_ctx context.Context, key string) (string, error) { ctx, cancel := context.WithTimeout(_ctx, etcdOpTimeout) defer cancel() res, err := w.etcdClient.Get(ctx, key) @@ -54,9 +54,8 @@ func (w *worker) etcdGet(_ctx context.Context, key, defval string) (string, erro return "", err } if len(res.Kvs) == 0 { - // nonexistent, create it atomically - // otherwise etcdCAS will fail - return defval, w.etcdCreate(_ctx, key, defval) + // key does not exist, just return an empty string + return "", nil } return string(res.Kvs[len(res.Kvs)-1].Value), nil } @@ -78,10 +77,14 @@ func (w *worker) etcdCAS(ctx context.Context, key, oval, nval string) error { } func (w *worker) getSnapID(ctx context.Context) (uint64, error) { - snapIDStr, err := w.etcdGet(ctx, snapIDKey, "0") + snapIDStr, err := w.etcdGet(ctx, snapIDKey) if err != nil { return 0, err } + if snapIDStr == "" { + // return zero when the key does not exist + return 0, nil + } return strconv.ParseUint(snapIDStr, 10, 64) } @@ -91,6 +94,10 @@ func (w *worker) updateSnapID(ctx context.Context, oid, nid uint64) error { strconv.FormatUint(nid, 10)) } +func (w *worker) createSnapID(ctx context.Context, nid uint64) error { + return w.etcdCreate(ctx, snapIDKey, strconv.FormatUint(nid, 10)) +} + func upsertHistSnapshot(ctx context.Context, sctx sessionctx.Context, snapID uint64) error { // TODO: fill DB_VER, WR_VER snapshotsInsert := sqlescape.MustEscapeSQL("INSERT INTO %n.%n (`BEGIN_TIME`, `SNAP_ID`) VALUES (now(), %%?) ON DUPLICATE KEY UPDATE `BEGIN_TIME` = now()", @@ -132,13 +139,19 @@ func (w *worker) takeSnapshot(ctx context.Context, sess sessionctx.Context, send // coordination logic if !w.owner.IsOwner() { if sendCommand { - command, err := w.etcdGet(ctx, snapCommandKey, "") + command, err := w.etcdGet(ctx, snapCommandKey) if err != nil { - logutil.BgLogger().Info("workload repository cannot get current snapid to send", zap.NamedError("err", err)) + logutil.BgLogger().Info("workload repository cannot get current snap command value", zap.NamedError("err", err)) return } - if err = w.etcdCAS(ctx, snapCommandKey, command, snapCommandTake); err != nil { + if command == "" { + err = w.etcdCreate(ctx, snapCommandKey, snapCommandTake) + } else { + err = w.etcdCAS(ctx, snapCommandKey, command, snapCommandTake) + } + + if err != nil { logutil.BgLogger().Info("workload repository cannot send snapshot command", zap.NamedError("err", err)) return } @@ -152,6 +165,7 @@ func (w *worker) takeSnapshot(ctx context.Context, sess sessionctx.Context, send logutil.BgLogger().Info("workload repository cannot get current snapid", zap.NamedError("err", err)) continue } + // Use UPSERT to ensure this SQL doesn't fail on duplicate snapID. // // NOTE: In a highly unlikely corner case, there could be two owners. @@ -159,12 +173,17 @@ func (w *worker) takeSnapshot(ctx context.Context, sess sessionctx.Context, send // due to another owner winning the etcd CAS loop. // While undesirable, this scenario is acceptable since both owners would // likely share similar datetime values and same cluster version. - if err := upsertHistSnapshot(ctx, sess, snapID+1); err != nil { logutil.BgLogger().Info("workload repository could not insert into hist_snapshots", zap.NamedError("err", err)) continue } - err = w.updateSnapID(ctx, snapID, snapID+1) + + if snapID == 0 { + err = w.createSnapID(ctx, snapID+1) + } else { + err = w.updateSnapID(ctx, snapID, snapID+1) + } + if err != nil { logutil.BgLogger().Info("workload repository cannot update current snapid", zap.Uint64("new_id", snapID), zap.NamedError("err", err)) continue @@ -177,9 +196,7 @@ func (w *worker) takeSnapshot(ctx context.Context, sess sessionctx.Context, send func (w *worker) startSnapshot(_ctx context.Context) func() { return func() { - w.Lock() - w.snapshotTicker = time.NewTicker(time.Duration(w.snapshotInterval) * time.Second) - w.Unlock() + w.resetSnapshotInterval(w.snapshotInterval) _sessctx := w.getSessionWithRetry() defer w.sesspool.Put(_sessctx) @@ -227,11 +244,11 @@ func (w *worker) startSnapshot(_ctx context.Context) func() { continue } - errs := make([]error, len(workloadTables)) + errs := make([]error, len(w.workloadTables)) var wg util.WaitGroupWrapper cnt := 0 - for rtIdx := range workloadTables { - rt := &workloadTables[rtIdx] + for rtIdx := range w.workloadTables { + rt := &w.workloadTables[rtIdx] if rt.tableType != snapshotTable { continue } @@ -256,9 +273,7 @@ func (w *worker) startSnapshot(_ctx context.Context) func() { } func (w *worker) resetSnapshotInterval(newRate int32) { - if w.snapshotTicker != nil { - w.snapshotTicker.Reset(time.Duration(newRate) * time.Second) - } + w.snapshotTicker.Reset(time.Duration(newRate) * time.Second) } func (w *worker) changeSnapshotInterval(_ context.Context, d string) error { @@ -272,7 +287,9 @@ func (w *worker) changeSnapshotInterval(_ context.Context, d string) error { if int32(n) != w.snapshotInterval { w.snapshotInterval = int32(n) - w.resetSnapshotInterval(w.snapshotInterval) + if w.snapshotTicker != nil { + w.resetSnapshotInterval(w.snapshotInterval) + } } return nil diff --git a/pkg/util/workloadrepo/table.go b/pkg/util/workloadrepo/table.go index d3c88a84bb972..b8de7d600fece 100644 --- a/pkg/util/workloadrepo/table.go +++ b/pkg/util/workloadrepo/table.go @@ -99,7 +99,7 @@ func buildInsertQuery(ctx context.Context, sess sessionctx.Context, rt *reposito return nil } -func (w *worker) createAllTables(ctx context.Context) error { +func (w *worker) createAllTables(ctx context.Context, now time.Time) error { _sessctx := w.getSessionWithRetry() sess := _sessctx.(sessionctx.Context) defer w.sesspool.Put(_sessctx) @@ -111,7 +111,7 @@ func (w *worker) createAllTables(ctx context.Context) error { } } - for _, tbl := range workloadTables { + for _, tbl := range w.workloadTables { if checkTableExistsByIS(ctx, is, tbl.destTable, zeroTime) { continue } @@ -128,12 +128,12 @@ func (w *worker) createAllTables(ctx context.Context) error { if tbl.tableType == metadataTable { sb := &strings.Builder{} fmt.Fprint(sb, createStmt) - generatePartitionDef(sb, "BEGIN_TIME") + generatePartitionDef(sb, "BEGIN_TIME", now) createStmt = sb.String() } else { sb := &strings.Builder{} fmt.Fprint(sb, createStmt) - generatePartitionDef(sb, "TS") + generatePartitionDef(sb, "TS", now) createStmt = sb.String() } @@ -142,18 +142,17 @@ func (w *worker) createAllTables(ctx context.Context) error { } } - return createAllPartitions(ctx, sess, is) + is = sess.GetDomainInfoSchema().(infoschema.InfoSchema) + return w.createAllPartitions(ctx, sess, is, now) } -// checkTablesExists will check if all tables are created and if the work is bootstrapped. -func (w *worker) checkTablesExists(ctx context.Context) bool { +func (w *worker) checkTablesExists(ctx context.Context, now time.Time) bool { _sessctx := w.getSessionWithRetry() sess := _sessctx.(sessionctx.Context) defer w.sesspool.Put(_sessctx) is := sess.GetDomainInfoSchema().(infoschema.InfoSchema) - now := time.Now() - return slice.AllOf(workloadTables, func(i int) bool { - return checkTableExistsByIS(ctx, is, workloadTables[i].destTable, now) + return slice.AllOf(w.workloadTables, func(i int) bool { + return checkTableExistsByIS(ctx, is, w.workloadTables[i].destTable, now) }) } @@ -168,16 +167,21 @@ func checkTableExistsByIS(ctx context.Context, is infoschema.InfoSchema, tblName return false } + // Insure that the table has a partition for tomorrow. tbInfo := tbSchema.Meta() - for i := range 2 { - newPtTime := now.AddDate(0, 0, i+1) - newPtName := "p" + newPtTime.Format("20060102") - ptInfos := tbInfo.GetPartitionInfo().Definitions - if slice.NoneOf(ptInfos, func(i int) bool { - return ptInfos[i].Name.L == newPtName - }) { - return false - } + if tbInfo == nil { + return false + } + pi := tbInfo.GetPartitionInfo() + if pi == nil || pi.Definitions == nil || len(pi.Definitions) == 0 { + return false } - return true + ptInfos := pi.Definitions + ot, err := parsePartitionName(ptInfos[len(ptInfos)-1].Name.L) + if err != nil { + return false + } + + // It doesn't matter if now has a timestamp. + return ot.After(now.AddDate(0, 0, 1)) } diff --git a/pkg/util/workloadrepo/utils.go b/pkg/util/workloadrepo/utils.go index 7c25ff6d04a94..fd350d02b9e43 100644 --- a/pkg/util/workloadrepo/utils.go +++ b/pkg/util/workloadrepo/utils.go @@ -16,47 +16,65 @@ package workloadrepo import ( "context" + "errors" "fmt" "strconv" "strings" "time" "github.com/pingcap/tidb/pkg/meta/model" - "github.com/pingcap/tidb/pkg/util/slice" ) -func generatePartitionDef(sb *strings.Builder, col string) { +func generatePartitionDef(sb *strings.Builder, col string, now time.Time) { fmt.Fprintf(sb, " PARTITION BY RANGE( TO_DAYS(%s) ) (", col) // tbInfo is nil, retval must be false - _ = generatePartitionRanges(sb, nil) + _, _ = generatePartitionRanges(sb, nil, now) fmt.Fprintf(sb, ")") } -func generatePartitionRanges(sb *strings.Builder, tbInfo *model.TableInfo) bool { - now := time.Now() - newPtNum := 2 - // add new partitions per day - // if all partitions to be added existed, do nothing - allExisted := true - for i := range newPtNum { - // TODO: should we make this UTC? timezone issues - newPtTime := now.AddDate(0, 0, i+1) - newPtName := "p" + newPtTime.Format("20060102") - if tbInfo != nil { - ptInfos := tbInfo.GetPartitionInfo().Definitions - if slice.AnyOf(ptInfos, func(i int) bool { - return ptInfos[i].Name.L == newPtName - }) { - continue +func generatePartitionName(t time.Time) string { + return "p" + t.Format("20060102") +} + +func parsePartitionName(part string) (time.Time, error) { + return time.ParseInLocation("p20060102", part, time.Local) +} + +func generatePartitionRanges(sb *strings.Builder, tbInfo *model.TableInfo, now time.Time) (bool, error) { + // Set lastPart to the latest partition found in table or the date for + // yesterday's partition if none is found. Note: The partition named for + // today's date holds yesterday's data. + lastPart := time.Date(now.Year(), now.Month(), now.Day(), 0, 0, 0, 0, time.Local) + if tbInfo != nil { + pi := tbInfo.GetPartitionInfo() + if pi != nil && pi.Definitions != nil && len(pi.Definitions) > 0 { + ptInfos := pi.Definitions + partDate, err := parsePartitionName(ptInfos[len(ptInfos)-1].Name.L) + if err != nil { + return true, err + } + + if partDate.After(lastPart) { + lastPart = partDate } } - if !allExisted && i > 0 { - fmt.Fprintf(sb, ",") + } + + // Add partitions for today and tomorrow. + allExisted := true + for i := range 2 { + newPartDate := time.Date(now.Year(), now.Month(), now.Day()+i+1, 0, 0, 0, 0, time.Local) + if newPartDate.After(lastPart) { + if !allExisted { + fmt.Fprintf(sb, ", ") + } + newPartName := generatePartitionName(newPartDate) + fmt.Fprintf(sb, "PARTITION %s VALUES LESS THAN (TO_DAYS('%s'))", newPartName, newPartDate.Format("2006-01-02")) + allExisted = false } - fmt.Fprintf(sb, "PARTITION %s VALUES LESS THAN (TO_DAYS('%s'))", newPtName, newPtTime.Format("2006-01-02")) - allExisted = false } - return allExisted + + return allExisted, nil } func (w *worker) setRetentionDays(_ context.Context, d string) error { @@ -72,5 +90,9 @@ func (w *worker) setRetentionDays(_ context.Context, d string) error { func validateDest(orig string) (string, error) { // validate S3 URL, etc... - return strings.ToLower(orig), nil + orig = strings.ToLower(orig) + if orig != "" && orig != "table" { + return "", errors.New("invalid repository destination") + } + return orig, nil } diff --git a/pkg/util/workloadrepo/worker.go b/pkg/util/workloadrepo/worker.go index 0882e82e4f6e0..d1879e45d6ede 100644 --- a/pkg/util/workloadrepo/worker.go +++ b/pkg/util/workloadrepo/worker.go @@ -79,8 +79,8 @@ var workloadTables = []repositoryTable{ ERROR TEXT DEFAULT NULL COMMENT 'extra messages are written if anything happens to block that snapshots.')`, WorkloadSchema, histSnapshotsTable), "", }, - //{"INFORMATION_SCHEMA", "TIDB_INDEX_USAGE", snapshotTable, "", "", "", ""}, - //{"INFORMATION_SCHEMA", "TIDB_STATEMENTS_STATS", snapshotTable, "", "", "", ""}, + {"INFORMATION_SCHEMA", "TIDB_INDEX_USAGE", snapshotTable, "", "", "", ""}, + {"INFORMATION_SCHEMA", "TIDB_STATEMENTS_STATS", snapshotTable, "", "", "", ""}, {"INFORMATION_SCHEMA", "CLIENT_ERRORS_SUMMARY_BY_HOST", snapshotTable, "", "", "", ""}, {"INFORMATION_SCHEMA", "CLIENT_ERRORS_SUMMARY_BY_USER", snapshotTable, "", "", "", ""}, {"INFORMATION_SCHEMA", "CLIENT_ERRORS_SUMMARY_GLOBAL", snapshotTable, "", "", "", ""}, @@ -106,14 +106,15 @@ type sessionPool interface { // worker is the main struct for workload repository. type worker struct { sync.Mutex - etcdClient *clientv3.Client - sesspool sessionPool - cancel context.CancelFunc - newOwner func(string, string) owner.Manager - owner owner.Manager - wg *util.WaitGroupEnhancedWrapper - enabled bool - instanceID string + etcdClient *clientv3.Client + sesspool sessionPool + cancel context.CancelFunc + newOwner func(string, string) owner.Manager + owner owner.Manager + wg *util.WaitGroupEnhancedWrapper + enabled bool + instanceID string + workloadTables []repositoryTable samplingInterval int32 samplingTicker *time.Ticker @@ -123,11 +124,7 @@ type worker struct { retentionDays int32 } -var workerCtx = worker{ - samplingInterval: defSamplingInterval, - snapshotInterval: defSnapshotInterval, - retentionDays: defRententionDays, -} +var workerCtx = worker{} func takeSnapshot() error { if workerCtx.snapshotChan == nil { @@ -187,10 +184,20 @@ func init() { }) } -func initializeWorker(w *worker, etcdCli *clientv3.Client, newOwner func(string, string) owner.Manager, sesspool sessionPool) { +func initializeWorker(w *worker, etcdCli *clientv3.Client, newOwner func(string, string) owner.Manager, sesspool sessionPool, workloadTables []repositoryTable) { w.etcdClient = etcdCli w.sesspool = sesspool w.newOwner = newOwner + w.workloadTables = workloadTables + w.samplingInterval = defSamplingInterval + w.snapshotInterval = defSnapshotInterval + w.retentionDays = defRententionDays + + w.snapshotTicker = time.NewTicker(time.Second) + w.snapshotTicker.Stop() + w.samplingTicker = time.NewTicker(time.Second) + w.samplingTicker.Stop() + w.wg = util.NewWaitGroupEnhancedWrapper("workloadrepo", nil, false) } @@ -199,7 +206,7 @@ func SetupRepository(dom *domain.Domain) { workerCtx.Lock() defer workerCtx.Unlock() - initializeWorker(&workerCtx, dom.GetEtcdClient(), dom.NewOwnerManager, dom.SysSessionPool()) + initializeWorker(&workerCtx, dom.GetEtcdClient(), dom.NewOwnerManager, dom.SysSessionPool(), workloadTables) if workerCtx.enabled { if err := workerCtx.start(); err != nil { @@ -274,6 +281,17 @@ func (w *worker) readInstanceID() error { return nil } +func (w *worker) fillInTableNames() { + for rtIdx := range w.workloadTables { + rt := &w.workloadTables[rtIdx] + if rt.table != "" { + if rt.destTable == "" { + rt.destTable = "HIST_" + rt.table + } + } + } +} + func (w *worker) startRepository(ctx context.Context) func() { // TODO: add another txn type ctx = kv.WithInternalSourceType(ctx, kv.InternalTxnOthers) @@ -284,14 +302,7 @@ func (w *worker) startRepository(ctx context.Context) func() { } ticker := time.NewTicker(time.Second) - for rtIdx := range workloadTables { - rt := &workloadTables[rtIdx] - if rt.table != "" { - if rt.destTable == "" { - rt.destTable = "HIST_" + rt.table - } - } - } + w.fillInTableNames() for { select { @@ -300,12 +311,12 @@ func (w *worker) startRepository(ctx context.Context) func() { case <-ticker.C: if w.owner.IsOwner() { logutil.BgLogger().Info("repository has owner!") - if err := w.createAllTables(ctx); err != nil { + if err := w.createAllTables(ctx, time.Now()); err != nil { logutil.BgLogger().Error("workload repository cannot create tables", zap.NamedError("err", err)) } } - if !w.checkTablesExists(ctx) { + if !w.checkTablesExists(ctx, time.Now()) { continue } diff --git a/pkg/util/workloadrepo/worker_test.go b/pkg/util/workloadrepo/worker_test.go index dc5e12e350dc5..5aed1f533b920 100644 --- a/pkg/util/workloadrepo/worker_test.go +++ b/pkg/util/workloadrepo/worker_test.go @@ -16,14 +16,19 @@ package workloadrepo import ( "context" + "fmt" "net/url" + "strconv" + "strings" "testing" "time" "github.com/pingcap/tidb/pkg/domain" + "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/owner" "github.com/pingcap/tidb/pkg/parser/ast" + "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/testkit" "github.com/stretchr/testify/require" clientv3 "go.etcd.io/etcd/client/v3" @@ -35,13 +40,17 @@ func setupWorkerForTest(ctx context.Context, etcdCli *clientv3.Client, dom *doma if !testWorker { wrk = &workerCtx } + + workloadTables2 := make([]repositoryTable, len(workloadTables)) + copy(workloadTables2, workloadTables) + owner.ManagerSessionTTL = 3 initializeWorker(wrk, etcdCli, func(s1, s2 string) owner.Manager { return owner.NewOwnerManager(ctx, etcdCli, s1, id, s2) }, dom.SysSessionPool(), - ) + workloadTables2) wrk.samplingInterval = 1 wrk.snapshotInterval = 1 wrk.instanceID = id @@ -50,7 +59,8 @@ func setupWorkerForTest(ctx context.Context, etcdCli *clientv3.Client, dom *doma func setupDomainAndContext(t *testing.T) (context.Context, kv.Storage, *domain.Domain, string) { ctx := context.Background() - var cancel context.CancelFunc + ctx = kv.WithInternalSourceType(ctx, kv.InternalTxnOthers) + var cancel context.CancelFunc = nil if ddl, ok := t.Deadline(); ok { ctx, cancel = context.WithDeadline(ctx, ddl) } @@ -106,7 +116,27 @@ func setupWorker(ctx context.Context, t *testing.T, addr string, dom *domain.Dom return wrk } -func TestMultipleWorker(t *testing.T) { +func eventuallyWithLock(t *testing.T, wrk *worker, fn func() bool) { + require.Eventually(t, func() bool { + wrk.Lock() + defer wrk.Unlock() + return fn() + }, time.Minute, time.Second) +} + +func trueWithLock(t *testing.T, wrk *worker, fn func() bool) { + wrk.Lock() + defer wrk.Unlock() + require.True(t, fn()) +} + +func waitForTables(ctx context.Context, t *testing.T, wrk *worker, now time.Time) { + require.Eventually(t, func() bool { + return wrk.checkTablesExists(ctx, now) + }, time.Minute, time.Second) +} + +func TestRaceToCreateTablesWorker(t *testing.T) { ctx, store, dom, addr := setupDomainAndContext(t) _, ok := dom.InfoSchema().SchemaByName(ast.NewCIStr("workload_schema")) @@ -116,11 +146,12 @@ func TestMultipleWorker(t *testing.T) { wrk2 := setupWorker(ctx, t, addr, dom, "worker2", true) wrk1.changeSnapshotInterval(nil, "3600") wrk2.changeSnapshotInterval(nil, "3600") + now := time.Now() require.NoError(t, wrk1.setRepositoryDest(ctx, "table")) require.NoError(t, wrk2.setRepositoryDest(ctx, "table")) require.Eventually(t, func() bool { - return wrk1.checkTablesExists(ctx) && wrk2.checkTablesExists(ctx) + return wrk1.checkTablesExists(ctx, now) && wrk2.checkTablesExists(ctx, now) }, time.Minute, time.Second) tk := testkit.NewTestKit(t, store) @@ -149,6 +180,76 @@ func TestMultipleWorker(t *testing.T) { }, time.Minute, time.Second) } +func getMultipleWorkerCount(tk *testkit.TestKit, worker string) int { + res := tk.MustQuery("select count(*) from workload_schema.hist_memory_usage group by instance_id having instance_id = '" + worker + "'").Rows() + return len(res) +} + +func TestMultipleWorker(t *testing.T) { + ctx, store, dom, addr := setupDomainAndContext(t) + tk := testkit.NewTestKit(t, store) + + wrk1 := setupWorker(ctx, t, addr, dom, "worker1", true) + wrk1.changeSamplingInterval(ctx, "1") + wrk2 := setupWorker(ctx, t, addr, dom, "worker2", true) + wrk2.changeSamplingInterval(ctx, "1") + + // start worker 1 + now := time.Now() + require.NoError(t, wrk1.setRepositoryDest(ctx, "table")) + waitForTables(ctx, t, wrk1, now) + require.True(t, wrk1.owner.IsOwner()) + + require.Eventually(t, func() bool { + return getMultipleWorkerCount(tk, "worker1") >= 1 + }, time.Minute, time.Second) + + // start worker 2 + require.NoError(t, wrk2.setRepositoryDest(ctx, "table")) + eventuallyWithLock(t, wrk2, func() bool { return wrk2.owner != nil }) + require.True(t, !wrk2.owner.IsOwner()) + + require.Eventually(t, func() bool { + return getMultipleWorkerCount(tk, "worker2") >= 1 + }, time.Minute, time.Second) + + // stop worker 1, worker 2 should become owner + require.NoError(t, wrk1.setRepositoryDest(ctx, "")) + require.Eventually(t, func() bool { + return wrk2.owner.IsOwner() + }, time.Minute, time.Second) + + // start worker 1 again + require.NoError(t, wrk1.setRepositoryDest(ctx, "table")) + eventuallyWithLock(t, wrk1, func() bool { return wrk1.owner != nil }) + require.True(t, !wrk1.owner.IsOwner()) + + // get counts from tables for both workers + cnt1 := getMultipleWorkerCount(tk, "worker1") + cnt2 := getMultipleWorkerCount(tk, "worker2") + + require.Eventually(t, func() bool { + cnt := getMultipleWorkerCount(tk, "worker1") + return cnt >= cnt1 + }, time.Minute, time.Second) + + // stop worker 2, worker 1 should become owner + require.NoError(t, wrk2.setRepositoryDest(ctx, "")) + require.Eventually(t, func() bool { + return wrk1.owner.IsOwner() + }, time.Minute, time.Second) + + // start worker 2 again + require.NoError(t, wrk2.setRepositoryDest(ctx, "table")) + eventuallyWithLock(t, wrk2, func() bool { return wrk2.owner != nil }) + require.True(t, !wrk2.owner.IsOwner()) + + require.Eventually(t, func() bool { + cnt := getMultipleWorkerCount(tk, "worker2") + return cnt >= cnt2 + }, time.Minute, time.Second) +} + func TestGlobalWorker(t *testing.T) { ctx, store, dom, addr := setupDomainAndContext(t) tk := testkit.NewTestKit(t, store) @@ -157,15 +258,14 @@ func TestGlobalWorker(t *testing.T) { require.False(t, ok) wrk := setupWorker(ctx, t, addr, dom, "worker", false) + now := time.Now() tk.MustExec("set @@global.tidb_workload_repository_dest='table'") - require.Eventually(t, func() bool { - return wrk.checkTablesExists(ctx) - }, time.Minute, time.Second) + waitForTables(ctx, t, wrk, now) // sampling succeeded require.Eventually(t, func() bool { - res := tk.MustQuery("select instance_id, count(*) from workload_schema.hist_memory_usage group by instance_id").Rows() + res := tk.MustQuery("select instance_id from workload_schema.hist_memory_usage").Rows() return len(res) >= 1 }, time.Minute, time.Second) } @@ -178,12 +278,13 @@ func TestAdminWorkloadRepo(t *testing.T) { require.False(t, ok) wrk := setupWorker(ctx, t, addr, dom, "worker", false) + now := time.Now() tk.MustExec("set @@global.tidb_workload_repository_snapshot_interval='5000'") tk.MustExec("set @@global.tidb_workload_repository_active_sampling_interval='600'") tk.MustExec("set @@global.tidb_workload_repository_dest='table'") require.Eventually(t, func() bool { - return wrk.checkTablesExists(ctx) + return wrk.checkTablesExists(ctx, now) }, time.Minute, time.Second) // able to snapshot manually @@ -194,6 +295,567 @@ func TestAdminWorkloadRepo(t *testing.T) { }, time.Minute, time.Second) // disable the worker and it will fail - tk.MustExec("set @@global.tidb_workload_repository_dest='ble'") + tk.MustExec("set @@global.tidb_workload_repository_dest=''") tk.MustExecToErr("admin create workload snapshot") } + +func getRows(t *testing.T, tk *testkit.TestKit, cnt int, maxSecs int, query string) [][]any { + var rows [][]any + require.Eventually(t, func() bool { + rows = tk.MustQuery(query).Rows() + return len(rows) == cnt + }, time.Second*time.Duration(maxSecs*cnt), time.Millisecond*100) + return rows +} + +func validateDate(t *testing.T, row []any, idx int, lastRowTs time.Time, maxSecs int) time.Time { + loc := lastRowTs.Location() + actualTs, err := time.ParseInLocation("2006-01-02 15:04:05", row[idx].(string), loc) + require.NoError(t, err) + require.Greater(t, actualTs.Unix(), lastRowTs.Unix()) // after last row + require.LessOrEqual(t, actualTs.Unix(), lastRowTs.Unix()+int64(maxSecs)) // within last maxSecs seconds of last row + return actualTs +} + +func SamplingTimingWorker(t *testing.T, tk *testkit.TestKit, lastRowTs time.Time, cnt int, maxSecs int) time.Time { + rows := getRows(t, tk, cnt, maxSecs, "select instance_id, ts from "+WorkloadSchema+".hist_memory_usage where ts > '"+lastRowTs.Format("2006-01-02 15:04:05")+"' order by ts asc") + + for _, row := range rows { + // check that the instance_id is correct + require.Equal(t, "worker", row[0]) // instance_id should match worker name + + // check that each rows are the correct interval apart + lastRowTs = validateDate(t, row, 1, lastRowTs, maxSecs) + } + + return lastRowTs +} + +func TestSamplingTimingWorker(t *testing.T) { + ctx, store, dom, addr := setupDomainAndContext(t) + tk := testkit.NewTestKit(t, store) + + wrk := setupWorker(ctx, t, addr, dom, "worker", true) + wrk.changeSamplingInterval(ctx, "2") + now := time.Now() + wrk.setRepositoryDest(ctx, "table") + + waitForTables(ctx, t, wrk, now) + + lastRowTs := time.Now() + lastRowTs = SamplingTimingWorker(t, tk, lastRowTs, 3, 3) + + // Change interval and verify new samples are taken at new interval + wrk.changeSamplingInterval(ctx, "5") + + _ = SamplingTimingWorker(t, tk, lastRowTs, 3, 6) +} + +func findMatchingRowForSnapshot(t *testing.T, rowidx int, snapRows [][]any, row []any, lastRowTs time.Time, maxSecs int) { + require.Less(t, rowidx, len(snapRows)) + row2 := snapRows[rowidx] + require.Equal(t, row2[0], row[0]) + require.Equal(t, row2[2], "worker") + validateDate(t, row2, 1, lastRowTs, maxSecs) +} + +func SnapshotTimingWorker(t *testing.T, tk *testkit.TestKit, lastRowTs time.Time, lastSnapID int, cnt int, maxSecs int) (time.Time, int) { + rows := getRows(t, tk, cnt, maxSecs, "select snap_id, begin_time from "+WorkloadSchema+"."+histSnapshotsTable+" where begin_time > '"+lastRowTs.Format("2006-01-02 15:04:05")+"' order by begin_time asc") + + // We want to get all rows if we are starting from 0. + snapWhere := "" + if lastSnapID > 0 { + snapWhere = " where snap_id > " + strconv.Itoa(lastSnapID) + } + + rows2 := getRows(t, tk, cnt, maxSecs, "select snap_id, ts, instance_id from WORKLOAD_SCHEMA.HIST_MEMORY_USAGE2"+snapWhere+" order by snap_id asc") + rows3 := getRows(t, tk, cnt, maxSecs, "select snap_id, ts, instance_id from WORKLOAD_SCHEMA.HIST_MEMORY_USAGE3"+snapWhere+" order by snap_id asc") + rowidx := 0 + + for _, row := range rows { + snapID, err := strconv.Atoi(row[0].(string)) + require.NoError(t, err) + require.Equal(t, lastSnapID+1, snapID) + + actualTs := validateDate(t, row, 1, lastRowTs, maxSecs) + + findMatchingRowForSnapshot(t, rowidx, rows2, row, lastRowTs, maxSecs) + findMatchingRowForSnapshot(t, rowidx, rows3, row, lastRowTs, maxSecs) + rowidx++ + + lastSnapID = snapID + lastRowTs = actualTs + } + + return lastRowTs, lastSnapID +} + +func TestSnapshotTimingWorker(t *testing.T) { + ctx, store, dom, addr := setupDomainAndContext(t) + tk := testkit.NewTestKit(t, store) + + wrk := setupWorker(ctx, t, addr, dom, "worker", true) + + // MEMORY_USAGE will contain a single row. Ideal for testing here. + wrk.workloadTables[1] = repositoryTable{ + "INFORMATION_SCHEMA", "MEMORY_USAGE", snapshotTable, "HIST_MEMORY_USAGE2", "", "", "", + } + wrk.workloadTables = append(wrk.workloadTables, repositoryTable{ + "INFORMATION_SCHEMA", "MEMORY_USAGE", snapshotTable, "HIST_MEMORY_USAGE3", "", "", "", + }) + + wrk.changeSnapshotInterval(ctx, "2") + now := time.Now() + wrk.setRepositoryDest(ctx, "table") + + waitForTables(ctx, t, wrk, now) + + // Check that snapshots are taken at 2 second intervals + lastRowTs := time.Now() + lastSnapID := 0 + lastRowTs, lastSnapID = SnapshotTimingWorker(t, tk, lastRowTs, lastSnapID, 3, 3) + + // Change interval and verify new snapshots are taken at new interval + wrk.changeSnapshotInterval(ctx, "6") + _, _ = SnapshotTimingWorker(t, tk, lastRowTs, lastSnapID, 1, 7) +} + +func TestStoppingAndRestartingWorker(t *testing.T) { + ctx, store, dom, addr := setupDomainAndContext(t) + tk := testkit.NewTestKit(t, store) + + wrk := setupWorker(ctx, t, addr, dom, "worker", true) + wrk.changeSamplingInterval(ctx, "1") + wrk.changeSnapshotInterval(ctx, "1") + now := time.Now() + wrk.setRepositoryDest(ctx, "table") + + waitForTables(ctx, t, wrk, now) + + // There should be one row every second. + require.Eventually(t, func() bool { + return len(tk.MustQuery("select instance_id from workload_schema.hist_memory_usage").Rows()) > 0 + }, time.Minute, time.Second) + require.Eventually(t, func() bool { + return len(tk.MustQuery("select snap_id from workload_schema."+histSnapshotsTable).Rows()) > 0 + }, time.Minute, time.Second) + + // Stop worker and verify no new samples are taken + wrk.setRepositoryDest(ctx, "") + eventuallyWithLock(t, wrk, func() bool { return wrk.cancel == nil }) + samplingCnt := len(tk.MustQuery("select instance_id from workload_schema.hist_memory_usage").Rows()) + snapshotCnt := len(tk.MustQuery("select snap_id from workload_schema." + histSnapshotsTable).Rows()) + + // Wait for 5 seconds to make sure no new samples are taken + time.Sleep(time.Second * 5) + require.True(t, len(tk.MustQuery("select instance_id from workload_schema.hist_memory_usage").Rows()) == samplingCnt) + require.True(t, len(tk.MustQuery("select snap_id from workload_schema."+histSnapshotsTable).Rows()) == snapshotCnt) + + // Restart worker and verify new samples are taken + wrk.setRepositoryDest(ctx, "table") + + require.Eventually(t, func() bool { + return len(tk.MustQuery("select instance_id from workload_schema.hist_memory_usage").Rows()) >= samplingCnt + }, time.Minute, time.Second) + require.Eventually(t, func() bool { + return len(tk.MustQuery("select snap_id from workload_schema."+histSnapshotsTable).Rows()) >= snapshotCnt + }, time.Minute, time.Second) +} + +func TestSettingSQLVariables(t *testing.T) { + ctx, store, dom, addr := setupDomainAndContext(t) + tk := testkit.NewTestKit(t, store) + + wrk := setupWorker(ctx, t, addr, dom, "worker", false) + + /* Order of less than minimum, maximum, minimum, and greater than maximum + tests are not random. Because we need to know if the invalid values are + converted to the minimum or maximum. */ + + // Test values less than minimum + tk.MustExec("set @@global." + repositorySamplingInterval + " = -1") + tk.MustExec("set @@global." + repositorySnapshotInterval + " = 899") + tk.MustExec("set @@global." + repositoryRetentionDays + " = -1") + eventuallyWithLock(t, wrk, func() bool { return int32(0) == wrk.samplingInterval }) + eventuallyWithLock(t, wrk, func() bool { return int32(900) == wrk.snapshotInterval }) + eventuallyWithLock(t, wrk, func() bool { return int32(0) == wrk.retentionDays }) + + // Test maximum values + tk.MustExec("set @@global." + repositorySamplingInterval + " = 600") + tk.MustExec("set @@global." + repositorySnapshotInterval + " = 7200") + tk.MustExec("set @@global." + repositoryRetentionDays + " = 365") + eventuallyWithLock(t, wrk, func() bool { return int32(600) == wrk.samplingInterval }) + eventuallyWithLock(t, wrk, func() bool { return int32(7200) == wrk.snapshotInterval }) + eventuallyWithLock(t, wrk, func() bool { return int32(365) == wrk.retentionDays }) + + // Test minimum values + tk.MustExec("set @@global." + repositorySamplingInterval + " = 0") + tk.MustExec("set @@global." + repositorySnapshotInterval + " = 900") + tk.MustExec("set @@global." + repositoryRetentionDays + " = 0") + eventuallyWithLock(t, wrk, func() bool { return int32(0) == wrk.samplingInterval }) + eventuallyWithLock(t, wrk, func() bool { return int32(900) == wrk.snapshotInterval }) + eventuallyWithLock(t, wrk, func() bool { return int32(0) == wrk.retentionDays }) + + // Test values greater than maximum + tk.MustExec("set @@global." + repositorySamplingInterval + " = 601") + tk.MustExec("set @@global." + repositorySnapshotInterval + " = 7201") + tk.MustExec("set @@global." + repositoryRetentionDays + " = 366") + eventuallyWithLock(t, wrk, func() bool { return int32(600) == wrk.samplingInterval }) + eventuallyWithLock(t, wrk, func() bool { return int32(7200) == wrk.snapshotInterval }) + eventuallyWithLock(t, wrk, func() bool { return int32(365) == wrk.retentionDays }) + + // Test invalid value for sampling interval + err := tk.ExecToErr("set @@global." + repositorySamplingInterval + " = 'invalid'") + require.Error(t, err) + require.Contains(t, err.Error(), "Incorrect argument type") + + err = tk.ExecToErr("set @@global." + repositorySnapshotInterval + " = 'invalid'") + require.Error(t, err) + require.Contains(t, err.Error(), "Incorrect argument type") + + err = tk.ExecToErr("set @@global." + repositoryRetentionDays + " = 'invalid'") + require.Error(t, err) + require.Contains(t, err.Error(), "Incorrect argument type") + + trueWithLock(t, wrk, func() bool { return int32(600) == wrk.samplingInterval }) + trueWithLock(t, wrk, func() bool { return int32(7200) == wrk.snapshotInterval }) + trueWithLock(t, wrk, func() bool { return int32(365) == wrk.retentionDays }) + + // Test setting repository destination + tk.MustExec("set @@global." + repositoryDest + " = 'table'") + eventuallyWithLock(t, wrk, func() bool { return wrk.enabled }) + + // Test disabling repository + tk.MustExec("set @@global." + repositoryDest + " = ''") + eventuallyWithLock(t, wrk, func() bool { return !wrk.enabled }) + + // Test invalid value for repository destination + err = tk.ExecToErr("set @@global." + repositoryDest + " = 'invalid'") + require.Error(t, err) + require.Contains(t, err.Error(), "invalid repository destination") +} + +func getTable(t *testing.T, tableName string, wrk *worker) *repositoryTable { + var tbl *repositoryTable = nil + for _, nt := range wrk.workloadTables { + if nt.table == tableName { + tbl = &nt + } + } + require.NotNil(t, tbl) + return tbl +} + +func validatePartitionsMatchExpected(ctx context.Context, t *testing.T, + sess sessionctx.Context, tbl *repositoryTable, partitions []time.Time) bool { + // validate that the partitions exactly match as expected + ep := make(map[string]struct{}) + for _, p := range partitions { + ep[generatePartitionName(p.AddDate(0, 0, 1))] = struct{}{} + } + + is := sess.GetDomainInfoSchema().(infoschema.InfoSchema) + tbSchema, err := is.TableByName(ctx, workloadSchemaCIStr, ast.NewCIStr(tbl.destTable)) + require.NoError(t, err) + tbInfo := tbSchema.Meta() + require.NotNil(t, tbInfo) + pi := tbInfo.GetPartitionInfo() + tp := make(map[string]struct{}) + if pi != nil && pi.Definitions != nil { + for _, p := range pi.Definitions { + tp[p.Name.L] = struct{}{} + } + } + + if len(tp) != len(ep) { + return false + } + + for p := range ep { + _, ok := tp[p] + if !ok { + return false + } + } + + return true +} + +func buildPartitionRow(now time.Time) string { + newPtTime := now.AddDate(0, 0, 1) + newPtName := "p" + newPtTime.Format("20060102") + parttemp := `PARTITION %s VALUES LESS THAN (TO_DAYS('%s'))` + return fmt.Sprintf(parttemp, newPtName, newPtTime.Format("2006-01-02")) +} + +func buildPartitionString(partitions []time.Time) string { + sb := &strings.Builder{} + if len(partitions) > 0 { + fmt.Fprint(sb, ` PARTITION BY RANGE (TO_DAYS(TS)) (`) + first := true + for _, p := range partitions { + if !first { + fmt.Fprint(sb, ", ") + } + fmt.Fprint(sb, buildPartitionRow(p)) + first = false + } + fmt.Fprint(sb, `)`) + } + return sb.String() +} + +func createTableWithParts(ctx context.Context, t *testing.T, tk *testkit.TestKit, tbl *repositoryTable, + sess sessionctx.Context, partitions []time.Time) { + createSQL, err := buildCreateQuery(ctx, sess, tbl) + require.NoError(t, err) + createSQL += buildPartitionString(partitions) + tk.MustExec(createSQL) + require.True(t, validatePartitionsMatchExpected(ctx, t, sess, tbl, partitions)) +} + +func validatePartitionCreation(ctx context.Context, now time.Time, t *testing.T, + sess sessionctx.Context, tk *testkit.TestKit, wrk *worker, + firstTestFails bool, tableName string, partitions []time.Time, expectedParts []time.Time) { + tbl := getTable(t, tableName, wrk) + createTableWithParts(ctx, t, tk, tbl, sess, partitions) + + is := sess.GetDomainInfoSchema().(infoschema.InfoSchema) + require.False(t, firstTestFails == checkTableExistsByIS(ctx, is, tbl.destTable, now)) + + is = sess.GetDomainInfoSchema().(infoschema.InfoSchema) + require.NoError(t, createPartition(ctx, is, tbl, sess, now)) + + require.True(t, validatePartitionsMatchExpected(ctx, t, sess, tbl, expectedParts)) + + is = sess.GetDomainInfoSchema().(infoschema.InfoSchema) + require.True(t, checkTableExistsByIS(ctx, is, tbl.destTable, now)) +} + +func TestCreatePartition(t *testing.T) { + ctx, store, dom, addr := setupDomainAndContext(t) + wrk := setupWorker(ctx, t, addr, dom, "worker", true) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("create database WORKLOAD_SCHEMA") + tk.MustExec("use WORKLOAD_SCHEMA") + + _sessctx := wrk.getSessionWithRetry() + sess := _sessctx.(sessionctx.Context) + + wrk.fillInTableNames() + + now := time.Now() + + /* Tables without partitions are not currently supported. */ + + // Should create one partition for today and tomorrow on a table with only old partitions before today. + partitions := []time.Time{now.AddDate(0, 0, -1)} + expectedParts := []time.Time{now.AddDate(0, 0, -1), now, now.AddDate(0, 0, 1)} + validatePartitionCreation(ctx, now, t, sess, tk, wrk, true, "PROCESSLIST", partitions, expectedParts) + + // Should create one partition for tomorrow on a table with only a partition for today. + partitions = []time.Time{now} + expectedParts = []time.Time{now, now.AddDate(0, 0, 1)} + validatePartitionCreation(ctx, now, t, sess, tk, wrk, true, "DATA_LOCK_WAITS", partitions, expectedParts) + + // Should not create any partitions on a table with only a partition for tomorrow. + partitions = []time.Time{now.AddDate(0, 0, 1)} + expectedParts = []time.Time{now.AddDate(0, 0, 1)} + validatePartitionCreation(ctx, now, t, sess, tk, wrk, false, "TIDB_TRX", partitions, expectedParts) + + // Should not create any partitions on a table with only partitions for both today and tomorrow. + partitions = []time.Time{now, now.AddDate(0, 0, 1)} + expectedParts = []time.Time{now, now.AddDate(0, 0, 1)} + validatePartitionCreation(ctx, now, t, sess, tk, wrk, false, "MEMORY_USAGE", partitions, expectedParts) + + // Should not create any partitions on a table with a partition for the day after tomorrow. + partitions = []time.Time{now.AddDate(0, 0, 2)} + expectedParts = []time.Time{now.AddDate(0, 0, 2)} + validatePartitionCreation(ctx, now, t, sess, tk, wrk, false, "CLUSTER_LOAD", partitions, expectedParts) + + // Should not fill in missing partitions on a table with a partition for dates beyond tomorrow. + partitions = []time.Time{now, now.AddDate(0, 0, 3)} + expectedParts = []time.Time{now, now.AddDate(0, 0, 3)} + validatePartitionCreation(ctx, now, t, sess, tk, wrk, false, "TIDB_HOT_REGIONS", partitions, expectedParts) + + // this table should be updated when the repository is enabled + partitions = []time.Time{now} + createTableWithParts(ctx, t, tk, getTable(t, "DEADLOCKS", wrk), sess, partitions) + + // turn on the repository and see if it creates the remaining tables + now = time.Now() + wrk.setRepositoryDest(ctx, "table") + waitForTables(ctx, t, wrk, now) +} + +func validatePartitionDrop(ctx context.Context, now time.Time, t *testing.T, + sess sessionctx.Context, tk *testkit.TestKit, wrk *worker, + tableName string, partitions []time.Time, retention int, shouldErr bool, expectedParts []time.Time) { + tbl := getTable(t, tableName, wrk) + createTableWithParts(ctx, t, tk, tbl, sess, partitions) + + require.True(t, validatePartitionsMatchExpected(ctx, t, sess, tbl, partitions)) + + is := sess.GetDomainInfoSchema().(infoschema.InfoSchema) + err := dropOldPartition(ctx, is, tbl, now, retention, sess) + if shouldErr { + require.Error(t, err) + } else { + require.NoError(t, err) + } + + require.True(t, validatePartitionsMatchExpected(ctx, t, sess, tbl, expectedParts)) +} + +func TestDropOldPartitions(t *testing.T) { + ctx, store, dom, addr := setupDomainAndContext(t) + wrk := setupWorker(ctx, t, addr, dom, "worker", true) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("create database WORKLOAD_SCHEMA") + tk.MustExec("use WORKLOAD_SCHEMA") + + _sessctx := wrk.getSessionWithRetry() + sess := _sessctx.(sessionctx.Context) + + wrk.fillInTableNames() + + now := time.Now() + + var partitions []time.Time + var expectedParts []time.Time + // Zero retention days disabled the partition deletion code. So we will not test it here. + + // Should not trim any partitions + partitions = []time.Time{now.AddDate(0, 0, -1), now, now.AddDate(0, 0, 1)} + expectedParts = []time.Time{now.AddDate(0, 0, -1), now, now.AddDate(0, 0, 1)} + validatePartitionDrop(ctx, now, t, sess, tk, wrk, "PROCESSLIST", partitions, 1, false, expectedParts) + + // Test trimming a single partition more than one date before retention date. + partitions = []time.Time{now.AddDate(0, 0, -2), now, now.AddDate(0, 0, 1)} + expectedParts = []time.Time{now, now.AddDate(0, 0, 1)} + validatePartitionDrop(ctx, now, t, sess, tk, wrk, "DATA_LOCK_WAITS", partitions, 1, false, expectedParts) + + // Check that multiple partitions can be removed. + partitions = []time.Time{now.AddDate(0, 0, -3), now.AddDate(0, 0, -2), now.AddDate(0, 0, -1), now, now.AddDate(0, 0, 1)} + expectedParts = []time.Time{now.AddDate(0, 0, -1), now, now.AddDate(0, 0, 1)} + validatePartitionDrop(ctx, now, t, sess, tk, wrk, "TIDB_TRX", partitions, 1, false, expectedParts) + + // should trim nothing + partitions = []time.Time{now.AddDate(0, 0, -2), now.AddDate(0, 0, -1), now, now.AddDate(0, 0, 1)} + expectedParts = []time.Time{now.AddDate(0, 0, -2), now.AddDate(0, 0, -1), now, now.AddDate(0, 0, 1)} + validatePartitionDrop(ctx, now, t, sess, tk, wrk, "MEMORY_USAGE", partitions, 2, false, expectedParts) + + // should trim one partition + partitions = []time.Time{now.AddDate(0, 0, -3), now.AddDate(0, 0, -2), now.AddDate(0, 0, 1)} + expectedParts = []time.Time{now.AddDate(0, 0, -2), now.AddDate(0, 0, 1)} + validatePartitionDrop(ctx, now, t, sess, tk, wrk, "CLUSTER_LOAD", partitions, 2, false, expectedParts) + + // validate that it works when not dropping any partitions + partitions = []time.Time{now.AddDate(0, 0, -1)} + expectedParts = []time.Time{now.AddDate(0, 0, -1)} + validatePartitionDrop(ctx, now, t, sess, tk, wrk, "TIDB_HOT_REGIONS", partitions, 2, false, expectedParts) + + // there must be partitions, so this should error + partitions = []time.Time{now.AddDate(0, 0, -2)} + expectedParts = []time.Time{now.AddDate(0, 0, -2)} + validatePartitionDrop(ctx, now, t, sess, tk, wrk, "TIKV_STORE_STATUS", partitions, 1, true, expectedParts) +} + +func TestAddNewPartitionsOnStart(t *testing.T) { + ctx, _, dom, addr := setupDomainAndContext(t) + + wrk := setupWorker(ctx, t, addr, dom, "worker", true) + wrk.fillInTableNames() + now := time.Now() + require.NoError(t, wrk.createAllTables(ctx, now)) + require.True(t, wrk.checkTablesExists(ctx, now)) + + _sessctx := wrk.getSessionWithRetry() + sess := _sessctx.(sessionctx.Context) + expectedParts := []time.Time{now, now.AddDate(0, 0, 1)} + for _, tbl := range wrk.workloadTables { + // check for now and now + 1 partitions + require.True(t, validatePartitionsMatchExpected(ctx, t, sess, &tbl, expectedParts)) + } +} + +func getNextTick(now time.Time) time.Duration { + return time.Second +} + +func TestHouseKeeperThread(t *testing.T) { + ctx, store, dom, addr := setupDomainAndContext(t) + wrk := setupWorker(ctx, t, addr, dom, "worker", true) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("create database WORKLOAD_SCHEMA") + tk.MustExec("use WORKLOAD_SCHEMA") + + _sessctx := wrk.getSessionWithRetry() + sess := _sessctx.(sessionctx.Context) + + wrk.workloadTables = []repositoryTable{ + {"INFORMATION_SCHEMA", "PROCESSLIST", samplingTable, "", "", "", ""}, + {"INFORMATION_SCHEMA", "DATA_LOCK_WAITS", samplingTable, "", "", "", ""}, + } + wrk.fillInTableNames() + + now := time.Now() + var parts []time.Time + + // This will have a partition added for tomorrow. + parts = []time.Time{now.AddDate(0, 0, -1), now} + plTbl := getTable(t, "PROCESSLIST", wrk) + createTableWithParts(ctx, t, tk, plTbl, sess, parts) + require.True(t, validatePartitionsMatchExpected(ctx, t, sess, plTbl, parts)) + + // This will have partitions removed. -3 will be removed when retention is 2, and -2 will be removed at 1. + parts = []time.Time{now.AddDate(0, 0, -3), now.AddDate(0, 0, -2), now.AddDate(0, 0, -1), now} + dlwTbl := getTable(t, "DATA_LOCK_WAITS", wrk) + createTableWithParts(ctx, t, tk, dlwTbl, sess, parts) + require.True(t, validatePartitionsMatchExpected(ctx, t, sess, dlwTbl, parts)) + + // start the thread + wrk.owner = wrk.newOwner(ownerKey, promptKey) + require.NoError(t, wrk.owner.CampaignOwner()) + require.Eventually(t, func() bool { + return wrk.owner.IsOwner() + }, time.Second*10, time.Second) + + // build remaining tables and start housekeeper + wrk.retentionDays = 2 + fn := wrk.getHouseKeeper(ctx, getNextTick) + go fn() + + // check paritions + require.Eventually(t, func() bool { + return validatePartitionsMatchExpected(ctx, t, sess, plTbl, []time.Time{now.AddDate(0, 0, -1), now, now.AddDate(0, 0, 1)}) + }, time.Second*10, time.Second) + require.Eventually(t, func() bool { + return validatePartitionsMatchExpected(ctx, t, sess, dlwTbl, []time.Time{now.AddDate(0, 0, -2), now.AddDate(0, 0, -1), now, now.AddDate(0, 0, 1)}) + }, time.Second*10, time.Second) + + // Change the retention value. We just want to see that it is still running. + wrk.Lock() + wrk.retentionDays = 1 + wrk.Unlock() + + // check partitions + time.Sleep(time.Second * 10) + require.Eventually(t, func() bool { + return validatePartitionsMatchExpected(ctx, t, sess, plTbl, []time.Time{now.AddDate(0, 0, -1), now, now.AddDate(0, 0, 1)}) + }, time.Second*10, time.Second) + require.Eventually(t, func() bool { + return validatePartitionsMatchExpected(ctx, t, sess, dlwTbl, []time.Time{now.AddDate(0, 0, -1), now, now.AddDate(0, 0, 1)}) + }, time.Second*10, time.Second) +} + +func TestCalcNextTick(t *testing.T) { + loc := time.Now().Location() + require.True(t, calcNextTick(time.Date(2024, 12, 6, 23, 59, 59, 999999999, loc)) == time.Hour*2+time.Nanosecond) + require.True(t, calcNextTick(time.Date(2024, 12, 7, 0, 0, 0, 0, loc)) == time.Hour*2) + require.True(t, calcNextTick(time.Date(2024, 12, 7, 2, 0, 0, 1, loc)) == time.Hour*24-time.Nanosecond) + require.True(t, calcNextTick(time.Date(2024, 12, 7, 1, 59, 59, 999999999, loc)) == time.Nanosecond) +} From fedc98a3a7de07f858b13000bb2f4977ec2db527 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=8E=8B=E8=B6=85?= Date: Thu, 9 Jan 2025 12:16:52 +0800 Subject: [PATCH 14/27] ttl: disable paging in TTL (#58759) close pingcap/tidb#58342 --- pkg/ttl/ttlworker/scan.go | 13 +--- pkg/ttl/ttlworker/session.go | 30 +++++++++ pkg/ttl/ttlworker/session_integration_test.go | 63 +++++++++++++++++++ 3 files changed, 96 insertions(+), 10 deletions(-) diff --git a/pkg/ttl/ttlworker/scan.go b/pkg/ttl/ttlworker/scan.go index 2b736f4ac9051..c9e67e10b4cf0 100644 --- a/pkg/ttl/ttlworker/scan.go +++ b/pkg/ttl/ttlworker/scan.go @@ -17,13 +17,11 @@ package ttlworker import ( "context" "fmt" - "strconv" "sync/atomic" "time" "github.com/pingcap/errors" "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/ttl/cache" "github.com/pingcap/tidb/pkg/ttl/metrics" @@ -199,17 +197,12 @@ func (t *ttlScanTask) doScan(ctx context.Context, delCh chan<- *ttlDeleteTask, s )) } - origConcurrency := rawSess.GetSessionVars().DistSQLScanConcurrency() - if _, err = rawSess.ExecuteSQL(ctx, "set @@tidb_distsql_scan_concurrency=1"); err != nil { + sess, restoreSession, err := NewScanSession(rawSess, t.tbl, t.ExpireTime) + if err != nil { return t.result(err) } + defer restoreSession() - defer func() { - _, err = rawSess.ExecuteSQL(ctx, "set @@tidb_distsql_scan_concurrency="+strconv.Itoa(origConcurrency)) - terror.Log(err) - }() - - sess := newTableSession(rawSess, t.tbl, t.ExpireTime) generator, err := sqlbuilder.NewScanQueryGenerator(t.tbl, t.ExpireTime, t.ScanRangeStart, t.ScanRangeEnd) if err != nil { return t.result(err) diff --git a/pkg/ttl/ttlworker/session.go b/pkg/ttl/ttlworker/session.go index 9b294193df8ec..db0be69ae958a 100644 --- a/pkg/ttl/ttlworker/session.go +++ b/pkg/ttl/ttlworker/session.go @@ -208,6 +208,36 @@ func newTableSession(se session.Session, tbl *cache.PhysicalTable, expire time.T } } +// NewScanSession creates a session for scan +func NewScanSession(se session.Session, tbl *cache.PhysicalTable, expire time.Time) (*ttlTableSession, func(), error) { + origConcurrency := se.GetSessionVars().DistSQLScanConcurrency() + origPaging := se.GetSessionVars().EnablePaging + + restore := func() { + _, err := se.ExecuteSQL(context.Background(), "set @@tidb_distsql_scan_concurrency=%?", origConcurrency) + terror.Log(err) + _, err = se.ExecuteSQL(context.Background(), "set @@tidb_enable_paging=%?", origPaging) + terror.Log(err) + } + + // Set the distsql scan concurrency to 1 to reduce the number of cop tasks in TTL scan. + if _, err := se.ExecuteSQL(context.Background(), "set @@tidb_distsql_scan_concurrency=1"); err != nil { + restore() + return nil, nil, err + } + + // Disable tidb_enable_paging because we have already had a `LIMIT` in the SQL to limit the result set. + // If `tidb_enable_paging` is enabled, it may have multiple cop tasks even in one region that makes some extra + // processed keys in TiKV side, see issue: https://github.com/pingcap/tidb/issues/58342. + // Disable it to make the scan more efficient. + if _, err := se.ExecuteSQL(context.Background(), "set @@tidb_enable_paging=OFF"); err != nil { + restore() + return nil, nil, err + } + + return newTableSession(se, tbl, expire), restore, nil +} + type ttlTableSession struct { session.Session tbl *cache.PhysicalTable diff --git a/pkg/ttl/ttlworker/session_integration_test.go b/pkg/ttl/ttlworker/session_integration_test.go index 2d0a7b91f523d..0527de9ba8ca4 100644 --- a/pkg/ttl/ttlworker/session_integration_test.go +++ b/pkg/ttl/ttlworker/session_integration_test.go @@ -20,11 +20,13 @@ import ( "strings" "sync/atomic" "testing" + "time" "github.com/ngaut/pools" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/testkit" + "github.com/pingcap/tidb/pkg/ttl/cache" "github.com/pingcap/tidb/pkg/ttl/ttlworker" "github.com/pingcap/tidb/pkg/util" "github.com/pingcap/tidb/pkg/util/logutil" @@ -185,3 +187,64 @@ func TestGetSessionWithFault(t *testing.T) { require.True(t, se != nil || err != nil) } } + +func TestNewScanSession(t *testing.T) { + _, dom := testkit.CreateMockStoreAndDomain(t) + pool := newFaultSessionPool(dom.SysSessionPool()) + pool.setFault(newFaultWithFilter(func(s string) bool { return false }, newFaultAfterCount(0))) + se, err := ttlworker.GetSessionForTest(pool) + require.NoError(t, err) + + _, err = se.ExecuteSQL(context.Background(), "set @@tidb_distsql_scan_concurrency=123") + require.NoError(t, err) + require.Equal(t, 123, se.GetSessionVars().DistSQLScanConcurrency()) + + _, err = se.ExecuteSQL(context.Background(), "set @@tidb_enable_paging=ON") + require.NoError(t, err) + require.True(t, se.GetSessionVars().EnablePaging) + + for _, errSQL := range []string{ + "", + "set @@tidb_distsql_scan_concurrency=1", + "set @@tidb_enable_paging=OFF", + } { + t.Run("test err in SQL: "+errSQL, func(t *testing.T) { + var faultCnt atomic.Int64 + pool.setFault(newFaultWithFilter(func(s string) bool { + if s == errSQL && s != "" { + faultCnt.Add(1) + return true + } + return false + }, newFaultAfterCount(0))) + tblSe, restore, err := ttlworker.NewScanSession(se, &cache.PhysicalTable{}, time.Now()) + if errSQL == "" { + // success case + require.NoError(t, err) + require.NotNil(t, tblSe) + require.NotNil(t, restore) + require.Same(t, se, tblSe.Session) + require.Equal(t, int64(0), faultCnt.Load()) + + // NewScanSession should override @@dist_sql_scan_concurrency and @@tidb_enable_paging + require.Equal(t, 1, se.GetSessionVars().DistSQLScanConcurrency()) + require.False(t, se.GetSessionVars().EnablePaging) + + // restore should restore the session variables + restore() + require.Equal(t, 123, se.GetSessionVars().DistSQLScanConcurrency()) + require.True(t, se.GetSessionVars().EnablePaging) + } else { + // error case + require.Equal(t, int64(1), faultCnt.Load()) + require.EqualError(t, err, "fault in test") + require.Nil(t, tblSe) + require.Nil(t, restore) + + // NewScanSession should not change session state if error occurs + require.Equal(t, 123, se.GetSessionVars().DistSQLScanConcurrency()) + require.True(t, se.GetSessionVars().EnablePaging) + } + }) + } +} From 9d0fb303ede00ea1f4a5bc02f449dd1bbf379020 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Thu, 9 Jan 2025 13:05:07 +0800 Subject: [PATCH 15/27] statistics: stats cache set default quota as 20% (#58013) close pingcap/tidb#58014 --- pkg/statistics/handle/cache/internal/lfu/lfu_cache.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pkg/statistics/handle/cache/internal/lfu/lfu_cache.go b/pkg/statistics/handle/cache/internal/lfu/lfu_cache.go index 4a16300e4b8c4..078a76e636a78 100644 --- a/pkg/statistics/handle/cache/internal/lfu/lfu_cache.go +++ b/pkg/statistics/handle/cache/internal/lfu/lfu_cache.go @@ -75,14 +75,14 @@ func NewLFU(totalMemCost int64) (*LFU, error) { } // adjustMemCost adjusts the memory cost according to the total memory cost. -// When the total memory cost is 0, the memory cost is set to half of the total memory. +// When the total memory cost is 0, the memory cost is set to 20% of the total memory. func adjustMemCost(totalMemCost int64) (result int64, err error) { if totalMemCost == 0 { memTotal, err := memory.MemTotal() if err != nil { return 0, err } - return int64(memTotal / 2), nil + return int64(memTotal * 20 / 100), nil } return totalMemCost, nil } From b6141ec589ed8e73176692dc982a210ad7cf070b Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Thu, 9 Jan 2025 14:03:19 +0800 Subject: [PATCH 16/27] executor: fix hash join analyze statistic when spill is triggered (#58669) close pingcap/tidb#58571 --- pkg/executor/join/BUILD.bazel | 1 + pkg/executor/join/hash_join_base.go | 100 ----------- pkg/executor/join/hash_join_stats.go | 248 +++++++++++++++++++++++++++ pkg/executor/join/hash_join_v1.go | 2 +- pkg/executor/join/hash_join_v2.go | 155 ++++------------- pkg/util/execdetails/execdetails.go | 2 + 6 files changed, 283 insertions(+), 225 deletions(-) create mode 100644 pkg/executor/join/hash_join_stats.go diff --git a/pkg/executor/join/BUILD.bazel b/pkg/executor/join/BUILD.bazel index 913820ff6bfd8..be8d78d7a1bea 100644 --- a/pkg/executor/join/BUILD.bazel +++ b/pkg/executor/join/BUILD.bazel @@ -10,6 +10,7 @@ go_library( "hash_join_base.go", "hash_join_spill.go", "hash_join_spill_helper.go", + "hash_join_stats.go", "hash_join_test_util.go", "hash_join_v1.go", "hash_join_v2.go", diff --git a/pkg/executor/join/hash_join_base.go b/pkg/executor/join/hash_join_base.go index 24718af2da760..20e66fa7d0f70 100644 --- a/pkg/executor/join/hash_join_base.go +++ b/pkg/executor/join/hash_join_base.go @@ -15,10 +15,7 @@ package join import ( - "bytes" "context" - "fmt" - "strconv" "sync" "sync/atomic" "time" @@ -31,7 +28,6 @@ import ( "github.com/pingcap/tidb/pkg/util" "github.com/pingcap/tidb/pkg/util/chunk" "github.com/pingcap/tidb/pkg/util/disk" - "github.com/pingcap/tidb/pkg/util/execdetails" "github.com/pingcap/tidb/pkg/util/memory" ) @@ -370,99 +366,3 @@ type probeChkResource struct { chk *chunk.Chunk dest chan<- *chunk.Chunk } - -type hashJoinRuntimeStats struct { - fetchAndBuildStartTime time.Time - fetchAndBuildHashTable time.Duration - hashStat hashStatistic - fetchAndProbe int64 - probe int64 - concurrent int - maxFetchAndProbe int64 -} - -func (e *hashJoinRuntimeStats) setMaxFetchAndProbeTime(t int64) { - for { - value := atomic.LoadInt64(&e.maxFetchAndProbe) - if t <= value { - return - } - if atomic.CompareAndSwapInt64(&e.maxFetchAndProbe, value, t) { - return - } - } -} - -// Tp implements the RuntimeStats interface. -func (*hashJoinRuntimeStats) Tp() int { - return execdetails.TpHashJoinRuntimeStats -} - -func (e *hashJoinRuntimeStats) String() string { - buf := bytes.NewBuffer(make([]byte, 0, 128)) - if e.fetchAndBuildHashTable > 0 { - buf.WriteString("build_hash_table:{total:") - buf.WriteString(execdetails.FormatDuration(e.fetchAndBuildHashTable)) - buf.WriteString(", fetch:") - buf.WriteString(execdetails.FormatDuration(e.fetchAndBuildHashTable - e.hashStat.buildTableElapse)) - buf.WriteString(", build:") - buf.WriteString(execdetails.FormatDuration(e.hashStat.buildTableElapse)) - buf.WriteString("}") - } - if e.probe > 0 { - buf.WriteString(", probe:{concurrency:") - buf.WriteString(strconv.Itoa(e.concurrent)) - buf.WriteString(", total:") - buf.WriteString(execdetails.FormatDuration(time.Duration(e.fetchAndProbe))) - buf.WriteString(", max:") - buf.WriteString(execdetails.FormatDuration(time.Duration(atomic.LoadInt64(&e.maxFetchAndProbe)))) - buf.WriteString(", probe:") - buf.WriteString(execdetails.FormatDuration(time.Duration(e.probe))) - // fetch time is the time wait fetch result from its child executor, - // wait time is the time wait its parent executor to fetch the joined result - buf.WriteString(", fetch and wait:") - buf.WriteString(execdetails.FormatDuration(time.Duration(e.fetchAndProbe - e.probe))) - if e.hashStat.probeCollision > 0 { - buf.WriteString(", probe_collision:") - buf.WriteString(strconv.FormatInt(e.hashStat.probeCollision, 10)) - } - buf.WriteString("}") - } - return buf.String() -} - -func (e *hashJoinRuntimeStats) Clone() execdetails.RuntimeStats { - return &hashJoinRuntimeStats{ - fetchAndBuildHashTable: e.fetchAndBuildHashTable, - hashStat: e.hashStat, - fetchAndProbe: e.fetchAndProbe, - probe: e.probe, - concurrent: e.concurrent, - maxFetchAndProbe: e.maxFetchAndProbe, - } -} - -func (e *hashJoinRuntimeStats) Merge(rs execdetails.RuntimeStats) { - tmp, ok := rs.(*hashJoinRuntimeStats) - if !ok { - return - } - e.fetchAndBuildHashTable += tmp.fetchAndBuildHashTable - e.hashStat.buildTableElapse += tmp.hashStat.buildTableElapse - e.hashStat.probeCollision += tmp.hashStat.probeCollision - e.fetchAndProbe += tmp.fetchAndProbe - e.probe += tmp.probe - if e.maxFetchAndProbe < tmp.maxFetchAndProbe { - e.maxFetchAndProbe = tmp.maxFetchAndProbe - } -} - -type hashStatistic struct { - // NOTE: probeCollision may be accessed from multiple goroutines concurrently. - probeCollision int64 - buildTableElapse time.Duration -} - -func (s *hashStatistic) String() string { - return fmt.Sprintf("probe_collision:%v, build:%v", s.probeCollision, execdetails.FormatDuration(s.buildTableElapse)) -} diff --git a/pkg/executor/join/hash_join_stats.go b/pkg/executor/join/hash_join_stats.go new file mode 100644 index 0000000000000..d4b61e793ac79 --- /dev/null +++ b/pkg/executor/join/hash_join_stats.go @@ -0,0 +1,248 @@ +// 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 join + +import ( + "bytes" + "fmt" + "strconv" + "sync/atomic" + "time" + + "github.com/pingcap/tidb/pkg/util/execdetails" +) + +type hashJoinRuntimeStats struct { + fetchAndBuildHashTable time.Duration + hashStat hashStatistic + fetchAndProbe int64 + probe int64 + concurrent int + maxFetchAndProbe int64 +} + +// Tp implements the RuntimeStats interface. +func (*hashJoinRuntimeStats) Tp() int { + return execdetails.TpHashJoinRuntimeStats +} + +func (e *hashJoinRuntimeStats) String() string { + buf := bytes.NewBuffer(make([]byte, 0, 128)) + if e.fetchAndBuildHashTable > 0 { + buf.WriteString("build_hash_table:{total:") + buf.WriteString(execdetails.FormatDuration(e.fetchAndBuildHashTable)) + buf.WriteString(", fetch:") + buf.WriteString(execdetails.FormatDuration(e.fetchAndBuildHashTable - e.hashStat.buildTableElapse)) + buf.WriteString(", build:") + buf.WriteString(execdetails.FormatDuration(e.hashStat.buildTableElapse)) + buf.WriteString("}") + } + if e.probe > 0 { + buf.WriteString(", probe:{concurrency:") + buf.WriteString(strconv.Itoa(e.concurrent)) + buf.WriteString(", total:") + buf.WriteString(execdetails.FormatDuration(time.Duration(e.fetchAndProbe))) + buf.WriteString(", max:") + buf.WriteString(execdetails.FormatDuration(time.Duration(atomic.LoadInt64(&e.maxFetchAndProbe)))) + buf.WriteString(", probe:") + buf.WriteString(execdetails.FormatDuration(time.Duration(e.probe))) + // fetch time is the time wait fetch result from its child executor, + // wait time is the time wait its parent executor to fetch the joined result + buf.WriteString(", fetch and wait:") + buf.WriteString(execdetails.FormatDuration(time.Duration(e.fetchAndProbe - e.probe))) + if e.hashStat.probeCollision > 0 { + buf.WriteString(", probe_collision:") + buf.WriteString(strconv.FormatInt(e.hashStat.probeCollision, 10)) + } + buf.WriteString("}") + } + return buf.String() +} + +func (e *hashJoinRuntimeStats) Clone() execdetails.RuntimeStats { + return &hashJoinRuntimeStats{ + fetchAndBuildHashTable: e.fetchAndBuildHashTable, + hashStat: e.hashStat, + fetchAndProbe: e.fetchAndProbe, + probe: e.probe, + concurrent: e.concurrent, + maxFetchAndProbe: e.maxFetchAndProbe, + } +} + +func (e *hashJoinRuntimeStats) Merge(rs execdetails.RuntimeStats) { + tmp, ok := rs.(*hashJoinRuntimeStats) + if !ok { + return + } + e.fetchAndBuildHashTable += tmp.fetchAndBuildHashTable + e.hashStat.buildTableElapse += tmp.hashStat.buildTableElapse + e.hashStat.probeCollision += tmp.hashStat.probeCollision + e.fetchAndProbe += tmp.fetchAndProbe + e.probe += tmp.probe + if e.maxFetchAndProbe < tmp.maxFetchAndProbe { + e.maxFetchAndProbe = tmp.maxFetchAndProbe + } +} + +type hashStatistic struct { + // NOTE: probeCollision may be accessed from multiple goroutines concurrently. + probeCollision int64 + buildTableElapse time.Duration +} + +func (s *hashStatistic) String() string { + return fmt.Sprintf("probe_collision:%v, build:%v", s.probeCollision, execdetails.FormatDuration(s.buildTableElapse)) +} + +type hashJoinRuntimeStatsV2 struct { + concurrent int + probeCollision int64 + + fetchAndBuildHashTable int64 + + partitionData int64 + buildHashTable int64 + probe int64 + fetchAndProbe int64 + + maxPartitionData int64 + maxBuildHashTable int64 + maxProbe int64 + maxFetchAndProbe int64 + + maxPartitionDataForCurrentRound int64 + maxBuildHashTableForCurrentRound int64 + maxProbeForCurrentRound int64 + maxFetchAndProbeForCurrentRound int64 +} + +func setMaxValue(addr *int64, currentValue int64) { + for { + value := atomic.LoadInt64(addr) + if currentValue <= value { + return + } + if atomic.CompareAndSwapInt64(addr, value, currentValue) { + return + } + } +} + +func (e *hashJoinRuntimeStatsV2) reset() { + e.probeCollision = 0 + e.fetchAndBuildHashTable = 0 + e.partitionData = 0 + e.buildHashTable = 0 + e.probe = 0 + e.fetchAndProbe = 0 + e.maxPartitionData = 0 + e.maxBuildHashTable = 0 + e.maxProbe = 0 + e.maxFetchAndProbe = 0 + e.maxPartitionDataForCurrentRound = 0 + e.maxBuildHashTableForCurrentRound = 0 + e.maxProbeForCurrentRound = 0 + e.maxFetchAndProbeForCurrentRound = 0 +} + +func (e *hashJoinRuntimeStatsV2) resetCurrentRound() { + e.maxPartitionData += e.maxPartitionDataForCurrentRound + e.maxBuildHashTable += e.maxBuildHashTableForCurrentRound + e.maxProbe += e.maxProbeForCurrentRound + e.maxFetchAndProbe += e.maxFetchAndProbeForCurrentRound + e.maxPartitionDataForCurrentRound = 0 + e.maxBuildHashTableForCurrentRound = 0 + e.maxProbeForCurrentRound = 0 + e.maxFetchAndProbeForCurrentRound = 0 +} + +// Tp implements the RuntimeStats interface. +func (*hashJoinRuntimeStatsV2) Tp() int { + return execdetails.TpHashJoinRuntimeStatsV2 +} + +func (e *hashJoinRuntimeStatsV2) String() string { + buf := bytes.NewBuffer(make([]byte, 0, 128)) + if e.fetchAndBuildHashTable > 0 { + buf.WriteString("build_hash_table:{total:") + buf.WriteString(execdetails.FormatDuration(time.Duration(e.fetchAndBuildHashTable))) + buf.WriteString(", fetch:") + buf.WriteString(execdetails.FormatDuration(time.Duration(e.fetchAndBuildHashTable - e.maxBuildHashTable - e.maxPartitionData))) + buf.WriteString(", build:") + buf.WriteString(execdetails.FormatDuration(time.Duration(e.maxBuildHashTable + e.maxPartitionData))) + buf.WriteString("}") + } + if e.probe > 0 { + buf.WriteString(", probe:{concurrency:") + buf.WriteString(strconv.Itoa(e.concurrent)) + buf.WriteString(", total:") + buf.WriteString(execdetails.FormatDuration(time.Duration(e.fetchAndProbe))) + buf.WriteString(", max:") + buf.WriteString(execdetails.FormatDuration(time.Duration(atomic.LoadInt64(&e.maxFetchAndProbe)))) + buf.WriteString(", probe:") + buf.WriteString(execdetails.FormatDuration(time.Duration(e.maxProbe))) + buf.WriteString(", fetch and wait:") + buf.WriteString(execdetails.FormatDuration(time.Duration(e.fetchAndProbe - e.maxProbe))) + if e.probeCollision > 0 { + buf.WriteString(", probe_collision:") + buf.WriteString(strconv.FormatInt(e.probeCollision, 10)) + } + buf.WriteString("}") + } + return buf.String() +} + +func (e *hashJoinRuntimeStatsV2) Clone() execdetails.RuntimeStats { + return &hashJoinRuntimeStatsV2{ + concurrent: e.concurrent, + probeCollision: e.probeCollision, + fetchAndBuildHashTable: e.fetchAndBuildHashTable, + partitionData: e.partitionData, + buildHashTable: e.buildHashTable, + probe: e.probe, + fetchAndProbe: e.fetchAndProbe, + maxPartitionData: e.maxPartitionData, + maxBuildHashTable: e.maxBuildHashTable, + maxProbe: e.maxProbe, + maxFetchAndProbe: e.maxFetchAndProbe, + maxPartitionDataForCurrentRound: e.maxPartitionDataForCurrentRound, + maxBuildHashTableForCurrentRound: e.maxBuildHashTableForCurrentRound, + maxProbeForCurrentRound: e.maxProbeForCurrentRound, + maxFetchAndProbeForCurrentRound: e.maxFetchAndProbeForCurrentRound, + } +} + +func (e *hashJoinRuntimeStatsV2) Merge(rs execdetails.RuntimeStats) { + tmp, ok := rs.(*hashJoinRuntimeStatsV2) + if !ok { + return + } + e.fetchAndBuildHashTable += tmp.fetchAndBuildHashTable + e.buildHashTable += tmp.buildHashTable + if e.maxBuildHashTable < tmp.maxBuildHashTable { + e.maxBuildHashTable = tmp.maxBuildHashTable + } + e.partitionData += tmp.partitionData + if e.maxPartitionData < tmp.maxPartitionData { + e.maxPartitionData = tmp.maxPartitionData + } + e.probeCollision += tmp.probeCollision + e.fetchAndProbe += tmp.fetchAndProbe + e.probe += tmp.probe + if e.maxFetchAndProbe < tmp.maxFetchAndProbe { + e.maxFetchAndProbe = tmp.maxFetchAndProbe + } +} diff --git a/pkg/executor/join/hash_join_v1.go b/pkg/executor/join/hash_join_v1.go index 1385784fee4fa..69a5bd9b4b268 100644 --- a/pkg/executor/join/hash_join_v1.go +++ b/pkg/executor/join/hash_join_v1.go @@ -306,7 +306,7 @@ func (w *ProbeWorkerV1) runJoinWorker() { t := time.Since(start) atomic.AddInt64(&w.HashJoinCtx.stats.probe, probeTime) atomic.AddInt64(&w.HashJoinCtx.stats.fetchAndProbe, int64(t)) - w.HashJoinCtx.stats.setMaxFetchAndProbeTime(int64(t)) + setMaxValue(&w.HashJoinCtx.stats.maxFetchAndProbe, int64(t)) }() } diff --git a/pkg/executor/join/hash_join_v2.go b/pkg/executor/join/hash_join_v2.go index 698a8405ff7de..56a067b4bcbc5 100644 --- a/pkg/executor/join/hash_join_v2.go +++ b/pkg/executor/join/hash_join_v2.go @@ -15,14 +15,12 @@ package join import ( - "bytes" "context" "hash" "math" "math/bits" "math/rand" "runtime/trace" - "strconv" "sync" "sync/atomic" "time" @@ -41,7 +39,6 @@ import ( "github.com/pingcap/tidb/pkg/util/channel" "github.com/pingcap/tidb/pkg/util/chunk" "github.com/pingcap/tidb/pkg/util/disk" - "github.com/pingcap/tidb/pkg/util/execdetails" "github.com/pingcap/tidb/pkg/util/intest" "github.com/pingcap/tidb/pkg/util/memory" ) @@ -359,14 +356,13 @@ type ProbeWorkerV2 struct { func (w *ProbeWorkerV2) updateProbeStatistic(start time.Time, probeTime int64) { t := time.Since(start) atomic.AddInt64(&w.HashJoinCtx.stats.probe, probeTime) - atomic.AddInt64(&w.HashJoinCtx.stats.fetchAndProbe, int64(t)) - setMaxValue(&w.HashJoinCtx.stats.maxFetchAndProbe, int64(t)) + setMaxValue(&w.HashJoinCtx.stats.maxProbeForCurrentRound, probeTime) + setMaxValue(&w.HashJoinCtx.stats.maxFetchAndProbeForCurrentRound, int64(t)) } -func (w *ProbeWorkerV2) restoreAndProbe(inDisk *chunk.DataInDiskByChunks) { +func (w *ProbeWorkerV2) restoreAndProbe(inDisk *chunk.DataInDiskByChunks, start time.Time) { probeTime := int64(0) if w.HashJoinCtx.stats != nil { - start := time.Now() defer func() { w.updateProbeStatistic(start, probeTime) }() @@ -441,7 +437,7 @@ func (b *BuildWorkerV2) clearSegmentsInRowTable(partID int) { func (b *BuildWorkerV2) updatePartitionData(cost int64) { atomic.AddInt64(&b.HashJoinCtx.stats.partitionData, cost) - setMaxValue(&b.HashJoinCtx.stats.maxPartitionData, cost) + setMaxValue(&b.HashJoinCtx.stats.maxPartitionDataForCurrentRound, cost) } func (b *BuildWorkerV2) processOneRestoredChunk(cost *int64) error { @@ -527,8 +523,7 @@ func (b *BuildWorkerV2) splitPartitionAndAppendToRowTable(typeCtx types.Context, cost := int64(0) defer func() { if b.HashJoinCtx.stats != nil { - atomic.AddInt64(&b.HashJoinCtx.stats.partitionData, cost) - setMaxValue(&b.HashJoinCtx.stats.maxPartitionData, cost) + b.updatePartitionData(cost) } }() @@ -585,7 +580,7 @@ func (b *BuildWorkerV2) buildHashTable(taskCh chan *buildTask) error { defer func() { if b.HashJoinCtx.stats != nil { atomic.AddInt64(&b.HashJoinCtx.stats.buildHashTable, cost) - setMaxValue(&b.HashJoinCtx.stats.maxBuildHashTable, cost) + setMaxValue(&b.HashJoinCtx.stats.maxBuildHashTableForCurrentRound, cost) } }() for task := range taskCh { @@ -749,6 +744,10 @@ func (e *HashJoinV2Exec) Open(ctx context.Context) error { e.stats = &hashJoinRuntimeStatsV2{} e.stats.concurrent = int(e.Concurrency) } + + if e.stats != nil { + e.stats.reset() + } return nil } @@ -809,6 +808,11 @@ func (e *HashJoinV2Exec) startProbeFetcher(ctx context.Context) { } func (e *HashJoinV2Exec) startProbeJoinWorkers(ctx context.Context) { + var start time.Time + if e.HashJoinCtxV2.stats != nil { + start = time.Now() + } + if e.inRestore { // Wait for the restore build err := <-e.buildFinished @@ -822,22 +826,26 @@ func (e *HashJoinV2Exec) startProbeJoinWorkers(ctx context.Context) { e.workerWg.RunWithRecover(func() { defer trace.StartRegion(ctx, "HashJoinWorker").End() if e.inRestore { - e.ProbeWorkers[workerID].restoreAndProbe(e.restoredProbeInDisk[workerID]) + e.ProbeWorkers[workerID].restoreAndProbe(e.restoredProbeInDisk[workerID], start) } else { - e.ProbeWorkers[workerID].runJoinWorker() + e.ProbeWorkers[workerID].runJoinWorker(start) } }, e.ProbeWorkers[workerID].handleProbeWorkerPanic) } } func (e *HashJoinV2Exec) fetchAndProbeHashTable(ctx context.Context) { + start := time.Now() e.startProbeFetcher(ctx) // Join workers directly read data from disk when we are in restore status // and read data from fetcher otherwise. e.startProbeJoinWorkers(ctx) - e.waiterWg.RunWithRecover(e.waitJoinWorkers, nil) + e.waiterWg.RunWithRecover( + func() { + e.waitJoinWorkers(start) + }, nil) } func (w *ProbeWorkerV2) handleProbeWorkerPanic(r any) { @@ -852,11 +860,12 @@ func (e *HashJoinV2Exec) handleJoinWorkerPanic(r any) { } } -func (e *HashJoinV2Exec) waitJoinWorkers() { +func (e *HashJoinV2Exec) waitJoinWorkers(start time.Time) { e.workerWg.Wait() if e.stats != nil { + e.HashJoinCtxV2.stats.fetchAndProbe += int64(time.Since(start)) for _, prober := range e.ProbeWorkers { - e.stats.hashStat.probeCollision += int64(prober.JoinProbe.GetProbeCollision()) + e.stats.probeCollision += int64(prober.JoinProbe.GetProbeCollision()) } } @@ -952,15 +961,11 @@ func (w *ProbeWorkerV2) probeAndSendResult(joinResult *hashjoinWorkerResult) (bo return true, waitTime, joinResult } -func (w *ProbeWorkerV2) runJoinWorker() { +func (w *ProbeWorkerV2) runJoinWorker(start time.Time) { probeTime := int64(0) if w.HashJoinCtx.stats != nil { - start := time.Now() defer func() { - t := time.Since(start) - atomic.AddInt64(&w.HashJoinCtx.stats.probe, probeTime) - atomic.AddInt64(&w.HashJoinCtx.stats.fetchAndProbe, int64(t)) - setMaxValue(&w.HashJoinCtx.stats.maxFetchAndProbe, int64(t)) + w.updateProbeStatistic(start, probeTime) }() } @@ -1037,6 +1042,9 @@ func (e *HashJoinV2Exec) reset() { e.releaseDisk() e.resetHashTableContextForRestore() e.spillHelper.setCanSpillFlag(true) + if e.HashJoinCtxV2.stats != nil { + e.HashJoinCtxV2.stats.resetCurrentRound() + } } func (e *HashJoinV2Exec) startBuildAndProbe(ctx context.Context) { @@ -1228,7 +1236,7 @@ func (e *HashJoinV2Exec) fetchAndBuildHashTableImpl(ctx context.Context) { if e.stats != nil { start := time.Now() defer func() { - e.stats.fetchAndBuildHashTable = time.Since(start) + e.stats.fetchAndBuildHashTable += int64(time.Since(start)) }() } @@ -1342,10 +1350,6 @@ func (e *HashJoinV2Exec) controlWorkersForRestore(chunkNum int, syncCh chan *chu close(waitForController) }() - if e.stats != nil { - e.stats.fetchAndBuildStartTime = time.Now() - } - for i := 0; i < chunkNum; i++ { if e.finished.Load() { return @@ -1449,103 +1453,6 @@ type buildTask struct { segEndIdx int } -type hashJoinRuntimeStatsV2 struct { - hashJoinRuntimeStats - partitionData int64 - maxPartitionData int64 - buildHashTable int64 - maxBuildHashTable int64 -} - -func setMaxValue(addr *int64, currentValue int64) { - for { - value := atomic.LoadInt64(addr) - if currentValue <= value { - return - } - if atomic.CompareAndSwapInt64(addr, value, currentValue) { - return - } - } -} - -// Tp implements the RuntimeStats interface. -func (*hashJoinRuntimeStatsV2) Tp() int { - return execdetails.TpHashJoinRuntimeStats -} - -func (e *hashJoinRuntimeStatsV2) String() string { - buf := bytes.NewBuffer(make([]byte, 0, 128)) - if e.fetchAndBuildHashTable > 0 { - buf.WriteString("build_hash_table:{total:") - buf.WriteString(execdetails.FormatDuration(e.fetchAndBuildHashTable)) - buf.WriteString(", fetch:") - buf.WriteString(execdetails.FormatDuration(time.Duration(int64(e.fetchAndBuildHashTable) - e.maxBuildHashTable - e.maxPartitionData))) - buf.WriteString(", build:") - buf.WriteString(execdetails.FormatDuration(time.Duration(e.buildHashTable))) - buf.WriteString("}") - } - if e.probe > 0 { - buf.WriteString(", probe:{concurrency:") - buf.WriteString(strconv.Itoa(e.concurrent)) - buf.WriteString(", total:") - buf.WriteString(execdetails.FormatDuration(time.Duration(e.fetchAndProbe))) - buf.WriteString(", max:") - buf.WriteString(execdetails.FormatDuration(time.Duration(atomic.LoadInt64(&e.maxFetchAndProbe)))) - buf.WriteString(", probe:") - buf.WriteString(execdetails.FormatDuration(time.Duration(e.probe))) - buf.WriteString(", fetch and wait:") - buf.WriteString(execdetails.FormatDuration(time.Duration(e.fetchAndProbe - e.probe))) - if e.hashStat.probeCollision > 0 { - buf.WriteString(", probe_collision:") - buf.WriteString(strconv.FormatInt(e.hashStat.probeCollision, 10)) - } - buf.WriteString("}") - } - return buf.String() -} - -func (e *hashJoinRuntimeStatsV2) Clone() execdetails.RuntimeStats { - stats := hashJoinRuntimeStats{ - fetchAndBuildHashTable: e.fetchAndBuildHashTable, - hashStat: e.hashStat, - fetchAndProbe: e.fetchAndProbe, - probe: e.probe, - concurrent: e.concurrent, - maxFetchAndProbe: e.maxFetchAndProbe, - } - return &hashJoinRuntimeStatsV2{ - hashJoinRuntimeStats: stats, - partitionData: e.partitionData, - maxPartitionData: e.maxPartitionData, - buildHashTable: e.buildHashTable, - maxBuildHashTable: e.maxBuildHashTable, - } -} - -func (e *hashJoinRuntimeStatsV2) Merge(rs execdetails.RuntimeStats) { - tmp, ok := rs.(*hashJoinRuntimeStatsV2) - if !ok { - return - } - e.fetchAndBuildHashTable += tmp.fetchAndBuildHashTable - e.buildHashTable += tmp.buildHashTable - if e.maxBuildHashTable < tmp.maxBuildHashTable { - e.maxBuildHashTable = tmp.maxBuildHashTable - } - e.partitionData += tmp.partitionData - if e.maxPartitionData < tmp.maxPartitionData { - e.maxPartitionData = tmp.maxPartitionData - } - e.hashStat.buildTableElapse += tmp.hashStat.buildTableElapse - e.hashStat.probeCollision += tmp.hashStat.probeCollision - e.fetchAndProbe += tmp.fetchAndProbe - e.probe += tmp.probe - if e.maxFetchAndProbe < tmp.maxFetchAndProbe { - e.maxFetchAndProbe = tmp.maxFetchAndProbe - } -} - func generatePartitionIndex(hashValue uint64, partitionMaskOffset int) uint64 { return hashValue >> uint64(partitionMaskOffset) } diff --git a/pkg/util/execdetails/execdetails.go b/pkg/util/execdetails/execdetails.go index d3da6e905d696..b4553ce2d13f9 100644 --- a/pkg/util/execdetails/execdetails.go +++ b/pkg/util/execdetails/execdetails.go @@ -1001,6 +1001,8 @@ const ( TpSnapshotRuntimeStats // TpHashJoinRuntimeStats is the tp for HashJoinRuntimeStats. TpHashJoinRuntimeStats + // TpHashJoinRuntimeStatsV2 is the tp for hashJoinRuntimeStatsV2. + TpHashJoinRuntimeStatsV2 // TpIndexLookUpJoinRuntimeStats is the tp for IndexLookUpJoinRuntimeStats. TpIndexLookUpJoinRuntimeStats // TpRuntimeStatsWithSnapshot is the tp for RuntimeStatsWithSnapshot. From 448e302b39bd204855aa894abbc4776425f2b06f Mon Sep 17 00:00:00 2001 From: Wenxuan Date: Thu, 9 Jan 2025 16:31:19 +0900 Subject: [PATCH 17/27] ddl: Fix vector index for high dimensional vectors (#58717) ref pingcap/tidb#54245 --- pkg/ddl/index.go | 8 ++ pkg/expression/integration_test/BUILD.bazel | 1 + .../integration_test/integration_test.go | 96 +++++++++++++++++++ pkg/meta/model/index.go | 6 ++ pkg/table/tables/tables.go | 15 +++ 5 files changed, 126 insertions(+) diff --git a/pkg/ddl/index.go b/pkg/ddl/index.go index 3c92e525a7df1..56e6021f86e76 100644 --- a/pkg/ddl/index.go +++ b/pkg/ddl/index.go @@ -285,6 +285,11 @@ func getIndexColumnLength(col *model.ColumnInfo, colLen int) (int, error) { } switch col.GetType() { + case mysql.TypeTiDBVectorFloat32: + // Vector Index does not actually create KV index, so it has length of 0. + // however 0 may cause some issues in other calculations, so we use 1 here. + // 1 is also minimal enough anyway. + return 1, nil case mysql.TypeBit: return (length + 7) >> 3, nil case mysql.TypeVarchar, mysql.TypeString, mysql.TypeVarString, mysql.TypeTinyBlob, mysql.TypeMediumBlob, mysql.TypeBlob, mysql.TypeLongBlob: @@ -2930,6 +2935,9 @@ func newCleanUpIndexWorker(id int, t table.PhysicalTable, decodeColMap map[int64 indexes := make([]table.Index, 0, len(t.Indices())) rowDecoder := decoder.NewRowDecoder(t, t.WritableCols(), decodeColMap) for _, index := range t.Indices() { + if index.Meta().IsTiFlashLocalIndex() { + continue + } if index.Meta().Global { indexes = append(indexes, index) } diff --git a/pkg/expression/integration_test/BUILD.bazel b/pkg/expression/integration_test/BUILD.bazel index 139264b797cca..e101be7024b28 100644 --- a/pkg/expression/integration_test/BUILD.bazel +++ b/pkg/expression/integration_test/BUILD.bazel @@ -12,6 +12,7 @@ go_test( deps = [ "//pkg/config", "//pkg/domain", + "//pkg/domain/infosync", "//pkg/errno", "//pkg/expression", "//pkg/kv", diff --git a/pkg/expression/integration_test/integration_test.go b/pkg/expression/integration_test/integration_test.go index 941f6726e5054..f0984cc1bb2f1 100644 --- a/pkg/expression/integration_test/integration_test.go +++ b/pkg/expression/integration_test/integration_test.go @@ -33,6 +33,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb/pkg/config" "github.com/pingcap/tidb/pkg/domain" + "github.com/pingcap/tidb/pkg/domain/infosync" "github.com/pingcap/tidb/pkg/errno" "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/kv" @@ -61,6 +62,101 @@ import ( "github.com/tikv/client-go/v2/oracle" ) +func TestVectorLong(t *testing.T) { + store := testkit.CreateMockStoreWithSchemaLease(t, 1*time.Second, mockstore.WithMockTiFlash(2)) + + tk := testkit.NewTestKit(t, store) + + tiflash := infosync.NewMockTiFlash() + infosync.SetMockTiFlash(tiflash) + defer func() { + tiflash.Lock() + tiflash.StatusServer.Close() + tiflash.Unlock() + }() + + genVec := func(d int, startValue int) string { + vb := strings.Builder{} + vb.WriteString("[") + value := startValue + for i := 0; i < d; i++ { + if i > 0 { + vb.WriteString(",") + } + vb.WriteString(strconv.FormatInt(int64(value), 10)) + value += 100 + } + vb.WriteString("]") + return vb.String() + } + + failpoint.Enable("github.com/pingcap/tidb/pkg/ddl/MockCheckVectorIndexProcess", `return(1)`) + defer func() { + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/ddl/MockCheckVectorIndexProcess")) + }() + + runWorkload := func() { + tk.MustExec(fmt.Sprintf(`insert into t1 values (1, '%s')`, genVec(16383, 100))) + tk.MustQuery(`select * from t1 order by id`).Check(testkit.Rows("1 " + genVec(16383, 100))) + tk.MustExec(fmt.Sprintf(`delete from t1 where vec > '%s'`, genVec(16383, 200))) + tk.MustQuery(`select * from t1 order by id`).Check(testkit.Rows("1 " + genVec(16383, 100))) + tk.MustExec(fmt.Sprintf(`delete from t1 where vec > '%s'`, genVec(16383, 50))) + tk.MustQuery(`select * from t1 order by id`).Check(testkit.Rows()) + tk.MustExec(fmt.Sprintf(`insert into t1 values (1, '%s')`, genVec(16383, 100))) + tk.MustExec(fmt.Sprintf(`insert into t1 values (2, '%s')`, genVec(16383, 200))) + tk.MustExec(fmt.Sprintf(`insert into t1 values (3, '%s')`, genVec(16383, 300))) + tk.MustQuery(fmt.Sprintf(`select id from t1 order by vec_l2_distance(vec, '%s') limit 2`, genVec(16383, 180))).Check(testkit.Rows( + "2", + "1", + )) + tk.MustExec(fmt.Sprintf(`update t1 set vec = '%s' where id = 1`, genVec(16383, 500))) + tk.MustQuery(`select * from t1 order by id`).Check(testkit.Rows( + "1 "+genVec(16383, 500), + "2 "+genVec(16383, 200), + "3 "+genVec(16383, 300), + )) + tk.MustQuery(fmt.Sprintf(`select id from t1 order by vec_l2_distance(vec, '%s') limit 2`, genVec(16383, 180))).Check(testkit.Rows( + "2", + "3", + )) + } + + tk.MustExec("use test") + tk.MustExec(` + create table t1 ( + id int primary key, + vec vector(16383) + ) + `) + runWorkload() + tk.MustExec("drop table t1") + + tk.MustExec(` + create table t1 ( + id int primary key, + vec vector(16383), + VECTOR INDEX ((vec_cosine_distance(vec))) + ) + `) + runWorkload() + tk.MustExec("drop table if exists t1") + tk.MustExec(` + create table t1 ( + id int primary key, + vec vector(16383) + ) + `) + tk.MustExec(`alter table t1 set tiflash replica 1`) + tbl, _ := domain.GetDomain(tk.Session()).InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t1")) + tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + tk.MustExec(`alter table t1 add VECTOR INDEX ((vec_cosine_distance(vec)))`) + runWorkload() + tk.MustExec("drop table if exists t1") +} + func TestVectorDefaultValue(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) diff --git a/pkg/meta/model/index.go b/pkg/meta/model/index.go index 4ec0f3ae05c0a..30e9424c990dd 100644 --- a/pkg/meta/model/index.go +++ b/pkg/meta/model/index.go @@ -143,6 +143,12 @@ func (index *IndexInfo) IsPublic() bool { return index.State == StatePublic } +// IsTiFlashLocalIndex checks whether the index is a TiFlash local index. +// For a TiFlash local index, no actual index data need to be written to KV layer. +func (index *IndexInfo) IsTiFlashLocalIndex() bool { + return index.VectorInfo != nil +} + // FindIndexByColumns find IndexInfo in indices which is cover the specified columns. func FindIndexByColumns(tbInfo *TableInfo, indices []*IndexInfo, cols ...ast.CIStr) *IndexInfo { for _, index := range indices { diff --git a/pkg/table/tables/tables.go b/pkg/table/tables/tables.go index 9e78c0b76e722..2c1ab0995987d 100644 --- a/pkg/table/tables/tables.go +++ b/pkg/table/tables/tables.go @@ -306,6 +306,9 @@ func GetWritableIndexByName(idxName string, t table.Table) table.Index { if !IsIndexWritable(idx) { continue } + if idx.Meta().IsTiFlashLocalIndex() { + continue + } if idxName == idx.Meta().Name.L { return idx } @@ -547,6 +550,9 @@ func (t *TableCommon) rebuildUpdateRecordIndices( if t.meta.IsCommonHandle && idx.Meta().Primary { continue } + if idx.Meta().IsTiFlashLocalIndex() { + continue + } for _, ic := range idx.Meta().Columns { if !touched[ic.Offset] { continue @@ -566,6 +572,9 @@ func (t *TableCommon) rebuildUpdateRecordIndices( if !IsIndexWritable(idx) { continue } + if idx.Meta().IsTiFlashLocalIndex() { + continue + } if t.meta.IsCommonHandle && idx.Meta().Primary { continue } @@ -926,6 +935,9 @@ func (t *TableCommon) addIndices(sctx table.MutateContext, recordID kv.Handle, r if !IsIndexWritable(v) { continue } + if v.Meta().IsTiFlashLocalIndex() { + continue + } if t.meta.IsCommonHandle && v.Meta().Primary { continue } @@ -1185,6 +1197,9 @@ func (t *TableCommon) removeRowIndices(ctx table.MutateContext, txn kv.Transacti if v.Meta().Primary && (t.Meta().IsCommonHandle || t.Meta().PKIsHandle) { continue } + if v.Meta().IsTiFlashLocalIndex() { + continue + } var vals []types.Datum if opt.HasIndexesLayout() { vals, err = fetchIndexRow(v.Meta(), rec, nil, opt.GetIndexLayout(v.Meta().ID)) From 4240ce4acc90cef2ba3f624aefd5f660dc6abeb6 Mon Sep 17 00:00:00 2001 From: JaySon Date: Thu, 9 Jan 2025 17:08:37 +0800 Subject: [PATCH 18/27] ddl: Remove explicit exclude for "engine" notIn "tiflash" (#58637) close pingcap/tidb#58633 --- pkg/ddl/placement/bundle.go | 14 +++--------- pkg/ddl/placement/bundle_test.go | 34 +++++----------------------- pkg/ddl/placement/constraint.go | 1 + pkg/ddl/placement/constraint_test.go | 9 ++++++++ 4 files changed, 19 insertions(+), 39 deletions(-) diff --git a/pkg/ddl/placement/bundle.go b/pkg/ddl/placement/bundle.go index ee84ddd3c00f7..4771c3a541d11 100644 --- a/pkg/ddl/placement/bundle.go +++ b/pkg/ddl/placement/bundle.go @@ -314,6 +314,9 @@ func (b *Bundle) String() string { // Tidy will post optimize Rules, trying to generate rules that suits PD. func (b *Bundle) Tidy() error { + // refer to tidb#58633 + // Does not explicitly set exclude rule with label.key==EngineLabelKey, because the + // PD may wrongly add peer to the unexpected stores if that key is specified. tempRules := b.Rules[:0] id := 0 for _, rule := range b.Rules { @@ -321,17 +324,6 @@ func (b *Bundle) Tidy() error { if rule.Count <= 0 { continue } - // refer to tidb#22065. - // add -engine=tiflash to every rule to avoid schedules to tiflash instances. - // placement rules in SQL is not compatible with `set tiflash replica` yet - err := AddConstraint(&rule.LabelConstraints, pd.LabelConstraint{ - Op: pd.NotIn, - Key: EngineLabelKey, - Values: []string{EngineLabelTiFlash}, - }) - if err != nil { - return err - } rule.ID = strconv.Itoa(id) tempRules = append(tempRules, rule) id++ diff --git a/pkg/ddl/placement/bundle_test.go b/pkg/ddl/placement/bundle_test.go index 204ca23ccdc6c..a280bc15d8896 100644 --- a/pkg/ddl/placement/bundle_test.go +++ b/pkg/ddl/placement/bundle_test.go @@ -347,9 +347,11 @@ func TestString(t *testing.T) { require.NoError(t, err) rules2, err := newRules(pd.Voter, 4, `["-zone=sh", "+zone=bj"]`) require.NoError(t, err) - bundle.Rules = append(rules1, rules2...) + rules3, err := newRules(pd.Voter, 3, `["-engine=tiflash", "-engine=tiflash_compute"]`) + require.NoError(t, err) + bundle.Rules = append(append(rules1, rules2...), rules3...) - require.Equal(t, "{\"group_id\":\"TiDB_DDL_1\",\"group_index\":0,\"group_override\":false,\"rules\":[{\"group_id\":\"\",\"id\":\"\",\"start_key\":\"\",\"end_key\":\"\",\"role\":\"voter\",\"is_witness\":false,\"count\":3,\"label_constraints\":[{\"key\":\"zone\",\"op\":\"in\",\"values\":[\"sh\"]}]},{\"group_id\":\"\",\"id\":\"\",\"start_key\":\"\",\"end_key\":\"\",\"role\":\"voter\",\"is_witness\":false,\"count\":4,\"label_constraints\":[{\"key\":\"zone\",\"op\":\"notIn\",\"values\":[\"sh\"]},{\"key\":\"zone\",\"op\":\"in\",\"values\":[\"bj\"]}]}]}", bundle.String()) + require.Equal(t, "{\"group_id\":\"TiDB_DDL_1\",\"group_index\":0,\"group_override\":false,\"rules\":[{\"group_id\":\"\",\"id\":\"\",\"start_key\":\"\",\"end_key\":\"\",\"role\":\"voter\",\"is_witness\":false,\"count\":3,\"label_constraints\":[{\"key\":\"zone\",\"op\":\"in\",\"values\":[\"sh\"]}]},{\"group_id\":\"\",\"id\":\"\",\"start_key\":\"\",\"end_key\":\"\",\"role\":\"voter\",\"is_witness\":false,\"count\":4,\"label_constraints\":[{\"key\":\"zone\",\"op\":\"notIn\",\"values\":[\"sh\"]},{\"key\":\"zone\",\"op\":\"in\",\"values\":[\"bj\"]}]},{\"group_id\":\"\",\"id\":\"\",\"start_key\":\"\",\"end_key\":\"\",\"role\":\"voter\",\"is_witness\":false,\"count\":3,\"label_constraints\":[{\"key\":\"engine\",\"op\":\"notIn\",\"values\":[\"tiflash\"]},{\"key\":\"engine\",\"op\":\"notIn\",\"values\":[\"tiflash_compute\"]}]}]}", bundle.String()) require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/ddl/placement/MockMarshalFailure", `return(true)`)) defer func() { @@ -956,12 +958,7 @@ func TestTidy(t *testing.T) { require.NoError(t, err) require.Len(t, bundle.Rules, 1) require.Equal(t, "0", bundle.Rules[0].ID) - require.Len(t, bundle.Rules[0].LabelConstraints, 3) - require.Equal(t, pd.LabelConstraint{ - Op: pd.NotIn, - Key: EngineLabelKey, - Values: []string{EngineLabelTiFlash}, - }, bundle.Rules[0].LabelConstraints[2]) + require.Len(t, bundle.Rules[0].LabelConstraints, 2) // merge rules3, err := newRules(pd.Follower, 4, "") @@ -986,13 +983,7 @@ func TestTidy(t *testing.T) { require.Equal(t, "0", bundle.Rules[0].ID) require.Equal(t, "1", bundle.Rules[1].ID) require.Equal(t, 9, bundle.Rules[1].Count) - require.Equal(t, []pd.LabelConstraint{ - { - Op: pd.NotIn, - Key: EngineLabelKey, - Values: []string{EngineLabelTiFlash}, - }, - }, bundle.Rules[1].LabelConstraints) + require.Equal(t, 0, len(bundle.Rules[1].LabelConstraints)) require.Equal(t, []string{"zone", "host"}, bundle.Rules[1].LocationLabels) } err = bundle.Tidy() @@ -1009,13 +1000,6 @@ func TestTidy(t *testing.T) { err = bundle2.Tidy() require.NoError(t, err) require.Equal(t, bundle, bundle2) - - bundle.Rules[1].LabelConstraints = append(bundle.Rules[1].LabelConstraints, pd.LabelConstraint{ - Op: pd.In, - Key: EngineLabelKey, - Values: []string{EngineLabelTiFlash}, - }) - require.ErrorIs(t, bundle.Tidy(), ErrConflictingConstraints) } func TestTidy2(t *testing.T) { @@ -1468,12 +1452,6 @@ func TestTidy2(t *testing.T) { for i, rule := range tt.bundle.Rules { expectedRule := tt.expected.Rules[i] - // Tiflash is always excluded from the constraints. - AddConstraint(&expectedRule.LabelConstraints, pd.LabelConstraint{ - Op: pd.NotIn, - Key: EngineLabelKey, - Values: []string{EngineLabelTiFlash}, - }) if !reflect.DeepEqual(rule, expectedRule) { t.Errorf("unexpected rule at index %d:\nactual=%#v,\nexpected=%#v\n", i, rule, expectedRule) } diff --git a/pkg/ddl/placement/constraint.go b/pkg/ddl/placement/constraint.go index a7463cd897f56..8d017c93523c3 100644 --- a/pkg/ddl/placement/constraint.go +++ b/pkg/ddl/placement/constraint.go @@ -54,6 +54,7 @@ func NewConstraint(label string) (pd.LabelConstraint, error) { return r, fmt.Errorf("%w: %s", ErrInvalidConstraintFormat, label) } + // Does not allow adding rule of tiflash. if op == pd.In && key == EngineLabelKey && strings.ToLower(val) == EngineLabelTiFlash { return r, fmt.Errorf("%w: %s", ErrUnsupportedConstraint, label) } diff --git a/pkg/ddl/placement/constraint_test.go b/pkg/ddl/placement/constraint_test.go index 577cbf0d0f837..8099e785e7c4f 100644 --- a/pkg/ddl/placement/constraint_test.go +++ b/pkg/ddl/placement/constraint_test.go @@ -64,6 +64,15 @@ func TestNewConstraint(t *testing.T) { Values: []string{"tiflash"}, }, }, + { + name: "not tiflash_compute", + input: "-engine = tiflash_compute ", + label: pd.LabelConstraint{ + Key: "engine", + Op: pd.NotIn, + Values: []string{"tiflash_compute"}, + }, + }, { name: "disallow tiflash", input: "+engine=Tiflash", From 18fbb855948ad21caa620dd17853ec560d8c6eed Mon Sep 17 00:00:00 2001 From: Rustin Date: Thu, 9 Jan 2025 17:08:47 +0800 Subject: [PATCH 19/27] statistics: remove unused Flag field from Column and Index structs (#58702) ref pingcap/tidb#55043 --- pkg/statistics/column.go | 2 -- pkg/statistics/handle/bootstrap.go | 6 ++-- .../handle/globalstats/global_stats.go | 1 - pkg/statistics/handle/storage/dump_test.go | 2 +- pkg/statistics/handle/storage/read.go | 32 ++++++++----------- pkg/statistics/handle/storage/save.go | 15 +++------ .../handle/storage/stats_read_writer.go | 7 ++-- .../handle/syncload/stats_syncload.go | 3 +- pkg/statistics/handle/types/interfaces.go | 2 +- pkg/statistics/histogram.go | 3 -- pkg/statistics/index.go | 2 -- 11 files changed, 26 insertions(+), 49 deletions(-) diff --git a/pkg/statistics/column.go b/pkg/statistics/column.go index 5c6c66e239926..e2b6fa3e5dd2d 100644 --- a/pkg/statistics/column.go +++ b/pkg/statistics/column.go @@ -39,7 +39,6 @@ type Column struct { // or it could possibly be -1, which means "stats not available". // The -1 case could happen in a pseudo stats table, and in this case, this stats should not trigger stats loading. PhysicalID int64 - Flag int64 StatsVer int64 // StatsVer is the version of the current stats, used to maintain compatibility IsHandle bool @@ -52,7 +51,6 @@ func (c *Column) Copy() *Column { } nc := &Column{ PhysicalID: c.PhysicalID, - Flag: c.Flag, StatsVer: c.StatsVer, IsHandle: c.IsHandle, } diff --git a/pkg/statistics/handle/bootstrap.go b/pkg/statistics/handle/bootstrap.go index 8d5b6ea8b396b..ee762bba9e381 100644 --- a/pkg/statistics/handle/bootstrap.go +++ b/pkg/statistics/handle/bootstrap.go @@ -190,7 +190,7 @@ func (h *Handle) initStatsHistograms4Chunk(is infoschema.InfoSchema, cache stats table.StatsVer = int(statsVer) } id, ndv, nullCount, version, totColSize := row.GetInt64(2), row.GetInt64(3), row.GetInt64(5), row.GetUint64(4), row.GetInt64(7) - lastAnalyzePos := row.GetDatum(11, types.NewFieldType(mysql.TypeBlob)) + lastAnalyzePos := row.GetDatum(10, types.NewFieldType(mysql.TypeBlob)) tbl, ok := h.TableInfoByID(is, table.PhysicalID) if !ok { // this table has been dropped. but stats meta still exists and wait for being deleted. @@ -227,7 +227,6 @@ func (h *Handle) initStatsHistograms4Chunk(is infoschema.InfoSchema, cache stats TopN: topN, Info: idxInfo, StatsVer: statsVer, - Flag: row.GetInt64(10), PhysicalID: tblID, } if statsVer != statistics.Version0 { @@ -257,7 +256,6 @@ func (h *Handle) initStatsHistograms4Chunk(is infoschema.InfoSchema, cache stats PhysicalID: table.PhysicalID, Info: colInfo, IsHandle: tbl.Meta().PKIsHandle && mysql.HasPriKeyFlag(colInfo.GetFlag()), - Flag: row.GetInt64(10), StatsVer: statsVer, } lastAnalyzePos.Copy(&col.LastAnalyzePos) @@ -285,7 +283,7 @@ func (h *Handle) initStatsHistograms4Chunk(is infoschema.InfoSchema, cache stats // genInitStatsHistogramsSQL generates the SQL to load all stats_histograms records. // We need to read all the records since we need to do initialization of table.ColAndIdxExistenceMap. func genInitStatsHistogramsSQL(isPaging bool) string { - selectPrefix := "select /*+ ORDER_INDEX(mysql.stats_histograms,tbl) */ HIGH_PRIORITY table_id, is_index, hist_id, distinct_count, version, null_count, cm_sketch, tot_col_size, stats_ver, correlation, flag, last_analyze_pos from mysql.stats_histograms" + selectPrefix := "select /*+ ORDER_INDEX(mysql.stats_histograms,tbl) */ HIGH_PRIORITY table_id, is_index, hist_id, distinct_count, version, null_count, cm_sketch, tot_col_size, stats_ver, correlation, last_analyze_pos from mysql.stats_histograms" orderSuffix := " order by table_id" if !isPaging { return selectPrefix + orderSuffix diff --git a/pkg/statistics/handle/globalstats/global_stats.go b/pkg/statistics/handle/globalstats/global_stats.go index 9ba5544994f7e..6e369eb62eb4b 100644 --- a/pkg/statistics/handle/globalstats/global_stats.go +++ b/pkg/statistics/handle/globalstats/global_stats.go @@ -387,7 +387,6 @@ func WriteGlobalStatsToStorage(statsHandle statstypes.StatsHandle, globalStats * cms, topN, info.StatsVersion, - 1, true, util.StatsMetaHistorySourceAnalyze, ) diff --git a/pkg/statistics/handle/storage/dump_test.go b/pkg/statistics/handle/storage/dump_test.go index bdec873c11c72..e1ec55990e791 100644 --- a/pkg/statistics/handle/storage/dump_test.go +++ b/pkg/statistics/handle/storage/dump_test.go @@ -410,7 +410,7 @@ func TestDumpCMSketchWithTopN(t *testing.T) { cms, _, _, _ := statistics.NewCMSketchAndTopN(5, 2048, fakeData, 20, 100) stat := h.GetTableStats(tableInfo) - err = h.SaveStatsToStorage(tableInfo.ID, 1, 0, 0, &stat.GetCol(tableInfo.Columns[0].ID).Histogram, cms, nil, statistics.Version1, 1, false, handleutil.StatsMetaHistorySourceLoadStats) + err = h.SaveStatsToStorage(tableInfo.ID, 1, 0, 0, &stat.GetCol(tableInfo.Columns[0].ID).Histogram, cms, nil, statistics.Version1, false, handleutil.StatsMetaHistorySourceLoadStats) require.NoError(t, err) require.Nil(t, h.Update(context.Background(), is)) diff --git a/pkg/statistics/handle/storage/read.go b/pkg/statistics/handle/storage/read.go index 439fb77db5ee2..18aefabc009ef 100644 --- a/pkg/statistics/handle/storage/read.go +++ b/pkg/statistics/handle/storage/read.go @@ -83,7 +83,7 @@ func statsMetaCountAndModifyCount( } // HistMetaFromStorageWithHighPriority reads the meta info of the histogram from the storage. -func HistMetaFromStorageWithHighPriority(sctx sessionctx.Context, item *model.TableItemID, possibleColInfo *model.ColumnInfo) (*statistics.Histogram, *types.Datum, int64, int64, error) { +func HistMetaFromStorageWithHighPriority(sctx sessionctx.Context, item *model.TableItemID, possibleColInfo *model.ColumnInfo) (*statistics.Histogram, *types.Datum, int64, error) { isIndex := 0 var tp *types.FieldType if item.IsIndex { @@ -93,21 +93,21 @@ func HistMetaFromStorageWithHighPriority(sctx sessionctx.Context, item *model.Ta tp = &possibleColInfo.FieldType } rows, _, err := util.ExecRows(sctx, - "select high_priority distinct_count, version, null_count, tot_col_size, stats_ver, correlation, flag, last_analyze_pos from mysql.stats_histograms where table_id = %? and hist_id = %? and is_index = %?", + "select high_priority distinct_count, version, null_count, tot_col_size, stats_ver, correlation, last_analyze_pos from mysql.stats_histograms where table_id = %? and hist_id = %? and is_index = %?", item.TableID, item.ID, isIndex, ) if err != nil { - return nil, nil, 0, 0, err + return nil, nil, 0, err } if len(rows) == 0 { - return nil, nil, 0, 0, nil + return nil, nil, 0, nil } hist := statistics.NewHistogram(item.ID, rows[0].GetInt64(0), rows[0].GetInt64(2), rows[0].GetUint64(1), tp, chunk.InitialCapacity, rows[0].GetInt64(3)) hist.Correlation = rows[0].GetFloat64(5) - lastPos := rows[0].GetDatum(7, types.NewFieldType(mysql.TypeBlob)) - return hist, &lastPos, rows[0].GetInt64(4), rows[0].GetInt64(6), nil + lastPos := rows[0].GetDatum(6, types.NewFieldType(mysql.TypeBlob)) + return hist, &lastPos, rows[0].GetInt64(4), nil } // HistogramFromStorageWithPriority wraps the HistogramFromStorage with the given kv.Priority. @@ -305,8 +305,7 @@ func indexStatsFromStorage(sctx sessionctx.Context, row chunk.Row, table *statis nullCount := row.GetInt64(5) statsVer := row.GetInt64(7) idx := table.GetIdx(histID) - flag := row.GetInt64(8) - lastAnalyzePos := row.GetDatum(10, types.NewFieldType(mysql.TypeBlob)) + lastAnalyzePos := row.GetDatum(9, types.NewFieldType(mysql.TypeBlob)) for _, idxInfo := range tableInfo.Indices { if histID != idxInfo.ID { @@ -337,7 +336,6 @@ func indexStatsFromStorage(sctx sessionctx.Context, row chunk.Row, table *statis Histogram: *statistics.NewHistogram(histID, distinct, nullCount, histVer, types.NewFieldType(mysql.TypeBlob), 0, 0), StatsVer: statsVer, Info: idxInfo, - Flag: flag, PhysicalID: table.PhysicalID, } if idx.IsAnalyzed() { @@ -371,7 +369,6 @@ func indexStatsFromStorage(sctx sessionctx.Context, row chunk.Row, table *statis FMSketch: fmSketch, Info: idxInfo, StatsVer: statsVer, - Flag: flag, PhysicalID: table.PhysicalID, } if statsVer != statistics.Version0 { @@ -399,10 +396,9 @@ func columnStatsFromStorage(sctx sessionctx.Context, row chunk.Row, table *stati nullCount := row.GetInt64(5) totColSize := row.GetInt64(6) statsVer := row.GetInt64(7) - correlation := row.GetFloat64(9) - lastAnalyzePos := row.GetDatum(10, types.NewFieldType(mysql.TypeBlob)) + correlation := row.GetFloat64(8) + lastAnalyzePos := row.GetDatum(9, types.NewFieldType(mysql.TypeBlob)) col := table.GetCol(histID) - flag := row.GetInt64(8) for _, colInfo := range tableInfo.Columns { if histID != colInfo.ID { @@ -446,7 +442,6 @@ func columnStatsFromStorage(sctx sessionctx.Context, row chunk.Row, table *stati Histogram: *statistics.NewHistogram(histID, distinct, nullCount, histVer, &colInfo.FieldType, 0, totColSize), Info: colInfo, IsHandle: tableInfo.PKIsHandle && mysql.HasPriKeyFlag(colInfo.GetFlag()), - Flag: flag, StatsVer: statsVer, } if col.StatsAvailable() { @@ -482,7 +477,6 @@ func columnStatsFromStorage(sctx sessionctx.Context, row chunk.Row, table *stati TopN: topN, FMSketch: fmSketch, IsHandle: tableInfo.PKIsHandle && mysql.HasPriKeyFlag(colInfo.GetFlag()), - Flag: flag, StatsVer: statsVer, } if col.StatsAvailable() { @@ -538,7 +532,7 @@ func TableStatsFromStorage(sctx sessionctx.Context, snapshot uint64, tableInfo * table.ModifyCount = modidyCount table.RealtimeCount = realtimeCount - rows, _, err := util.ExecRows(sctx, "select table_id, is_index, hist_id, distinct_count, version, null_count, tot_col_size, stats_ver, flag, correlation, last_analyze_pos from mysql.stats_histograms where table_id = %?", tableID) + rows, _, err := util.ExecRows(sctx, "select table_id, is_index, hist_id, distinct_count, version, null_count, tot_col_size, stats_ver, correlation, last_analyze_pos from mysql.stats_histograms where table_id = %?", tableID) if err != nil { return nil, err } @@ -670,7 +664,7 @@ func loadNeededColumnHistograms(sctx sessionctx.Context, statsHandle statstypes. return nil } - hg, _, statsVer, _, err := HistMetaFromStorageWithHighPriority(sctx, &col, colInfo) + hg, _, statsVer, err := HistMetaFromStorageWithHighPriority(sctx, &col, colInfo) if hg == nil || err != nil { asyncload.AsyncLoadHistogramNeededItems.Delete(col) return err @@ -749,7 +743,7 @@ func loadNeededIndexHistograms(sctx sessionctx.Context, is infoschema.InfoSchema asyncload.AsyncLoadHistogramNeededItems.Delete(idx) return nil } - hgMeta, lastAnalyzePos, statsVer, flag, err := HistMetaFromStorageWithHighPriority(sctx, &idx, nil) + hgMeta, lastAnalyzePos, statsVer, err := HistMetaFromStorageWithHighPriority(sctx, &idx, nil) if hgMeta == nil || err != nil { asyncload.AsyncLoadHistogramNeededItems.Delete(idx) return err @@ -776,7 +770,7 @@ func loadNeededIndexHistograms(sctx sessionctx.Context, is infoschema.InfoSchema } idxHist := &statistics.Index{Histogram: *hg, CMSketch: cms, TopN: topN, FMSketch: fms, Info: idxInfo, StatsVer: statsVer, - Flag: flag, PhysicalID: idx.TableID, + PhysicalID: idx.TableID, StatsLoadedStatus: statistics.NewStatsFullLoadStatus()} lastAnalyzePos.Copy(&idxHist.LastAnalyzePos) diff --git a/pkg/statistics/handle/storage/save.go b/pkg/statistics/handle/storage/save.go index 128f2698bd613..33ea4009f00e2 100644 --- a/pkg/statistics/handle/storage/save.go +++ b/pkg/statistics/handle/storage/save.go @@ -274,8 +274,8 @@ func SaveTableStatsToStorage(sctx sessionctx.Context, return 0, err } } - if _, err = util.Exec(sctx, "replace into mysql.stats_histograms (table_id, is_index, hist_id, distinct_count, version, null_count, cm_sketch, tot_col_size, stats_ver, flag, correlation) values (%?, %?, %?, %?, %?, %?, %?, GREATEST(%?, 0), %?, %?, %?)", - tableID, result.IsIndex, hg.ID, hg.NDV, version, hg.NullCount, cmSketch, hg.TotColSize, results.StatsVer, statistics.AnalyzeFlag, hg.Correlation); err != nil { + if _, err = util.Exec(sctx, "replace into mysql.stats_histograms (table_id, is_index, hist_id, distinct_count, version, null_count, cm_sketch, tot_col_size, stats_ver, correlation) values (%?, %?, %?, %?, %?, %?, %?, GREATEST(%?, 0), %?, %?)", + tableID, result.IsIndex, hg.ID, hg.NDV, version, hg.NullCount, cmSketch, hg.TotColSize, results.StatsVer, hg.Correlation); err != nil { return 0, err } if _, err = util.Exec(sctx, "delete from mysql.stats_buckets where table_id = %? and is_index = %? and hist_id = %?", tableID, result.IsIndex, hg.ID); err != nil { @@ -337,7 +337,6 @@ func SaveStatsToStorage( cms *statistics.CMSketch, topN *statistics.TopN, statsVersion int, - isAnalyzed int64, updateAnalyzeTime bool, ) (statsVer uint64, err error) { version, err := util.GetStartTS(sctx) @@ -370,12 +369,8 @@ func SaveStatsToStorage( if _, err := util.Exec(sctx, "delete from mysql.stats_fm_sketch where table_id = %? and is_index = %? and hist_id = %?", tableID, isIndex, hg.ID); err != nil { return 0, err } - flag := 0 - if isAnalyzed == 1 { - flag = statistics.AnalyzeFlag - } - if _, err = util.Exec(sctx, "replace into mysql.stats_histograms (table_id, is_index, hist_id, distinct_count, version, null_count, cm_sketch, tot_col_size, stats_ver, flag, correlation) values (%?, %?, %?, %?, %?, %?, %?, GREATEST(%?, 0), %?, %?, %?)", - tableID, isIndex, hg.ID, hg.NDV, version, hg.NullCount, cmSketch, hg.TotColSize, statsVersion, flag, hg.Correlation); err != nil { + if _, err = util.Exec(sctx, "replace into mysql.stats_histograms (table_id, is_index, hist_id, distinct_count, version, null_count, cm_sketch, tot_col_size, stats_ver, correlation) values (%?, %?, %?, %?, %?, %?, %?, GREATEST(%?, 0), %?, %?)", + tableID, isIndex, hg.ID, hg.NDV, version, hg.NullCount, cmSketch, hg.TotColSize, statsVersion, hg.Correlation); err != nil { return 0, err } if _, err = util.Exec(sctx, "delete from mysql.stats_buckets where table_id = %? and is_index = %? and hist_id = %?", tableID, isIndex, hg.ID); err != nil { @@ -386,7 +381,7 @@ func SaveStatsToStorage( if err != nil { return 0, err } - if isAnalyzed == 1 && len(lastAnalyzePos) > 0 { + if len(lastAnalyzePos) > 0 { if _, err = util.Exec(sctx, "update mysql.stats_histograms set last_analyze_pos = %? where table_id = %? and is_index = %? and hist_id = %?", lastAnalyzePos, tableID, isIndex, hg.ID); err != nil { return 0, err } diff --git a/pkg/statistics/handle/storage/stats_read_writer.go b/pkg/statistics/handle/storage/stats_read_writer.go index 3b1d1f4b0812f..696ed86a0997e 100644 --- a/pkg/statistics/handle/storage/stats_read_writer.go +++ b/pkg/statistics/handle/storage/stats_read_writer.go @@ -175,14 +175,13 @@ func (s *statsReadWriter) SaveStatsToStorage( cms *statistics.CMSketch, topN *statistics.TopN, statsVersion int, - isAnalyzed int64, updateAnalyzeTime bool, source string, ) (err error) { var statsVer uint64 err = util.CallWithSCtx(s.statsHandler.SPool(), func(sctx sessionctx.Context) error { statsVer, err = SaveStatsToStorage(sctx, tableID, - count, modifyCount, isIndex, hg, cms, topN, statsVersion, isAnalyzed, updateAnalyzeTime) + count, modifyCount, isIndex, hg, cms, topN, statsVersion, updateAnalyzeTime) return err }, util.FlagWrapTxn) if err == nil && statsVer != 0 { @@ -627,7 +626,7 @@ func (s *statsReadWriter) loadStatsFromJSON(tableInfo *model.TableInfo, physical // loadStatsFromJSON doesn't support partition table now. // The table level count and modify_count would be overridden by the SaveMetaToStorage below, so we don't need // to care about them here. - if err := s.SaveStatsToStorage(tbl.PhysicalID, tbl.RealtimeCount, 0, 0, &col.Histogram, col.CMSketch, col.TopN, int(col.GetStatsVer()), statistics.AnalyzeFlag, false, util.StatsMetaHistorySourceLoadStats); err != nil { + if err := s.SaveStatsToStorage(tbl.PhysicalID, tbl.RealtimeCount, 0, 0, &col.Histogram, col.CMSketch, col.TopN, int(col.GetStatsVer()), false, util.StatsMetaHistorySourceLoadStats); err != nil { outerErr = err return true } @@ -640,7 +639,7 @@ func (s *statsReadWriter) loadStatsFromJSON(tableInfo *model.TableInfo, physical // loadStatsFromJSON doesn't support partition table now. // The table level count and modify_count would be overridden by the SaveMetaToStorage below, so we don't need // to care about them here. - if err := s.SaveStatsToStorage(tbl.PhysicalID, tbl.RealtimeCount, 0, 1, &idx.Histogram, idx.CMSketch, idx.TopN, int(idx.GetStatsVer()), statistics.AnalyzeFlag, false, util.StatsMetaHistorySourceLoadStats); err != nil { + if err := s.SaveStatsToStorage(tbl.PhysicalID, tbl.RealtimeCount, 0, 1, &idx.Histogram, idx.CMSketch, idx.TopN, int(idx.GetStatsVer()), false, util.StatsMetaHistorySourceLoadStats); err != nil { outerErr = err return true } diff --git a/pkg/statistics/handle/syncload/stats_syncload.go b/pkg/statistics/handle/syncload/stats_syncload.go index 695a69643830a..6374b1e24d838 100644 --- a/pkg/statistics/handle/syncload/stats_syncload.go +++ b/pkg/statistics/handle/syncload/stats_syncload.go @@ -404,7 +404,7 @@ func (*statsSyncLoad) readStatsForOneItem(sctx sessionctx.Context, item model.Ta var hg *statistics.Histogram var err error isIndexFlag := int64(0) - hg, lastAnalyzePos, statsVer, flag, err := storage.HistMetaFromStorageWithHighPriority(sctx, &item, w.colInfo) + hg, lastAnalyzePos, statsVer, err := storage.HistMetaFromStorageWithHighPriority(sctx, &item, w.colInfo) if err != nil { return nil, err } @@ -451,7 +451,6 @@ func (*statsSyncLoad) readStatsForOneItem(sctx sessionctx.Context, item model.Ta FMSketch: fms, Info: w.idxInfo, StatsVer: statsVer, - Flag: flag, PhysicalID: item.TableID, } if statsVer != statistics.Version0 { diff --git a/pkg/statistics/handle/types/interfaces.go b/pkg/statistics/handle/types/interfaces.go index df66858b9ebf7..bb02b36f6f8c9 100644 --- a/pkg/statistics/handle/types/interfaces.go +++ b/pkg/statistics/handle/types/interfaces.go @@ -345,7 +345,7 @@ type StatsReadWriter interface { // SaveStatsToStorage save the stats data to the storage. SaveStatsToStorage(tableID int64, count, modifyCount int64, isIndex int, hg *statistics.Histogram, - cms *statistics.CMSketch, topN *statistics.TopN, statsVersion int, isAnalyzed int64, updateAnalyzeTime bool, source string) (err error) + cms *statistics.CMSketch, topN *statistics.TopN, statsVersion int, updateAnalyzeTime bool, source string) (err error) // SaveTableStatsToStorage saves the stats of a table to storage. SaveTableStatsToStorage(results *statistics.AnalyzeResults, analyzeSnapshot bool, source string) (err error) diff --git a/pkg/statistics/histogram.go b/pkg/statistics/histogram.go index 030d53208601b..80c4c02dc74f7 100644 --- a/pkg/statistics/histogram.go +++ b/pkg/statistics/histogram.go @@ -263,9 +263,6 @@ const ( Version2 = 2 ) -// AnalyzeFlag is set when the statistics comes from analyze. -const AnalyzeFlag = 1 - // ValueToString converts a possible encoded value to a formatted string. If the value is encoded, then // idxCols equals to number of origin values, else idxCols is 0. func ValueToString(vars *variable.SessionVars, value *types.Datum, idxCols int, idxColumnTypes []byte) (string, error) { diff --git a/pkg/statistics/index.go b/pkg/statistics/index.go index 53686852f0eb8..a2fed8bcf9be8 100644 --- a/pkg/statistics/index.go +++ b/pkg/statistics/index.go @@ -35,7 +35,6 @@ type Index struct { Histogram StatsLoadedStatus StatsVer int64 // StatsVer is the version of the current stats, used to maintain compatibility - Flag int64 // PhysicalID is the physical table id, // or it could possibly be -1, which means "stats not available". // The -1 case could happen in a pseudo stats table, and in this case, this stats should not trigger stats loading. @@ -49,7 +48,6 @@ func (idx *Index) Copy() *Index { } nc := &Index{ PhysicalID: idx.PhysicalID, - Flag: idx.Flag, StatsVer: idx.StatsVer, } idx.LastAnalyzePos.Copy(&nc.LastAnalyzePos) From b22555b1c8bda6b272d199a5c7bacd1e3b299858 Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Thu, 9 Jan 2025 11:12:49 +0100 Subject: [PATCH 20/27] ddl: Fix issue with concurrent update getting reverted by BackfillData (#58229) close pingcap/tidb#58226, close pingcap/tidb#58692 --- pkg/ddl/partition.go | 202 ++++++------------- pkg/ddl/tests/partition/multi_domain_test.go | 177 ++++++++++++++-- 2 files changed, 222 insertions(+), 157 deletions(-) diff --git a/pkg/ddl/partition.go b/pkg/ddl/partition.go index 9d5a4d4914fd1..1198b932c4375 100644 --- a/pkg/ddl/partition.go +++ b/pkg/ddl/partition.go @@ -3694,42 +3694,15 @@ func doPartitionReorgWork(w *worker, jobCtx *jobContext, job *model.Job, tbl tab rh := newReorgHandler(sess.NewSession(sctx)) reorgTblInfo := tbl.Meta().Clone() var elements []*meta.Element - isClustered := tbl.Meta().PKIsHandle || tbl.Meta().IsCommonHandle - if isClustered { - indices := make([]*model.IndexInfo, 0, len(tbl.Meta().Indices)) - for _, index := range tbl.Meta().Indices { - if isNew, ok := tbl.Meta().GetPartitionInfo().DDLChangedIndex[index.ID]; ok && !isNew { - // Skip old replaced indexes, but rebuild all other indexes - continue - } - indices = append(indices, index) - } - elements = BuildElements(tbl.Meta().Columns[0], indices) - } else { - // Non-clustered tables needs to generate new _tidb_rowid for each row, since - // there might be duplicates due to EXCHANGE PARTITION. - // That means that we can not first copy all table records and then - // recreate all indexes, since we cannot determine if a table record - // has been copied or not, since its _tidb_rowid handle has been recreated - // in the new partition. - // So we will read a batch of records from one partition at a time, - // do a BatchGet for all the record keys in the new partitions, - // to see if any of the records is already there with the same handle/_tidb_rowid - // which means they were double written and does not need to be copied. - // use AddRecord for all non-matching records. - // TODO: if there is an issue where we will retry the same batch and we have committed - // backfilled records and indexes without committing the updated reorgInfo start/end key, - // then the DDL can fail due to duplicate key. - reorgTblInfo.Indices = reorgTblInfo.Indices[:0] - for _, index := range tbl.Meta().Indices { - if isNew, ok := tbl.Meta().GetPartitionInfo().DDLChangedIndex[index.ID]; ok && !isNew { - // Skip old replaced indexes, but rebuild all other indexes - continue - } - reorgTblInfo.Indices = append(reorgTblInfo.Indices, index) + indices := make([]*model.IndexInfo, 0, len(tbl.Meta().Indices)) + for _, index := range tbl.Meta().Indices { + if isNew, ok := tbl.Meta().GetPartitionInfo().DDLChangedIndex[index.ID]; ok && !isNew { + // Skip old replaced indexes, but rebuild all other indexes + continue } - elements = BuildElements(tbl.Meta().Columns[0], reorgTblInfo.Indices) + indices = append(indices, index) } + elements = BuildElements(tbl.Meta().Columns[0], indices) reorgTbl, err := getTable(jobCtx.getAutoIDRequirement(), job.SchemaID, reorgTblInfo) if err != nil { return false, ver, errors.Trace(err) @@ -3776,7 +3749,6 @@ func doPartitionReorgWork(w *worker, jobCtx *jobContext, job *model.Job, tbl tab type reorgPartitionWorker struct { *backfillCtx - records int // Static allocated to limit memory allocations rowRecords []*rowRecord rowDecoder *decoder.RowDecoder @@ -3788,11 +3760,7 @@ type reorgPartitionWorker struct { // and check if the old _tidb_rowid was already written or not. // If the old _tidb_rowid already exists, then the row is already backfilled (double written) // and can be skipped. Otherwise, we will insert it and generate index entries. - rows [][]types.Datum - // The original _tidb_rowids, used to check if already backfilled (double written). oldKeys []kv.Key - // partition ids of the new rows - newPids []int64 } func newReorgPartitionWorker(i int, t table.PhysicalTable, decodeColMap map[int64]decoder.Column, reorgInfo *reorgInfo, jc *ReorgContext) (*reorgPartitionWorker, error) { @@ -3852,54 +3820,55 @@ func (w *reorgPartitionWorker) BackfillData(handleRange reorgBackfillTask) (task taskCtx.nextKey = nextKey taskCtx.done = taskDone - isClustered := w.reorgedTbl.Meta().IsCommonHandle || w.reorgedTbl.Meta().PKIsHandle - if !isClustered { - // non-clustered table, we need to replace the _tidb_rowid handles since - // there may be duplicates across different partitions, due to EXCHANGE PARTITION. - // Meaning we need to check here if a record was double written to the new partition, - // i.e. concurrently written by StateWriteOnly or StateWriteReorganization. - // and we should skip it. - var found map[string][]byte + failpoint.InjectCall("PartitionBackfillData", len(w.rowRecords) > 0) + // For non-clustered tables, we need to replace the _tidb_rowid handles since + // there may be duplicates across different partitions, due to EXCHANGE PARTITION. + // Meaning we need to check here if a record was double written to the new partition, + // i.e. concurrently written by StateWriteOnly or StateWriteReorganization. + // and if so, skip it. + var found map[string][]byte + if len(w.oldKeys) > 0 { + // we must check if old IDs already been written, + // i.e. double written by StateWriteOnly or StateWriteReorganization. + // TODO: while waiting for BatchGet to check for duplicate, do another round of reads in parallel? + found, err = txn.BatchGet(ctx, w.oldKeys) + if err != nil { + return errors.Trace(err) + } + } + + for i, prr := range w.rowRecords { + taskCtx.scanCount++ + key := prr.key + + // w.oldKeys is only set for non-clustered tables, in w.fetchRowColVals(). if len(w.oldKeys) > 0 { - // we must check if old IDs already been written, - // i.e. double written by StateWriteOnly or StateWriteReorganization. - // The good thing is that we can then also skip the index generation for that row and we don't need to - // check if duplicate index entries was already copied either! - // TODO: while waiting for BatchGet to check for duplicate, do another round of reads in parallel? - found, err = txn.BatchGet(ctx, w.oldKeys) + if _, ok := found[string(w.oldKeys[i])]; ok { + // Already filled, i.e. double written earlier by concurrent DML + continue + } + + // Check if we can lock the old key, since there can still be concurrent update + // happening on the rows from fetchRowColVals(), if we cannot lock the keys in this + // transaction and succeed when committing, then another transaction did update + // the same key, and we will fail and retry. When retrying, this key would be found + // through BatchGet and skipped. + err = txn.LockKeys(context.Background(), new(kv.LockCtx), w.oldKeys[i]) if err != nil { return errors.Trace(err) } - } - for i := 0; i < w.records; i++ { - taskCtx.scanCount++ - - if len(w.oldKeys) > 0 { - if _, ok := found[string(w.oldKeys[i])]; ok { - // Alredy filled - continue - } - tbl := w.reorgedTbl.GetPartition(w.newPids[i]) - if tbl == nil { - return dbterror.ErrUnsupportedReorganizePartition.GenWithStackByArgs() - } - // TODO: is this looking up each index entry at a time or in an optimistic way and only checks - // at commit time? - // AddRecord will assign a new _tidb_rowid, since we don't provide one. - _, err = tbl.AddRecord(w.tblCtx, txn, w.rows[i]) - if err != nil { - return errors.Trace(err) - } - taskCtx.addedCount++ + // Due to EXCHANGE PARTITION, the existing _tidb_rowid may collide between partitions! + // Generate new _tidb_rowid. + recordID, err := tables.AllocHandle(w.ctx, w.tblCtx, w.reorgedTbl) + if err != nil { + return errors.Trace(err) } + + // tablecodec.prefixLen is not exported, but is just TableSplitKeyLen + 2 + key = tablecodec.EncodeRecordKey(key[:tablecodec.TableSplitKeyLen+2], recordID) } - return nil - } - // Clustered table, use tried implementation - for _, prr := range w.rowRecords { - taskCtx.scanCount++ - err = txn.Set(prr.key, prr.vals) + err = txn.Set(key, prr.vals) if err != nil { return errors.Trace(err) } @@ -3914,15 +3883,8 @@ func (w *reorgPartitionWorker) BackfillData(handleRange reorgBackfillTask) (task func (w *reorgPartitionWorker) fetchRowColVals(txn kv.Transaction, taskRange reorgBackfillTask) (kv.Key, bool, error) { w.rowRecords = w.rowRecords[:0] - w.records = 0 isClustered := w.reorgedTbl.Meta().IsCommonHandle || w.reorgedTbl.Meta().PKIsHandle - if !isClustered { - if cap(w.rows) < w.batchCnt { - w.rows = make([][]types.Datum, w.batchCnt) - } - } w.oldKeys = w.oldKeys[:0] - w.newPids = w.newPids[:0] startTime := time.Now() // taskDone means that the added handle is out of taskRange.endHandle. @@ -3940,7 +3902,7 @@ func (w *reorgPartitionWorker) fetchRowColVals(txn kv.Transaction, taskRange reo taskDone = recordKey.Cmp(taskRange.endKey) >= 0 - if taskDone || w.records >= w.batchCnt { + if taskDone || len(w.rowRecords) >= w.batchCnt { return false, nil } @@ -3949,51 +3911,26 @@ func (w *reorgPartitionWorker) fetchRowColVals(txn kv.Transaction, taskRange reo return false, errors.Trace(err) } - if isClustered { - // Set all partitioning columns and calculate which partition to write to - for colID, offset := range w.writeColOffsetMap { - d, ok := w.rowMap[colID] - if !ok { - return false, dbterror.ErrUnsupportedReorganizePartition.GenWithStackByArgs() - } - tmpRow[offset] = d - } - } else { - // _tidb_rowid needs to be regenerated, due to EXCHANGE PARTITION, meaning we cannot - // delay the index generation, but need to check if the current _tidb_rowid already exists - // in the new partition or not, before we write the newly generated one. - // and later in the caller of this function write both Record and all indexes - - // Set all columns and calculate which partition to write to - // We will later copy the row, so use all writable columns - for _, col := range w.reorgedTbl.WritableCols() { - d, ok := w.rowMap[col.ID] - if !ok { - return false, dbterror.ErrUnsupportedReorganizePartition.GenWithStackByArgs() - } - tmpRow[col.Offset] = d + // Set all partitioning columns and calculate which partition to write to + for colID, offset := range w.writeColOffsetMap { + d, ok := w.rowMap[colID] + if !ok { + return false, dbterror.ErrUnsupportedReorganizePartition.GenWithStackByArgs() } + tmpRow[offset] = d } p, err := w.reorgedTbl.GetPartitionByRow(w.exprCtx.GetEvalCtx(), tmpRow) if err != nil { return false, errors.Trace(err) } - if isClustered { - newKey := tablecodec.EncodeTablePrefix(p.GetPhysicalID()) - newKey = append(newKey, recordKey[tablecodec.TableSplitKeyLen:]...) - w.rowRecords = append(w.rowRecords, &rowRecord{key: newKey, vals: rawRow}) - w.records++ - } else { - if cap(w.rows[w.records]) < len(tmpRow) { - w.rows[w.records] = make([]types.Datum, len(tmpRow)) - } - copy(w.rows[w.records], tmpRow) - w.newPids = append(w.newPids, p.GetPhysicalID()) + newKey := tablecodec.EncodeTablePrefix(p.GetPhysicalID()) + newKey = append(newKey, recordKey[tablecodec.TableSplitKeyLen:]...) + w.rowRecords = append(w.rowRecords, &rowRecord{key: newKey, vals: rawRow}) - oldKey := tablecodec.EncodeTablePrefix(p.GetPhysicalID()) + if !isClustered { + oldKey := newKey[:tablecodec.TableSplitKeyLen] oldKey = append(oldKey, recordKey[tablecodec.TableSplitKeyLen:]...) w.oldKeys = append(w.oldKeys, oldKey) - w.records++ } w.cleanRowMap() @@ -4005,7 +3942,7 @@ func (w *reorgPartitionWorker) fetchRowColVals(txn kv.Transaction, taskRange reo return true, nil }) - if w.records == 0 { + if len(w.rowRecords) == 0 { taskDone = true } @@ -4048,11 +3985,8 @@ func (w *worker) reorgPartitionDataAndIndex( // - Transactions on different TiDB nodes/domains may see different states of the table/partitions // - We cannot have multiple partition ids for a unique index entry. - isClustered := t.Meta().PKIsHandle || t.Meta().IsCommonHandle - // Copy the data from the DroppingDefinitions to the AddingDefinitions if bytes.Equal(reorgInfo.currElement.TypeKey, meta.ColumnElementKey) { - // if non-clustered table it will also create its indexes! err = w.updatePhysicalTableRow(ctx, t, reorgInfo) if err != nil { return errors.Trace(err) @@ -4111,14 +4045,10 @@ func (w *worker) reorgPartitionDataAndIndex( pi := t.Meta().GetPartitionInfo() if _, err = findNextPartitionID(reorgInfo.PhysicalTableID, pi.AddingDefinitions); err == nil { - // Now build all the indexes in the new partitions - // apart from non-clustered index tables, where new partitions already - // created its indexes together with the table records. - if isClustered { - err = w.addTableIndex(ctx, t, reorgInfo) - if err != nil { - return errors.Trace(err) - } + // Now build all the indexes in the new partitions. + err = w.addTableIndex(ctx, t, reorgInfo) + if err != nil { + return errors.Trace(err) } // All indexes are up-to-date for new partitions, // now we only need to add the existing non-touched partitions diff --git a/pkg/ddl/tests/partition/multi_domain_test.go b/pkg/ddl/tests/partition/multi_domain_test.go index d85a11122a876..63baa67dc4c72 100644 --- a/pkg/ddl/tests/partition/multi_domain_test.go +++ b/pkg/ddl/tests/partition/multi_domain_test.go @@ -811,6 +811,9 @@ func TestMultiSchemaDropUniqueIndex(t *testing.T) { //} func runMultiSchemaTest(t *testing.T, createSQL, alterSQL string, initFn func(*testkit.TestKit), postFn func(*testkit.TestKit, kv.Storage), loopFn func(tO, tNO *testkit.TestKit)) { + runMultiSchemaTestWithBackfillDML(t, createSQL, alterSQL, "", initFn, postFn, loopFn) +} +func runMultiSchemaTestWithBackfillDML(t *testing.T, createSQL, alterSQL, backfillDML string, initFn func(*testkit.TestKit), postFn func(*testkit.TestKit, kv.Storage), loopFn func(tO, tNO *testkit.TestKit)) { // When debugging, increase the lease, so the schema does not auto reload :) distCtx := testkit.NewDistExecutionContextWithLease(t, 2, 15*time.Second) store := distCtx.Store @@ -828,8 +831,9 @@ func runMultiSchemaTest(t *testing.T, createSQL, alterSQL string, initFn func(*t tkDDLOwner := testkit.NewTestKitWithSession(t, store, seOwner) tkDDLOwner.MustExec(`use test`) - tkDDLOwner.MustExec(`set @@global.tidb_enable_global_index = 1`) - tkDDLOwner.MustExec(`set @@session.tidb_enable_global_index = 1`) + // Just to ensure we are not relying on the configurable assertions + tkDDLOwner.MustExec(`set @@global.tidb_txn_assertion_level = off`) + tkDDLOwner.MustExec(`set @@session.tidb_txn_assertion_level = off`) tkO := testkit.NewTestKitWithSession(t, store, seOwner) tkO.MustExec(`use test`) tkNO := testkit.NewTestKitWithSession(t, store, seNonOwner) @@ -873,8 +877,21 @@ func runMultiSchemaTest(t *testing.T, createSQL, alterSQL string, initFn func(*t testfailpoint.EnableCall(t, "github.com/pingcap/tidb/pkg/ddl/afterRunOneJobStep", hookFunc) alterChan := make(chan error) go func() { + if backfillDML != "" { + // This can be used for testing concurrent writes during backfill. + testfailpoint.EnableCall(t, "github.com/pingcap/tidb/pkg/ddl/PartitionBackfillData", func(b bool) { + if b { + logutil.BgLogger().Info("XXXXXXXXXXX Concurrent UPDATE!") + tkO.MustExec(backfillDML) + } + }) + } + logutil.BgLogger().Info("XXXXXXXXXXX DDL starting!", zap.String("alterSQL", alterSQL)) err := tkDDLOwner.ExecToErr(alterSQL) logutil.BgLogger().Info("XXXXXXXXXXX DDL done!", zap.String("alterSQL", alterSQL)) + if backfillDML != "" { + testfailpoint.Disable(t, "github.com/pingcap/tidb/pkg/ddl/PartitionBackfillData") + } alterChan <- err }() // Skip the first state, since we want to compare before vs after in the loop @@ -1022,45 +1039,163 @@ func HaveEntriesForTableIndex(t *testing.T, tk *testkit.TestKit, tableID, indexI } return false } +func TestMultiSchemaReorganizePK(t *testing.T) { + createSQL := `create table t (c1 INT primary key, c2 CHAR(255), c3 CHAR(255), c4 CHAR(255), c5 CHAR(255)) partition by range (c1) (partition p1 values less than (200), partition pMax values less than (maxvalue))` + i := 1 + initFn := func(tkO *testkit.TestKit) { + tkO.MustExec(fmt.Sprintf(`insert into t values (%d,'%s','Original',%d,%d)`, i, "init O", 4185725186-i, 7483634197-i)) + i++ + tkO.MustExec(fmt.Sprintf(`insert into t values (%d,'%s','Original',%d,%d)`, i, "init O", 4185725186-i, 7483634197-i)) + i++ + } + alterSQL := `alter table t reorganize partition p1 into (partition p0 values less than (100), partition p1 values less than (200))` + loopFn := func(tkO, tkNO *testkit.TestKit) { + res := tkO.MustQuery(`select schema_state from information_schema.DDL_JOBS where table_name = 't' order by job_id desc limit 1`) + schemaState := res.Rows()[0][0].(string) + tkO.MustExec(fmt.Sprintf(`insert into t values (%d,'%s','Original',%d,%d)`, i, schemaState+" O", 4185725186-i, 7483634197-i)) + i++ + tkNO.MustExec(fmt.Sprintf(`insert into t values (%d,'%s','Original',%d,%d)`, i, schemaState+" NO", 4185725186-i, 7483634197-i)) + i++ + } + postFn := func(tkO *testkit.TestKit, _ kv.Storage) { + require.Equal(t, int(7*2+1), i) + tkO.MustQuery(`select c1,c2 from t`).Sort().Check(testkit.Rows(""+ + "1 init O", + "10 delete reorganization NO", + "11 public O", + "12 public NO", + "13 none O", + "14 none NO", + "2 init O", + "3 delete only O", + "4 delete only NO", + "5 write only O", + "6 write only NO", + "7 write reorganization O", + "8 write reorganization NO", + "9 delete reorganization O")) + } + runMultiSchemaTest(t, createSQL, alterSQL, initFn, postFn, loopFn) +} + +func TestMultiSchemaReorganizePKBackfillDML(t *testing.T) { + createSQL := `create table t (c1 INT primary key, c2 CHAR(255), c3 CHAR(255), c4 CHAR(255), c5 CHAR(255)) partition by range (c1) (partition p1 values less than (200), partition pMax values less than (maxvalue))` + i := 1 + initFn := func(tkO *testkit.TestKit) { + tkO.MustExec(fmt.Sprintf(`insert into t values (%d,'%s','Original',%d,%d)`, i, "init O", 4185725186-i, 7483634197-i)) + i++ + tkO.MustExec(fmt.Sprintf(`insert into t values (%d,'%s','Original',%d,%d)`, i, "init O", 4185725186-i, 7483634197-i)) + i++ + } + alterSQL := `alter table t reorganize partition p1 into (partition p0 values less than (100), partition p1 values less than (200))` + loopFn := func(tkO, tkNO *testkit.TestKit) { + res := tkO.MustQuery(`select schema_state from information_schema.DDL_JOBS where table_name = 't' order by job_id desc limit 1`) + schemaState := res.Rows()[0][0].(string) + tkO.MustExec(fmt.Sprintf(`insert into t values (%d,'%s','Original',%d,%d)`, i, schemaState+" O", 4185725186-i, 7483634197-i)) + i++ + tkNO.MustExec(fmt.Sprintf(`insert into t values (%d,'%s','Original',%d,%d)`, i, schemaState+" NO", 4185725186-i, 7483634197-i)) + i++ + } + postFn := func(tkO *testkit.TestKit, _ kv.Storage) { + tkO.MustQuery(`select c1,c2,c3 from t`).Sort().Check(testkit.Rows(""+ + "1 init O updated", + "10 delete reorganization NO Original", + "11 public O Original", + "12 public NO Original", + "13 none O Original", + "14 none NO Original", + "2 init O updated", + "3 delete only O updated", + "4 delete only NO updated", + "5 write only O updated", + "6 write only NO updated", + "7 write reorganization O Original", + "8 write reorganization NO Original", + "9 delete reorganization O Original")) + } + runMultiSchemaTestWithBackfillDML(t, createSQL, alterSQL, "update t set c3 = 'updated'", initFn, postFn, loopFn) +} func TestMultiSchemaReorganizeNoPK(t *testing.T) { createSQL := `create table t (c1 INT, c2 CHAR(255), c3 CHAR(255), c4 CHAR(255), c5 CHAR(255)) partition by range (c1) (partition p1 values less than (200), partition pMax values less than (maxvalue))` i := 1 initFn := func(tkO *testkit.TestKit) { - tkO.MustExec(fmt.Sprintf(`insert into t values (%d,repeat('%d', 25),repeat('%d', 25),repeat('%d', 25),repeat('%d', 25))`, i, 9786756453-i, 6821527184-i, 4185725186-i, 7483634197-i)) + tkO.MustExec(fmt.Sprintf(`insert into t values (%d,'%s','Original',repeat('%d', 25),repeat('%d', 25))`, i, "init O", 4185725186-i, 7483634197-i)) i++ - tkO.MustExec(fmt.Sprintf(`insert into t values (%d,repeat('%d', 25),repeat('%d', 25),repeat('%d', 25),repeat('%d', 25))`, i, 9786756453-i, 6821527184-i, 4185725186-i, 7483634197-i)) + tkO.MustExec(fmt.Sprintf(`insert into t values (%d,'%s','Original',repeat('%d', 25),repeat('%d', 25))`, i, "init O", 4185725186-i, 7483634197-i)) i++ } alterSQL := `alter table t reorganize partition p1 into (partition p0 values less than (100), partition p1 values less than (200))` loopFn := func(tkO, tkNO *testkit.TestKit) { res := tkO.MustQuery(`select schema_state from information_schema.DDL_JOBS where table_name = 't' order by job_id desc limit 1`) schemaState := res.Rows()[0][0].(string) - tkO.MustExec(fmt.Sprintf(`insert into t values (%d,'%s',concat('O-', repeat('%d', 25)),repeat('%d', 25),repeat('%d', 25))`, i, schemaState, 6821527184-i, 4185725186-i, 7483634197-i)) + tkO.MustExec(fmt.Sprintf(`insert into t values (%d,'%s','Original',repeat('%d', 25),repeat('%d', 25))`, i, schemaState+" O", 4185725186-i, 7483634197-i)) i++ - tkNO.MustExec(fmt.Sprintf(`insert into t values (%d,'%s',concat('NO-',repeat('%d', 25)),repeat('%d', 25),repeat('%d', 25))`, i, schemaState, 6821527184-i, 4185725186-i, 7483634197-i)) + tkNO.MustExec(fmt.Sprintf(`insert into t values (%d,'%s','Original',repeat('%d', 25),repeat('%d', 25))`, i, schemaState+" NO", 4185725186-i, 7483634197-i)) i++ } postFn := func(tkO *testkit.TestKit, _ kv.Storage) { - tkO.MustQuery(`select * from t`).Sort().Check(testkit.Rows(""+ - "1 9786756452978675645297867564529786756452978675645297867564529786756452978675645297867564529786756452978675645297867564529786756452978675645297867564529786756452978675645297867564529786756452978675645297867564529786756452978675645297867564529786756452 6821527183682152718368215271836821527183682152718368215271836821527183682152718368215271836821527183682152718368215271836821527183682152718368215271836821527183682152718368215271836821527183682152718368215271836821527183682152718368215271836821527183 4185725185418572518541857251854185725185418572518541857251854185725185418572518541857251854185725185418572518541857251854185725185418572518541857251854185725185418572518541857251854185725185418572518541857251854185725185418572518541857251854185725185 7483634196748363419674836341967483634196748363419674836341967483634196748363419674836341967483634196748363419674836341967483634196748363419674836341967483634196748363419674836341967483634196748363419674836341967483634196748363419674836341967483634196", - "10 delete reorganization NO-6821527174682152717468215271746821527174682152717468215271746821527174682152717468215271746821527174682152717468215271746821527174682152717468215271746821527174682152717468215271746821527174682152717468215271746821527174682152717468215271746821527174 4185725176418572517641857251764185725176418572517641857251764185725176418572517641857251764185725176418572517641857251764185725176418572517641857251764185725176418572517641857251764185725176418572517641857251764185725176418572517641857251764185725176 7483634187748363418774836341877483634187748363418774836341877483634187748363418774836341877483634187748363418774836341877483634187748363418774836341877483634187748363418774836341877483634187748363418774836341877483634187748363418774836341877483634187", - "11 public O-6821527173682152717368215271736821527173682152717368215271736821527173682152717368215271736821527173682152717368215271736821527173682152717368215271736821527173682152717368215271736821527173682152717368215271736821527173682152717368215271736821527173 4185725175418572517541857251754185725175418572517541857251754185725175418572517541857251754185725175418572517541857251754185725175418572517541857251754185725175418572517541857251754185725175418572517541857251754185725175418572517541857251754185725175 7483634186748363418674836341867483634186748363418674836341867483634186748363418674836341867483634186748363418674836341867483634186748363418674836341867483634186748363418674836341867483634186748363418674836341867483634186748363418674836341867483634186", - "12 public NO-6821527172682152717268215271726821527172682152717268215271726821527172682152717268215271726821527172682152717268215271726821527172682152717268215271726821527172682152717268215271726821527172682152717268215271726821527172682152717268215271726821527172 4185725174418572517441857251744185725174418572517441857251744185725174418572517441857251744185725174418572517441857251744185725174418572517441857251744185725174418572517441857251744185725174418572517441857251744185725174418572517441857251744185725174 7483634185748363418574836341857483634185748363418574836341857483634185748363418574836341857483634185748363418574836341857483634185748363418574836341857483634185748363418574836341857483634185748363418574836341857483634185748363418574836341857483634185", - "13 none O-6821527171682152717168215271716821527171682152717168215271716821527171682152717168215271716821527171682152717168215271716821527171682152717168215271716821527171682152717168215271716821527171682152717168215271716821527171682152717168215271716821527171 4185725173418572517341857251734185725173418572517341857251734185725173418572517341857251734185725173418572517341857251734185725173418572517341857251734185725173418572517341857251734185725173418572517341857251734185725173418572517341857251734185725173 7483634184748363418474836341847483634184748363418474836341847483634184748363418474836341847483634184748363418474836341847483634184748363418474836341847483634184748363418474836341847483634184748363418474836341847483634184748363418474836341847483634184", - "14 none NO-6821527170682152717068215271706821527170682152717068215271706821527170682152717068215271706821527170682152717068215271706821527170682152717068215271706821527170682152717068215271706821527170682152717068215271706821527170682152717068215271706821527170 4185725172418572517241857251724185725172418572517241857251724185725172418572517241857251724185725172418572517241857251724185725172418572517241857251724185725172418572517241857251724185725172418572517241857251724185725172418572517241857251724185725172 7483634183748363418374836341837483634183748363418374836341837483634183748363418374836341837483634183748363418374836341837483634183748363418374836341837483634183748363418374836341837483634183748363418374836341837483634183748363418374836341837483634183", - "2 9786756451978675645197867564519786756451978675645197867564519786756451978675645197867564519786756451978675645197867564519786756451978675645197867564519786756451978675645197867564519786756451978675645197867564519786756451978675645197867564519786756451 6821527182682152718268215271826821527182682152718268215271826821527182682152718268215271826821527182682152718268215271826821527182682152718268215271826821527182682152718268215271826821527182682152718268215271826821527182682152718268215271826821527182 4185725184418572518441857251844185725184418572518441857251844185725184418572518441857251844185725184418572518441857251844185725184418572518441857251844185725184418572518441857251844185725184418572518441857251844185725184418572518441857251844185725184 7483634195748363419574836341957483634195748363419574836341957483634195748363419574836341957483634195748363419574836341957483634195748363419574836341957483634195748363419574836341957483634195748363419574836341957483634195748363419574836341957483634195", - "3 delete only O-6821527181682152718168215271816821527181682152718168215271816821527181682152718168215271816821527181682152718168215271816821527181682152718168215271816821527181682152718168215271816821527181682152718168215271816821527181682152718168215271816821527181 4185725183418572518341857251834185725183418572518341857251834185725183418572518341857251834185725183418572518341857251834185725183418572518341857251834185725183418572518341857251834185725183418572518341857251834185725183418572518341857251834185725183 7483634194748363419474836341947483634194748363419474836341947483634194748363419474836341947483634194748363419474836341947483634194748363419474836341947483634194748363419474836341947483634194748363419474836341947483634194748363419474836341947483634194", - "4 delete only NO-6821527180682152718068215271806821527180682152718068215271806821527180682152718068215271806821527180682152718068215271806821527180682152718068215271806821527180682152718068215271806821527180682152718068215271806821527180682152718068215271806821527180 4185725182418572518241857251824185725182418572518241857251824185725182418572518241857251824185725182418572518241857251824185725182418572518241857251824185725182418572518241857251824185725182418572518241857251824185725182418572518241857251824185725182 7483634193748363419374836341937483634193748363419374836341937483634193748363419374836341937483634193748363419374836341937483634193748363419374836341937483634193748363419374836341937483634193748363419374836341937483634193748363419374836341937483634193", - "5 write only O-6821527179682152717968215271796821527179682152717968215271796821527179682152717968215271796821527179682152717968215271796821527179682152717968215271796821527179682152717968215271796821527179682152717968215271796821527179682152717968215271796821527179 4185725181418572518141857251814185725181418572518141857251814185725181418572518141857251814185725181418572518141857251814185725181418572518141857251814185725181418572518141857251814185725181418572518141857251814185725181418572518141857251814185725181 7483634192748363419274836341927483634192748363419274836341927483634192748363419274836341927483634192748363419274836341927483634192748363419274836341927483634192748363419274836341927483634192748363419274836341927483634192748363419274836341927483634192", - "6 write only NO-6821527178682152717868215271786821527178682152717868215271786821527178682152717868215271786821527178682152717868215271786821527178682152717868215271786821527178682152717868215271786821527178682152717868215271786821527178682152717868215271786821527178 4185725180418572518041857251804185725180418572518041857251804185725180418572518041857251804185725180418572518041857251804185725180418572518041857251804185725180418572518041857251804185725180418572518041857251804185725180418572518041857251804185725180 7483634191748363419174836341917483634191748363419174836341917483634191748363419174836341917483634191748363419174836341917483634191748363419174836341917483634191748363419174836341917483634191748363419174836341917483634191748363419174836341917483634191", - "7 write reorganization O-6821527177682152717768215271776821527177682152717768215271776821527177682152717768215271776821527177682152717768215271776821527177682152717768215271776821527177682152717768215271776821527177682152717768215271776821527177682152717768215271776821527177 4185725179418572517941857251794185725179418572517941857251794185725179418572517941857251794185725179418572517941857251794185725179418572517941857251794185725179418572517941857251794185725179418572517941857251794185725179418572517941857251794185725179 7483634190748363419074836341907483634190748363419074836341907483634190748363419074836341907483634190748363419074836341907483634190748363419074836341907483634190748363419074836341907483634190748363419074836341907483634190748363419074836341907483634190", - "8 write reorganization NO-6821527176682152717668215271766821527176682152717668215271766821527176682152717668215271766821527176682152717668215271766821527176682152717668215271766821527176682152717668215271766821527176682152717668215271766821527176682152717668215271766821527176 4185725178418572517841857251784185725178418572517841857251784185725178418572517841857251784185725178418572517841857251784185725178418572517841857251784185725178418572517841857251784185725178418572517841857251784185725178418572517841857251784185725178 7483634189748363418974836341897483634189748363418974836341897483634189748363418974836341897483634189748363418974836341897483634189748363418974836341897483634189748363418974836341897483634189748363418974836341897483634189748363418974836341897483634189", - "9 delete reorganization O-6821527175682152717568215271756821527175682152717568215271756821527175682152717568215271756821527175682152717568215271756821527175682152717568215271756821527175682152717568215271756821527175682152717568215271756821527175682152717568215271756821527175 4185725177418572517741857251774185725177418572517741857251774185725177418572517741857251774185725177418572517741857251774185725177418572517741857251774185725177418572517741857251774185725177418572517741857251774185725177418572517741857251774185725177 7483634188748363418874836341887483634188748363418874836341887483634188748363418874836341887483634188748363418874836341887483634188748363418874836341887483634188748363418874836341887483634188748363418874836341887483634188748363418874836341887483634188")) + require.Equal(t, int(7*2+1), i) + tkO.MustQuery(`select c1,_tidb_rowid,c2 from t`).Sort().Check(testkit.Rows(""+ + "1 60001 init O", + "10 30004 delete reorganization NO", + "11 7 public O", + "12 30005 public NO", + "13 8 none O", + "14 30006 none NO", + "2 60002 init O", + "3 60003 delete only O", + "4 60004 delete only NO", + "5 4 write only O", + // Before, there were a DUPLICATE ROW here!!! + //"5 60004 write only O", + "6 60005 write only NO", + "7 5 write reorganization O", + "8 30003 write reorganization NO", + "9 6 delete reorganization O")) } runMultiSchemaTest(t, createSQL, alterSQL, initFn, postFn, loopFn) } +func TestMultiSchemaReorganizeNoPKBackfillDML(t *testing.T) { + createSQL := `create table t (c1 INT, c2 CHAR(255), c3 CHAR(255), c4 CHAR(255), c5 CHAR(255)) partition by range (c1) (partition p1 values less than (200), partition pMax values less than (maxvalue))` + i := 1 + initFn := func(tkO *testkit.TestKit) { + tkO.MustExec(fmt.Sprintf(`insert into t values (%d,'%s','Original',repeat('%d', 25),repeat('%d', 25))`, i, "init O", 4185725186-i, 7483634197-i)) + i++ + tkO.MustExec(fmt.Sprintf(`insert into t values (%d,'%s','Original',repeat('%d', 25),repeat('%d', 25))`, i, "init O", 4185725186-i, 7483634197-i)) + i++ + } + alterSQL := `alter table t reorganize partition p1 into (partition p0 values less than (100), partition p1 values less than (200))` + loopFn := func(tkO, tkNO *testkit.TestKit) { + res := tkO.MustQuery(`select schema_state from information_schema.DDL_JOBS where table_name = 't' order by job_id desc limit 1`) + schemaState := res.Rows()[0][0].(string) + tkO.MustExec(fmt.Sprintf(`insert into t values (%d,'%s','Original',repeat('%d', 25),repeat('%d', 25))`, i, schemaState+" O", 4185725186-i, 7483634197-i)) + i++ + tkNO.MustExec(fmt.Sprintf(`insert into t values (%d,'%s','Original',repeat('%d', 25),repeat('%d', 25))`, i, schemaState+" NO", 4185725186-i, 7483634197-i)) + i++ + } + postFn := func(tkO *testkit.TestKit, _ kv.Storage) { + require.Equal(t, int(7*2+1), i) + tkO.MustQuery(`select c1,_tidb_rowid,c2,c3 from t`).Sort().Check(testkit.Rows(""+ + "1 1 init O updated", + "10 30004 delete reorganization NO Original", + "11 7 public O Original", + "12 30005 public NO Original", + "13 8 none O Original", + "14 30006 none NO Original", + "2 2 init O updated", + "3 3 delete only O updated", + "4 30001 delete only NO updated", + "5 4 write only O updated", + "6 30002 write only NO updated", + "7 5 write reorganization O Original", + "8 30003 write reorganization NO Original", + "9 6 delete reorganization O Original")) + } + runMultiSchemaTestWithBackfillDML(t, createSQL, alterSQL, "update t set c3 = 'updated'", initFn, postFn, loopFn) +} + // TestMultiSchemaTruncatePartitionWithGlobalIndex to show behavior when // truncating a partition with a global index func TestMultiSchemaTruncatePartitionWithGlobalIndex(t *testing.T) { From 43d929801889626bb40d27a1dd332edecaf81c18 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Thu, 9 Jan 2025 19:18:40 +0800 Subject: [PATCH 21/27] infoschema: avoid panic when updating infoschema v2 btree conflicts (#58815) close pingcap/tidb#58712 --- pkg/infoschema/infoschema_v2.go | 32 ++++++++++++++++++++++---------- 1 file changed, 22 insertions(+), 10 deletions(-) diff --git a/pkg/infoschema/infoschema_v2.go b/pkg/infoschema/infoschema_v2.go index 5512ee35c97b6..cef3ca762e867 100644 --- a/pkg/infoschema/infoschema_v2.go +++ b/pkg/infoschema/infoschema_v2.go @@ -38,8 +38,10 @@ import ( "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/util" + "github.com/pingcap/tidb/pkg/util/logutil" "github.com/pingcap/tidb/pkg/util/size" "github.com/pingcap/tidb/pkg/util/tracing" + "go.uber.org/zap" "golang.org/x/sync/singleflight" ) @@ -77,15 +79,17 @@ type versionAndTimestamp struct { } // btreeSet updates the btree. -// It is concurrent safe for one writer and multiple reader, -// but not safe for multiple writing concurrently. +// Concurrent write is supported, but should be avoided as much as possible. func btreeSet[T any](ptr *atomic.Pointer[btree.BTreeG[T]], item T) { - var t *btree.BTreeG[T] = ptr.Load() - t2 := t.Clone() - t2.ReplaceOrInsert(item) - succ := ptr.CompareAndSwap(t, t2) - if !succ { - panic("concurrently multiple writes are not allowed") + succ := false + for !succ { + var t *btree.BTreeG[T] = ptr.Load() + t2 := t.Clone() + t2.ReplaceOrInsert(item) + succ = ptr.CompareAndSwap(t, t2) + if !succ { + logutil.BgLogger().Info("infoschema v2 btree concurrently multiple writes detected, this should be rare") + } } } @@ -317,8 +321,16 @@ func (isd *Data) GCOldVersion(schemaVersion int64) (int, int64) { byNameNew.Delete(item) byIDNew.Delete(item) } - isd.byName.CompareAndSwap(byNameOld, byNameNew) - isd.byID.CompareAndSwap(byIDOld, byIDNew) + succ1 := isd.byID.CompareAndSwap(byIDOld, byIDNew) + var succ2 bool + if succ1 { + succ2 = isd.byName.CompareAndSwap(byNameOld, byNameNew) + } + if !succ1 || !succ2 { + logutil.BgLogger().Info("infoschema v2 GCOldVersion() writes conflict, leave it to the next time.", + zap.Bool("byID success", succ1), + zap.Bool("byName success", succ2)) + } return len(deletes), total } From 0da20ee5e690419138af43909ca9f413232cb03f Mon Sep 17 00:00:00 2001 From: Henry Baba-Weiss Date: Thu, 9 Jan 2025 04:13:28 -0800 Subject: [PATCH 22/27] stmtsummary: check authUsers using the polymorphic stmtSummaryStats (#58427) close pingcap/tidb#58403 --- pkg/util/stmtsummary/reader.go | 8 +++--- .../stmtsummary/statement_summary_test.go | 28 +++++++++++++++++++ 2 files changed, 32 insertions(+), 4 deletions(-) diff --git a/pkg/util/stmtsummary/reader.go b/pkg/util/stmtsummary/reader.go index ee9bebd821c2f..e0308c9ec7243 100644 --- a/pkg/util/stmtsummary/reader.go +++ b/pkg/util/stmtsummary/reader.go @@ -139,10 +139,10 @@ func (ssr *stmtSummaryReader) SetChecker(checker *stmtSummaryChecker) { ssr.checker = checker } -func (ssr *stmtSummaryReader) isAuthed(ssbd *stmtSummaryByDigest) bool { +func (ssr *stmtSummaryReader) isAuthed(ssStats *stmtSummaryStats) bool { isAuthed := true if ssr.user != nil && !ssr.hasProcessPriv { - _, isAuthed = ssbd.cumulative.authUsers[ssr.user.Username] + _, isAuthed = ssStats.authUsers[ssr.user.Username] } return isAuthed } @@ -150,7 +150,7 @@ func (ssr *stmtSummaryReader) isAuthed(ssbd *stmtSummaryByDigest) bool { func (ssr *stmtSummaryReader) getStmtByDigestCumulativeRow(ssbd *stmtSummaryByDigest) []types.Datum { ssbd.Lock() defer ssbd.Unlock() - if !ssr.isAuthed(ssbd) { + if !ssr.isAuthed(&ssbd.cumulative) { return nil } @@ -181,7 +181,7 @@ func (ssr *stmtSummaryReader) getStmtByDigestRow(ssbd *stmtSummaryByDigest, begi func (ssr *stmtSummaryReader) getStmtByDigestElementRow(ssElement *stmtSummaryByDigestElement, ssbd *stmtSummaryByDigest) []types.Datum { ssElement.Lock() defer ssElement.Unlock() - if !ssr.isAuthed(ssbd) { + if !ssr.isAuthed(&ssElement.stmtSummaryStats) { return nil } diff --git a/pkg/util/stmtsummary/statement_summary_test.go b/pkg/util/stmtsummary/statement_summary_test.go index 3ee938c90f807..42b3096a16a87 100644 --- a/pkg/util/stmtsummary/statement_summary_test.go +++ b/pkg/util/stmtsummary/statement_summary_test.go @@ -1540,4 +1540,32 @@ func TestAccessPrivilege(t *testing.T) { reader.hasProcessPriv = true datums = reader.GetStmtSummaryHistoryRows() require.Len(t, datums, loops) + + // Test the same query digests, but run as a different user in a new statement + // summary interval. The old user should not be able to access the rows generated + // for the new user. + ssMap.beginTimeForCurInterval = time.Now().Unix() + stmtExecInfo2 := generateAnyExecInfo() + stmtExecInfo2.User = "new_user" + + for i := range loops { + stmtExecInfo2.Digest = fmt.Sprintf("digest%d", i) + ssMap.AddStatement(stmtExecInfo2) + } + + oldUser := user + newUser := &auth.UserIdentity{Username: "new_user"} + + reader.user = newUser + reader.hasProcessPriv = false + datums = reader.GetStmtSummaryCurrentRows() + require.Len(t, datums, loops) + reader.user = oldUser + reader.hasProcessPriv = false + datums = reader.GetStmtSummaryCurrentRows() + require.Len(t, datums, 0) + reader.user = oldUser + reader.hasProcessPriv = true + datums = reader.GetStmtSummaryCurrentRows() + require.Len(t, datums, loops) } From 5301602262ca7c6ccf0b3c3cb40533137c9385c6 Mon Sep 17 00:00:00 2001 From: Hangjie Mo Date: Thu, 9 Jan 2025 22:48:34 +0800 Subject: [PATCH 23/27] planner: correct plan when scan tidb related cluster table with KeepOrder (#51922) close pingcap/tidb#51723 --- pkg/planner/core/find_best_task.go | 4 + pkg/planner/core/task.go | 77 ++++++++++++++++++- .../r/infoschema/cluster_tables.result | 48 ++++++++++++ .../t/infoschema/cluster_tables.test | 10 +++ 4 files changed, 138 insertions(+), 1 deletion(-) diff --git a/pkg/planner/core/find_best_task.go b/pkg/planner/core/find_best_task.go index ac93c82bdb5f4..f999ac72f8071 100644 --- a/pkg/planner/core/find_best_task.go +++ b/pkg/planner/core/find_best_task.go @@ -785,6 +785,10 @@ func compareCandidates(sctx base.PlanContext, statsTbl *statistics.Table, prop * } func isMatchProp(ds *logicalop.DataSource, path *util.AccessPath, prop *property.PhysicalProperty) bool { + if ds.Table.Type().IsClusterTable() && !prop.IsSortItemEmpty() { + // TableScan with cluster table can't keep order. + return false + } if prop.VectorProp.VSInfo != nil && path.Index != nil && path.Index.VectorInfo != nil { if path.Index == nil || path.Index.VectorInfo == nil { return false diff --git a/pkg/planner/core/task.go b/pkg/planner/core/task.go index f7fb6ec00d232..6d12a45596aeb 100644 --- a/pkg/planner/core/task.go +++ b/pkg/planner/core/task.go @@ -21,6 +21,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/expression/aggregation" + "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/charset" @@ -1087,7 +1088,75 @@ func (p *PhysicalTopN) canPushDownToTiFlash(mppTask *MppTask) bool { return true } -// Attach2Task implements physical plan +// For https://github.com/pingcap/tidb/issues/51723, +// This function only supports `CLUSTER_SLOW_QUERY`, +// it will change plan from +// TopN -> TableReader -> TableFullScan[cop] to +// TopN -> TableReader -> Limit[cop] -> TableFullScan[cop] + keepOrder +func (p *PhysicalTopN) pushLimitDownToTiDBCop(copTsk *CopTask) (base.Task, bool) { + if copTsk.indexPlan != nil || copTsk.tablePlan == nil { + return nil, false + } + + var ( + selOnTblScan *PhysicalSelection + selSelectivity float64 + tblScan *PhysicalTableScan + err error + ok bool + ) + + copTsk.tablePlan, err = copTsk.tablePlan.Clone(p.SCtx()) + if err != nil { + return nil, false + } + finalTblScanPlan := copTsk.tablePlan + for len(finalTblScanPlan.Children()) > 0 { + selOnTblScan, _ = finalTblScanPlan.(*PhysicalSelection) + finalTblScanPlan = finalTblScanPlan.Children()[0] + } + + if tblScan, ok = finalTblScanPlan.(*PhysicalTableScan); !ok { + return nil, false + } + + // Check the table is `CLUSTER_SLOW_QUERY` or not. + if tblScan.Table.Name.O != infoschema.ClusterTableSlowLog { + return nil, false + } + + colsProp, ok := GetPropByOrderByItems(p.ByItems) + if !ok { + return nil, false + } + if len(colsProp.SortItems) != 1 || !colsProp.SortItems[0].Col.Equal(p.SCtx().GetExprCtx().GetEvalCtx(), tblScan.HandleCols.GetCol(0)) { + return nil, false + } + if selOnTblScan != nil && tblScan.StatsInfo().RowCount > 0 { + selSelectivity = selOnTblScan.StatsInfo().RowCount / tblScan.StatsInfo().RowCount + } + tblScan.Desc = colsProp.SortItems[0].Desc + tblScan.KeepOrder = true + + childProfile := copTsk.Plan().StatsInfo() + newCount := p.Offset + p.Count + stats := util.DeriveLimitStats(childProfile, float64(newCount)) + pushedLimit := PhysicalLimit{ + Count: newCount, + }.Init(p.SCtx(), stats, p.QueryBlockOffset()) + pushedLimit.SetSchema(copTsk.tablePlan.Schema()) + copTsk = attachPlan2Task(pushedLimit, copTsk).(*CopTask) + child := pushedLimit.Children()[0] + child.SetStats(child.StatsInfo().ScaleByExpectCnt(float64(newCount))) + if selSelectivity > 0 && selSelectivity < 1 { + scaledRowCount := child.StatsInfo().RowCount / selSelectivity + tblScan.SetStats(tblScan.StatsInfo().ScaleByExpectCnt(scaledRowCount)) + } + rootTask := copTsk.ConvertToRootTask(p.SCtx()) + return attachPlan2Task(p, rootTask), true +} + +// Attach2Task implements the PhysicalPlan interface. func (p *PhysicalTopN) Attach2Task(tasks ...base.Task) base.Task { t := tasks[0].Copy() cols := make([]*expression.Column, 0, len(p.ByItems)) @@ -1095,6 +1164,12 @@ func (p *PhysicalTopN) Attach2Task(tasks ...base.Task) base.Task { cols = append(cols, expression.ExtractColumns(item.Expr)...) } needPushDown := len(cols) > 0 + if copTask, ok := t.(*CopTask); ok && needPushDown && copTask.getStoreType() == kv.TiDB && len(copTask.rootTaskConds) == 0 { + newTask, changed := p.pushLimitDownToTiDBCop(copTask) + if changed { + return newTask + } + } if copTask, ok := t.(*CopTask); ok && needPushDown && p.canPushDownToTiKV(copTask) && len(copTask.rootTaskConds) == 0 { // If all columns in topN are from index plan, we push it to index plan, otherwise we finish the index plan and // push it to table plan. diff --git a/tests/integrationtest/r/infoschema/cluster_tables.result b/tests/integrationtest/r/infoschema/cluster_tables.result index bca044ad116a9..043cc263b3500 100644 --- a/tests/integrationtest/r/infoschema/cluster_tables.result +++ b/tests/integrationtest/r/infoschema/cluster_tables.result @@ -10,3 +10,51 @@ select /*+ ignore_index(t, a) */ * from t where a = 1; id a create session binding from history using plan digest '20cf414ff6bd6fff3de17a266966020e81099b9fd1a29c4fd4b8aaf212f5c2c0'; drop binding for sql digest '83de0854921816c038565229b8008f5d679d373d16bf6b2a5cacd5937e11ea21'; +explain select * from information_schema.cluster_slow_query order by time limit 1; +id estRows task access object operator info +TopN_7 1.00 root information_schema.cluster_slow_query.time, offset:0, count:1 +└─TableReader_16 1.00 root data:Limit_15 + └─Limit_15 1.00 cop[tidb] offset:0, count:1 + └─MemTableScan_14 1.00 cop[tidb] table:CLUSTER_SLOW_QUERY +explain select * from information_schema.cluster_slow_query order by time; +id estRows task access object operator info +Sort_4 10000.00 root information_schema.cluster_slow_query.time +└─TableReader_8 10000.00 root data:MemTableScan_7 + └─MemTableScan_7 10000.00 cop[tidb] table:CLUSTER_SLOW_QUERY +explain select * from information_schema.cluster_slow_query order by time desc limit 1; +id estRows task access object operator info +TopN_7 1.00 root information_schema.cluster_slow_query.time:desc, offset:0, count:1 +└─TableReader_16 1.00 root data:Limit_15 + └─Limit_15 1.00 cop[tidb] offset:0, count:1 + └─MemTableScan_14 1.00 cop[tidb] table:CLUSTER_SLOW_QUERY +explain select * from information_schema.cluster_slow_query order by time desc; +id estRows task access object operator info +Sort_4 10000.00 root information_schema.cluster_slow_query.time:desc +└─TableReader_8 10000.00 root data:MemTableScan_7 + └─MemTableScan_7 10000.00 cop[tidb] table:CLUSTER_SLOW_QUERY +explain select * from information_schema.cluster_slow_query WHERE (time between '2020-09-24 15:23:41.421396' and '2020-09-25 17:57:35.047111') and query != 'x' order by time limit 1; +id estRows task access object operator info +TopN_8 1.00 root information_schema.cluster_slow_query.time, offset:0, count:1 +└─TableReader_18 1.00 root data:Limit_17 + └─Limit_17 1.00 cop[tidb] offset:0, count:1 + └─Selection_16 1.00 cop[tidb] ne(information_schema.cluster_slow_query.query, "x") + └─MemTableScan_15 1.50 cop[tidb] table:CLUSTER_SLOW_QUERY +explain select * from information_schema.cluster_slow_query WHERE (time between '2020-09-24 15:23:41.421396' and '2020-09-25 17:57:35.047111') and query != 'x' order by time; +id estRows task access object operator info +Sort_5 166.42 root information_schema.cluster_slow_query.time +└─TableReader_10 166.42 root data:Selection_9 + └─Selection_9 166.42 cop[tidb] ne(information_schema.cluster_slow_query.query, "x") + └─MemTableScan_8 250.00 cop[tidb] table:CLUSTER_SLOW_QUERY +explain select * from information_schema.cluster_slow_query WHERE (time between '2020-09-24 15:23:41.421396' and '2020-09-25 17:57:35.047111') and query != 'x' order by time desc limit 1; +id estRows task access object operator info +TopN_8 1.00 root information_schema.cluster_slow_query.time:desc, offset:0, count:1 +└─TableReader_18 1.00 root data:Limit_17 + └─Limit_17 1.00 cop[tidb] offset:0, count:1 + └─Selection_16 1.00 cop[tidb] ne(information_schema.cluster_slow_query.query, "x") + └─MemTableScan_15 1.50 cop[tidb] table:CLUSTER_SLOW_QUERY +explain select * from information_schema.cluster_slow_query WHERE (time between '2020-09-24 15:23:41.421396' and '2020-09-25 17:57:35.047111') and query != 'x' order by time desc; +id estRows task access object operator info +Sort_5 166.42 root information_schema.cluster_slow_query.time:desc +└─TableReader_10 166.42 root data:Selection_9 + └─Selection_9 166.42 cop[tidb] ne(information_schema.cluster_slow_query.query, "x") + └─MemTableScan_8 250.00 cop[tidb] table:CLUSTER_SLOW_QUERY diff --git a/tests/integrationtest/t/infoschema/cluster_tables.test b/tests/integrationtest/t/infoschema/cluster_tables.test index 020221645c7d9..2ef15d299b93a 100644 --- a/tests/integrationtest/t/infoschema/cluster_tables.test +++ b/tests/integrationtest/t/infoschema/cluster_tables.test @@ -10,3 +10,13 @@ select /*+ ignore_index(t, a) */ * from t where a = 1; create session binding from history using plan digest '20cf414ff6bd6fff3de17a266966020e81099b9fd1a29c4fd4b8aaf212f5c2c0'; drop binding for sql digest '83de0854921816c038565229b8008f5d679d373d16bf6b2a5cacd5937e11ea21'; +# TestIssue51723 +explain select * from information_schema.cluster_slow_query order by time limit 1; +explain select * from information_schema.cluster_slow_query order by time; +explain select * from information_schema.cluster_slow_query order by time desc limit 1; +explain select * from information_schema.cluster_slow_query order by time desc; +explain select * from information_schema.cluster_slow_query WHERE (time between '2020-09-24 15:23:41.421396' and '2020-09-25 17:57:35.047111') and query != 'x' order by time limit 1; +explain select * from information_schema.cluster_slow_query WHERE (time between '2020-09-24 15:23:41.421396' and '2020-09-25 17:57:35.047111') and query != 'x' order by time; +explain select * from information_schema.cluster_slow_query WHERE (time between '2020-09-24 15:23:41.421396' and '2020-09-25 17:57:35.047111') and query != 'x' order by time desc limit 1; +explain select * from information_schema.cluster_slow_query WHERE (time between '2020-09-24 15:23:41.421396' and '2020-09-25 17:57:35.047111') and query != 'x' order by time desc; + From 244062f826f75494525e3ac319d0e8ad794b0a43 Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Fri, 10 Jan 2025 00:29:25 +0800 Subject: [PATCH 24/27] planner: allow semi_join_rewrite hint to work for in-subquery (#58830) close pingcap/tidb#58829 --- .../physicalplantest/testdata/plan_suite_out.json | 4 ++-- pkg/planner/core/expression_rewriter.go | 3 ++- pkg/planner/core/integration_test.go | 12 ++++++++++++ pkg/planner/core/logical_plan_builder.go | 1 + pkg/util/hint/hint.go | 6 +++--- 5 files changed, 20 insertions(+), 6 deletions(-) diff --git a/pkg/planner/core/casetest/physicalplantest/testdata/plan_suite_out.json b/pkg/planner/core/casetest/physicalplantest/testdata/plan_suite_out.json index 16bf9bc7f0ba3..6224962aa4adf 100644 --- a/pkg/planner/core/casetest/physicalplantest/testdata/plan_suite_out.json +++ b/pkg/planner/core/casetest/physicalplantest/testdata/plan_suite_out.json @@ -3179,7 +3179,7 @@ "TableReader 10000.00 root data:TableFullScan", "└─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" ], - "Warning": "[planner:1815]The SEMI_JOIN_REWRITE hint is not used correctly, maybe it's not in a subquery or the subquery is not EXISTS clause." + "Warning": "[planner:1815]The SEMI_JOIN_REWRITE hint is not used correctly, maybe it's not in a subquery or the subquery is not IN/EXISTS clause." }, { "SQL": "select * from t where a > (select /*+ SEMI_JOIN_REWRITE() */ min(b) from t t1 where t1.c = t.c)", @@ -3195,7 +3195,7 @@ " └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.c))", " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" ], - "Warning": "[planner:1815]The SEMI_JOIN_REWRITE hint is not used correctly, maybe it's not in a subquery or the subquery is not EXISTS clause." + "Warning": "[planner:1815]The SEMI_JOIN_REWRITE hint is not used correctly, maybe it's not in a subquery or the subquery is not IN/EXISTS clause." }, { "SQL": "select * from t where exists (select /*+ SEMI_JOIN_REWRITE() */ 1 from t t1 where t1.a=t.a)", diff --git a/pkg/planner/core/expression_rewriter.go b/pkg/planner/core/expression_rewriter.go index 80669d0870933..c6bbb4d511b4d 100644 --- a/pkg/planner/core/expression_rewriter.go +++ b/pkg/planner/core/expression_rewriter.go @@ -1256,7 +1256,8 @@ func (er *expressionRewriter) handleInSubquery(ctx context.Context, planCtx *exp } planCtx.plan = join } else { - planCtx.plan, er.err = planCtx.builder.buildSemiApply(planCtx.plan, np, expression.SplitCNFItems(checkCondition), asScalar, v.Not, false, noDecorrelate) + semiRewrite := hintFlags&hint.HintFlagSemiJoinRewrite > 0 + planCtx.plan, er.err = planCtx.builder.buildSemiApply(planCtx.plan, np, expression.SplitCNFItems(checkCondition), asScalar, v.Not, semiRewrite, noDecorrelate) if er.err != nil { return v, true } diff --git a/pkg/planner/core/integration_test.go b/pkg/planner/core/integration_test.go index 4efe5629186e6..635db5f26e221 100644 --- a/pkg/planner/core/integration_test.go +++ b/pkg/planner/core/integration_test.go @@ -2321,3 +2321,15 @@ func TestNestedVirtualGeneratedColumnUpdate(t *testing.T) { tk.MustExec("UPDATE test1 SET col7 = '{\"col10\":\"DDDDD\",\"col9\":[\"abcdefg\"]}';\n") tk.MustExec("DELETE FROM test1 WHERE col1 < 0;\n") } + +func TestIssue58829(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + + tk.MustExec(`create table t1 (id varchar(64) not null, key(id))`) + tk.MustExec(`create table t2 (id bigint(20), k int)`) + + // the semi_join_rewrite hint can convert the semi-join to inner-join and finally allow the optimizer to choose the IndexJoin + tk.MustHavePlan(`delete from t1 where t1.id in (select /*+ semi_join_rewrite() */ cast(id as char) from t2 where k=1)`, "IndexHashJoin") +} diff --git a/pkg/planner/core/logical_plan_builder.go b/pkg/planner/core/logical_plan_builder.go index cd5058a5a785a..1e4465ad31e0e 100644 --- a/pkg/planner/core/logical_plan_builder.go +++ b/pkg/planner/core/logical_plan_builder.go @@ -3645,6 +3645,7 @@ func (b *PlanBuilder) pushTableHints(hints []*ast.TableOptimizerHint, currentLev warnHandler := b.ctx.GetSessionVars().StmtCtx planHints, subQueryHintFlags, err := h.ParsePlanHints(hints, currentLevel, currentDB, b.hintProcessor, b.ctx.GetSessionVars().StmtCtx.StraightJoinOrder, + b.subQueryCtx == handlingInSubquery, b.subQueryCtx == handlingExistsSubquery, b.subQueryCtx == notHandlingSubquery, warnHandler) if err != nil { return diff --git a/pkg/util/hint/hint.go b/pkg/util/hint/hint.go index 88c40c7042b6c..d6ce62cba80fb 100644 --- a/pkg/util/hint/hint.go +++ b/pkg/util/hint/hint.go @@ -728,7 +728,7 @@ func (*PlanHints) MatchTableName(tables []*HintedTable, hintTables []HintedTable func ParsePlanHints(hints []*ast.TableOptimizerHint, currentLevel int, currentDB string, hintProcessor *QBHintHandler, straightJoinOrder bool, - handlingExistsSubquery, notHandlingSubquery bool, + handlingInSubquery, handlingExistsSubquery, notHandlingSubquery bool, warnHandler hintWarnHandler) (p *PlanHints, subQueryHintFlags uint64, err error) { var ( sortMergeTables, inljTables, inlhjTables, inlmjTables, hashJoinTables, bcTables []HintedTable @@ -872,8 +872,8 @@ func ParsePlanHints(hints []*ast.TableOptimizerHint, } leadingHintCnt++ case HintSemiJoinRewrite: - if !handlingExistsSubquery { - warnHandler.SetHintWarning("The SEMI_JOIN_REWRITE hint is not used correctly, maybe it's not in a subquery or the subquery is not EXISTS clause.") + if !handlingExistsSubquery && !handlingInSubquery { + warnHandler.SetHintWarning("The SEMI_JOIN_REWRITE hint is not used correctly, maybe it's not in a subquery or the subquery is not IN/EXISTS clause.") continue } subQueryHintFlags |= HintFlagSemiJoinRewrite From d9dc3f4a3ff2ba6454548c6526bb9dac41a7f9bc Mon Sep 17 00:00:00 2001 From: Wenxuan Date: Fri, 10 Jan 2025 04:01:43 +0900 Subject: [PATCH 25/27] planner: Fix vector not truncated after CBO (#58809) close pingcap/tidb#58837 --- .../integration_test/integration_test.go | 71 +++++++++++++++++++ pkg/planner/core/explain.go | 2 +- 2 files changed, 72 insertions(+), 1 deletion(-) diff --git a/pkg/expression/integration_test/integration_test.go b/pkg/expression/integration_test/integration_test.go index f0984cc1bb2f1..98538714b2a1a 100644 --- a/pkg/expression/integration_test/integration_test.go +++ b/pkg/expression/integration_test/integration_test.go @@ -418,6 +418,24 @@ func TestVectorColumnInfo(t *testing.T) { tk.MustGetErrMsg("create table t(embedding VECTOR(16384))", "vector cannot have more than 16383 dimensions") } +func TestVectorExplainTruncate(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("CREATE TABLE t(c VECTOR);") + + // TODO: The output can be improved. + tk.MustQuery(`EXPLAIN format='brief' SELECT + VEC_COSINE_DISTANCE(c, '[3,100,12345,10000]'), + VEC_COSINE_DISTANCE(c, '[11111111111,11111111111.23456789,3.1,5.12456]'), + VEC_COSINE_DISTANCE(c, '[-11111111111,-11111111111.23456789,-3.1,-5.12456]') + FROM t;`).Check(testkit.Rows( + `Projection 10000.00 root vec_cosine_distance(test.t.c, [3,1e+02,1.2e+04,1e+04])->Column#3, vec_cosine_distance(test.t.c, [1.1e+10,1.1e+10,3.1,5.1])->Column#4, vec_cosine_distance(test.t.c, [-1.1e+10,-1.1e+10,-3.1,-5.1])->Column#5`, + `└─TableReader 10000.00 root data:TableFullScan`, + ` └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo`, + )) +} + func TestVectorConstantExplain(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -475,6 +493,59 @@ func TestVectorConstantExplain(t *testing.T) { tk.ResultSetToResult(rs, fmt.Sprintf("%v", rs)) } +func TestVectorIndexExplain(t *testing.T) { + store := testkit.CreateMockStoreWithSchemaLease(t, 1*time.Second, mockstore.WithMockTiFlash(2)) + + tk := testkit.NewTestKit(t, store) + + tiflash := infosync.NewMockTiFlash() + infosync.SetMockTiFlash(tiflash) + defer func() { + tiflash.Lock() + tiflash.StatusServer.Close() + tiflash.Unlock() + }() + + failpoint.Enable("github.com/pingcap/tidb/pkg/ddl/MockCheckVectorIndexProcess", `return(1)`) + defer func() { + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/ddl/MockCheckVectorIndexProcess")) + }() + + tk.MustExec("use test") + tk.MustExec("drop table if exists t1") + tk.MustExec(` + create table t1 ( + vec vector(100) + ) + `) + tk.MustExec("alter table t1 set tiflash replica 1;") + tk.MustExec("alter table t1 add vector index ((vec_cosine_distance(vec))) USING HNSW;") + tbl, _ := domain.GetDomain(tk.Session()).InfoSchema().TableByName(context.Background(), ast.NewCIStr("test"), ast.NewCIStr("t1")) + tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + + vb := strings.Builder{} + vb.WriteString("[") + for i := 0; i < 100; i++ { + if i > 0 { + vb.WriteString(",") + } + vb.WriteString("100") + } + vb.WriteString("]") + + tk.MustQuery(fmt.Sprintf("explain format = 'brief' select * from t1 order by vec_cosine_distance(vec, '%s') limit 1", vb.String())).Check(testkit.Rows( + `TopN 1.00 root Column#5, offset:0, count:1`, + `└─TableReader 1.00 root MppVersion: 2, data:ExchangeSender`, + ` └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough`, + ` └─TopN 1.00 mpp[tiflash] Column#5, offset:0, count:1`, + ` └─Projection 1.00 mpp[tiflash] test.t1.vec, vec_cosine_distance(test.t1.vec, [1e+02,1e+02,1e+02,1e+02,1e+02,(95 more)...])->Column#5`, + ` └─TableFullScan 1.00 mpp[tiflash] table:t1, index:vector_index(vec) keep order:false, stats:pseudo, annIndex:COSINE(vec..[1e+02,1e+02,1e+02,1e+02,1e+02,(95 more)...], limit:1)`, + )) +} + func TestFixedVector(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) diff --git a/pkg/planner/core/explain.go b/pkg/planner/core/explain.go index 191d09f61caee..0ebbfe2fcf73e 100644 --- a/pkg/planner/core/explain.go +++ b/pkg/planner/core/explain.go @@ -284,7 +284,7 @@ func (p *PhysicalTableScan) OperatorInfo(normalized bool) string { if err != nil { buffer.WriteString("[?]") } else { - buffer.WriteString(v.String()) + buffer.WriteString(v.TruncatedString()) } } buffer.WriteString(", limit:") From 837aec49eb28025aacec7f50f69b73fb423b590a Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Fri, 10 Jan 2025 12:09:58 +0800 Subject: [PATCH 26/27] OWNERS: Auto Sync OWNERS files from community membership (#58854) --- OWNERS | 2 ++ 1 file changed, 2 insertions(+) diff --git a/OWNERS b/OWNERS index 4cb737923ec42..e8b11ad96687c 100644 --- a/OWNERS +++ b/OWNERS @@ -143,6 +143,7 @@ reviewers: - nolouch - nongfushanquan - pingyu + - RidRisR - rleungx - SeaRise - shihongzhi @@ -151,6 +152,7 @@ reviewers: - terry1purcell - Tjianke - TonsnakeLin + - Tristan1900 - tsthght - TszKitLo40 - xzhangxian1008 From 1d0c33a167d6080095d3fe13b45977ada0cbc987 Mon Sep 17 00:00:00 2001 From: Zequan <110292382+EricZequan@users.noreply.github.com> Date: Fri, 10 Jan 2025 12:51:58 +0800 Subject: [PATCH 27/27] types: throw error when input exceeds the range of float32 in vector (#58841) ref pingcap/tidb#58379 --- pkg/expression/integration_test/integration_test.go | 3 +++ pkg/types/vector.go | 5 +++++ pkg/types/vector_test.go | 3 +++ 3 files changed, 11 insertions(+) diff --git a/pkg/expression/integration_test/integration_test.go b/pkg/expression/integration_test/integration_test.go index 98538714b2a1a..63b7078bf92e7 100644 --- a/pkg/expression/integration_test/integration_test.go +++ b/pkg/expression/integration_test/integration_test.go @@ -961,9 +961,12 @@ func TestVectorArithmatic(t *testing.T) { tk.MustQueryToErr(`SELECT VEC_FROM_TEXT('[1]') + 2;`) tk.MustQueryToErr(`SELECT VEC_FROM_TEXT('[1]') + '2';`) + // Input outside the float32 value range will result in an error. tk.MustQueryToErr(`SELECT VEC_FROM_TEXT('[3e38]') + '[3e38]';`) tk.MustQuery(`SELECT VEC_FROM_TEXT('[1,2,3]') * '[4,5,6]';`).Check(testkit.Rows("[4,10,18]")) tk.MustQueryToErr(`SELECT VEC_FROM_TEXT('[1e37]') * '[1e37]';`) + tk.MustQueryToErr("select VEC_L2_NORM('[1e39]') + 1") + tk.MustQueryToErr("select VEC_L2_NORM('[1e39]')*0 + 1") } func TestVectorFunctions(t *testing.T) { diff --git a/pkg/types/vector.go b/pkg/types/vector.go index 479fa08d32d43..9791abce5882f 100644 --- a/pkg/types/vector.go +++ b/pkg/types/vector.go @@ -233,6 +233,11 @@ func ParseVectorFloat32(s string) (VectorFloat32, error) { valueError = errors.Errorf("infinite value not allowed in vector") return false } + // Check if the value can be safely converted to float32 + if v < -math.MaxFloat32 || v > math.MaxFloat32 { + valueError = errors.Errorf("value %v out of range for float32", v) + return false + } values = append(values, float32(v)) return true }) diff --git a/pkg/types/vector_test.go b/pkg/types/vector_test.go index d87de14fad713..44c164dcb4451 100644 --- a/pkg/types/vector_test.go +++ b/pkg/types/vector_test.go @@ -101,6 +101,9 @@ func TestVectorParse(t *testing.T) { require.False(t, v.IsZeroValue()) require.Equal(t, 1, v.Compare(types.ZeroVectorFloat32)) require.Equal(t, -1, types.ZeroVectorFloat32.Compare(v)) + + v, err = types.ParseVectorFloat32(`[-1e39, 1e39]`) + require.EqualError(t, err, "value -1e+39 out of range for float32") } func TestVectorDatum(t *testing.T) {