From ba456b811a2791a2898eaa137495187c22aeeb65 Mon Sep 17 00:00:00 2001 From: Chao Wang Date: Thu, 26 Sep 2024 21:28:21 +0800 Subject: [PATCH] *: remove `mock.NewContext()` usage when building table meta in production code --- cmd/importer/BUILD.bazel | 2 +- cmd/importer/parser.go | 5 ++-- lightning/pkg/importer/BUILD.bazel | 3 ++- lightning/pkg/importer/chunk_process_test.go | 3 ++- lightning/pkg/importer/get_pre_info.go | 6 ++--- pkg/ddl/create_table.go | 5 ++-- pkg/infoschema/perfschema/BUILD.bazel | 1 + pkg/infoschema/perfschema/init.go | 5 +++- pkg/meta/metabuild/context.go | 11 ++++++++ pkg/planner/core/BUILD.bazel | 2 +- pkg/planner/core/partition_pruning_test.go | 6 ++--- pkg/session/BUILD.bazel | 1 + pkg/session/session.go | 5 ++-- pkg/table/tables/BUILD.bazel | 1 + pkg/table/tables/index_test.go | 3 ++- pkg/testkit/ddlhelper/BUILD.bazel | 13 ---------- pkg/testkit/ddlhelper/helper.go | 27 -------------------- pkg/util/dbutil/dbutiltest/BUILD.bazel | 2 +- pkg/util/dbutil/dbutiltest/utils.go | 20 ++------------- 19 files changed, 43 insertions(+), 78 deletions(-) delete mode 100644 pkg/testkit/ddlhelper/BUILD.bazel delete mode 100644 pkg/testkit/ddlhelper/helper.go diff --git a/cmd/importer/BUILD.bazel b/cmd/importer/BUILD.bazel index bf0fc5857c225..cd8de53165ea9 100644 --- a/cmd/importer/BUILD.bazel +++ b/cmd/importer/BUILD.bazel @@ -16,6 +16,7 @@ go_library( visibility = ["//visibility:private"], deps = [ "//pkg/ddl", + "//pkg/meta/metabuild", "//pkg/meta/model", "//pkg/parser", "//pkg/parser/ast", @@ -26,7 +27,6 @@ go_library( "//pkg/statistics/handle/util", "//pkg/types", "//pkg/util/mathutil", - "//pkg/util/mock", "@com_github_burntsushi_toml//:toml", "@com_github_go_sql_driver_mysql//:mysql", "@com_github_pingcap_errors//:errors", diff --git a/cmd/importer/parser.go b/cmd/importer/parser.go index 158bfda152215..9ef2a538b28d3 100644 --- a/cmd/importer/parser.go +++ b/cmd/importer/parser.go @@ -22,12 +22,12 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" "github.com/pingcap/tidb/pkg/ddl" + "github.com/pingcap/tidb/pkg/meta/metabuild" "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/planner/core" "github.com/pingcap/tidb/pkg/types" - "github.com/pingcap/tidb/pkg/util/mock" "go.uber.org/zap" ) @@ -237,7 +237,8 @@ func parseTable(t *table, stmt *ast.CreateTableStmt) error { t.name = stmt.Table.Name.L t.columns = make([]*column, 0, len(stmt.Cols)) - mockTbl, err := ddl.MockTableInfo(mock.NewContext(), stmt, 1) + mockTbl, err := ddl.BuildTableInfoFromAST(metabuild.NewNonStrictContext(), stmt) + mockTbl.ID = 1 if err != nil { return errors.Trace(err) } diff --git a/lightning/pkg/importer/BUILD.bazel b/lightning/pkg/importer/BUILD.bazel index 0dfb42e114c84..dc25bdc18b42d 100644 --- a/lightning/pkg/importer/BUILD.bazel +++ b/lightning/pkg/importer/BUILD.bazel @@ -52,6 +52,7 @@ go_library( "//pkg/lightning/verification", "//pkg/lightning/worker", "//pkg/meta/autoid", + "//pkg/meta/metabuild", "//pkg/meta/model", "//pkg/parser", "//pkg/parser/ast", @@ -72,7 +73,6 @@ go_library( "//pkg/util/engine", "//pkg/util/etcd", "//pkg/util/extsort", - "//pkg/util/mock", "//pkg/util/redact", "//pkg/util/regexpr-router", "//pkg/util/set", @@ -150,6 +150,7 @@ go_test( "//pkg/lightning/worker", "//pkg/meta", "//pkg/meta/autoid", + "//pkg/meta/metabuild", "//pkg/meta/model", "//pkg/parser", "//pkg/parser/ast", diff --git a/lightning/pkg/importer/chunk_process_test.go b/lightning/pkg/importer/chunk_process_test.go index c539b353624dd..32afc724e95ea 100644 --- a/lightning/pkg/importer/chunk_process_test.go +++ b/lightning/pkg/importer/chunk_process_test.go @@ -41,6 +41,7 @@ import ( "github.com/pingcap/tidb/pkg/lightning/log" "github.com/pingcap/tidb/pkg/lightning/mydump" "github.com/pingcap/tidb/pkg/lightning/worker" + "github.com/pingcap/tidb/pkg/meta/metabuild" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser" "github.com/pingcap/tidb/pkg/parser/ast" @@ -710,7 +711,7 @@ func TestCompressChunkRestore(t *testing.T) { ) `, "", "") require.NoError(t, err) - core, err := ddl.BuildTableInfoFromAST(node.(*ast.CreateTableStmt)) + core, err := ddl.BuildTableInfoFromAST(metabuild.NewContext(), node.(*ast.CreateTableStmt)) require.NoError(t, err) core.State = model.StatePublic diff --git a/lightning/pkg/importer/get_pre_info.go b/lightning/pkg/importer/get_pre_info.go index 09d45de143630..db8e569e5f95b 100644 --- a/lightning/pkg/importer/get_pre_info.go +++ b/lightning/pkg/importer/get_pre_info.go @@ -43,6 +43,7 @@ import ( "github.com/pingcap/tidb/pkg/lightning/mydump" "github.com/pingcap/tidb/pkg/lightning/verification" "github.com/pingcap/tidb/pkg/lightning/worker" + "github.com/pingcap/tidb/pkg/meta/metabuild" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser" "github.com/pingcap/tidb/pkg/parser/ast" @@ -50,7 +51,6 @@ import ( "github.com/pingcap/tidb/pkg/table/tables" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/dbterror" - "github.com/pingcap/tidb/pkg/util/mock" pdhttp "github.com/tikv/pd/client/http" "go.uber.org/zap" ) @@ -428,15 +428,15 @@ func newTableInfo(createTblSQL string, tableID int64) (*model.TableInfo, error) log.L().Error(errMsg, zap.Error(err), zap.String("sql", createTblSQL)) return nil, errors.Trace(err) } - sctx := mock.NewContext() createTableStmt, ok := astNode.(*ast.CreateTableStmt) if !ok { return nil, errors.New("cannot transfer the parsed SQL as an CREATE TABLE statement") } - info, err := ddl.MockTableInfo(sctx, createTableStmt, tableID) + info, err := ddl.BuildTableInfoFromAST(metabuild.NewNonStrictContext(), createTableStmt) if err != nil { return nil, errors.Trace(err) } + info.ID = tableID info.State = model.StatePublic return info, nil } diff --git a/pkg/ddl/create_table.go b/pkg/ddl/create_table.go index 42a01fb41e121..804bd369b2a4b 100644 --- a/pkg/ddl/create_table.go +++ b/pkg/ddl/create_table.go @@ -48,7 +48,6 @@ import ( "github.com/pingcap/tidb/pkg/types" driver "github.com/pingcap/tidb/pkg/types/parser_driver" "github.com/pingcap/tidb/pkg/util/dbterror" - "github.com/pingcap/tidb/pkg/util/mock" "github.com/pingcap/tidb/pkg/util/set" "go.uber.org/zap" ) @@ -390,8 +389,8 @@ func findTableIDFromStore(t *meta.Meta, schemaID int64, tableName string) (int64 // BuildTableInfoFromAST builds model.TableInfo from a SQL statement. // Note: TableID and PartitionID are left as uninitialized value. -func BuildTableInfoFromAST(s *ast.CreateTableStmt) (*model.TableInfo, error) { - return buildTableInfoWithCheck(NewMetaBuildContextWithSctx(mock.NewContext()), s, mysql.DefaultCharset, "", nil) +func BuildTableInfoFromAST(ctx *metabuild.Context, s *ast.CreateTableStmt) (*model.TableInfo, error) { + return buildTableInfoWithCheck(ctx, s, mysql.DefaultCharset, "", nil) } // buildTableInfoWithCheck builds model.TableInfo from a SQL statement. diff --git a/pkg/infoschema/perfschema/BUILD.bazel b/pkg/infoschema/perfschema/BUILD.bazel index e002f5168387d..70f51f697ace1 100644 --- a/pkg/infoschema/perfschema/BUILD.bazel +++ b/pkg/infoschema/perfschema/BUILD.bazel @@ -15,6 +15,7 @@ go_library( "//pkg/infoschema", "//pkg/kv", "//pkg/meta/autoid", + "//pkg/meta/metabuild", "//pkg/meta/model", "//pkg/parser", "//pkg/parser/ast", diff --git a/pkg/infoschema/perfschema/init.go b/pkg/infoschema/perfschema/init.go index d79593b129f36..ac9faef5288d7 100644 --- a/pkg/infoschema/perfschema/init.go +++ b/pkg/infoschema/perfschema/init.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/meta/autoid" + "github.com/pingcap/tidb/pkg/meta/metabuild" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser" "github.com/pingcap/tidb/pkg/parser/ast" @@ -43,12 +44,13 @@ func Init() { p := parser.New() tbls := make([]*model.TableInfo, 0) dbID := autoid.PerformanceSchemaDBID + ctx := metabuild.NewNonStrictContext() for _, sql := range perfSchemaTables { stmt, err := p.ParseOneStmt(sql, "", "") if err != nil { panic(err) } - meta, err := ddl.BuildTableInfoFromAST(stmt.(*ast.CreateTableStmt)) + meta, err := ddl.BuildTableInfoFromAST(ctx, stmt.(*ast.CreateTableStmt)) if err != nil { panic(err) } @@ -62,6 +64,7 @@ func Init() { c.ID = int64(i) + 1 } meta.DBID = dbID + meta.State = model.StatePublic } dbInfo := &model.DBInfo{ ID: dbID, diff --git a/pkg/meta/metabuild/context.go b/pkg/meta/metabuild/context.go index faa2056cedbd2..972b32d1170cc 100644 --- a/pkg/meta/metabuild/context.go +++ b/pkg/meta/metabuild/context.go @@ -132,6 +132,17 @@ func NewContext(opts ...Option) *Context { return ctx } +// NewNonStrictContext creates a new context for meta-building with non-strict mode. +func NewNonStrictContext() *Context { + evalCtx := exprstatic.NewEvalContext( + // use mysql.ModeNone to avoid some special values like datetime `0000-00-00 00:00:00` + exprstatic.WithSQLMode(mysql.ModeNone), + ) + return NewContext(WithExprCtx(exprstatic.NewExprContext( + exprstatic.WithEvalCtx(evalCtx), + ))) +} + // GetExprCtx returns the expression context of the session. func (ctx *Context) GetExprCtx() exprctx.ExprContext { return ctx.exprCtx diff --git a/pkg/planner/core/BUILD.bazel b/pkg/planner/core/BUILD.bazel index d28544468a919..bfaeb4bc9ab80 100644 --- a/pkg/planner/core/BUILD.bazel +++ b/pkg/planner/core/BUILD.bazel @@ -270,6 +270,7 @@ go_test( shard_count = 50, deps = [ "//pkg/config", + "//pkg/ddl", "//pkg/domain", "//pkg/expression", "//pkg/expression/aggregation", @@ -306,7 +307,6 @@ go_test( "//pkg/statistics", "//pkg/table", "//pkg/testkit", - "//pkg/testkit/ddlhelper", "//pkg/testkit/external", "//pkg/testkit/testdata", "//pkg/testkit/testmain", diff --git a/pkg/planner/core/partition_pruning_test.go b/pkg/planner/core/partition_pruning_test.go index 50ddba64f980b..08d4621663d24 100644 --- a/pkg/planner/core/partition_pruning_test.go +++ b/pkg/planner/core/partition_pruning_test.go @@ -20,11 +20,11 @@ import ( "strings" "testing" + "github.com/pingcap/tidb/pkg/ddl" "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" "github.com/stretchr/testify/require" @@ -183,7 +183,7 @@ func prepareBenchCtx(createTable string, partitionExpr string) *testCtx { return nil } sctx := mock.NewContext() - tblInfo, err := ddlhelper.BuildTableInfoFromAST(stmt.(*ast.CreateTableStmt)) + tblInfo, err := ddl.BuildTableInfoFromAST(ddl.NewMetaBuildContextWithSctx(sctx), stmt.(*ast.CreateTableStmt)) if err != nil { return nil } @@ -213,7 +213,7 @@ func prepareTestCtx(t *testing.T, createTable string, partitionExpr string) *tes stmt, err := p.ParseOneStmt(createTable, "", "") require.NoError(t, err) sctx := mock.NewContext() - tblInfo, err := ddlhelper.BuildTableInfoFromAST(stmt.(*ast.CreateTableStmt)) + tblInfo, err := ddl.BuildTableInfoFromAST(ddl.NewMetaBuildContextWithSctx(sctx), stmt.(*ast.CreateTableStmt)) require.NoError(t, err) columns, names, err := expression.ColumnInfos2ColumnsAndNames(sctx, model.NewCIStr("t"), tblInfo.Name, tblInfo.Cols(), tblInfo) require.NoError(t, err) diff --git a/pkg/session/BUILD.bazel b/pkg/session/BUILD.bazel index fd5c7abfa7f7e..34b00407b45d6 100644 --- a/pkg/session/BUILD.bazel +++ b/pkg/session/BUILD.bazel @@ -44,6 +44,7 @@ go_library( "//pkg/infoschema/context", "//pkg/kv", "//pkg/meta", + "//pkg/meta/metabuild", "//pkg/meta/model", "//pkg/metrics", "//pkg/owner", diff --git a/pkg/session/session.go b/pkg/session/session.go index 61b6aa8234d98..878883efc0b81 100644 --- a/pkg/session/session.go +++ b/pkg/session/session.go @@ -62,6 +62,7 @@ import ( infoschemactx "github.com/pingcap/tidb/pkg/infoschema/context" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta" + "github.com/pingcap/tidb/pkg/meta/metabuild" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/metrics" "github.com/pingcap/tidb/pkg/owner" @@ -3253,7 +3254,7 @@ func createAndSplitTables(store kv.Storage, t *meta.Meta, dbID int64, tables []t if err != nil { return errors.Trace(err) } - tblInfo, err := ddl.BuildTableInfoFromAST(stmt.(*ast.CreateTableStmt)) + tblInfo, err := ddl.BuildTableInfoFromAST(metabuild.NewContext(), stmt.(*ast.CreateTableStmt)) if err != nil { return errors.Trace(err) } @@ -3286,7 +3287,7 @@ func InitMDLTable(store kv.Storage) error { if err != nil { return errors.Trace(err) } - tblInfo, err := ddl.BuildTableInfoFromAST(stmt.(*ast.CreateTableStmt)) + tblInfo, err := ddl.BuildTableInfoFromAST(metabuild.NewContext(), stmt.(*ast.CreateTableStmt)) if err != nil { return errors.Trace(err) } diff --git a/pkg/table/tables/BUILD.bazel b/pkg/table/tables/BUILD.bazel index 3059b66a5463b..d6519927d68bc 100644 --- a/pkg/table/tables/BUILD.bazel +++ b/pkg/table/tables/BUILD.bazel @@ -85,6 +85,7 @@ go_test( "//pkg/lightning/backend/encode", "//pkg/lightning/backend/kv", "//pkg/meta/autoid", + "//pkg/meta/metabuild", "//pkg/meta/model", "//pkg/metrics", "//pkg/parser", diff --git a/pkg/table/tables/index_test.go b/pkg/table/tables/index_test.go index be91f0987d844..ad45594184985 100644 --- a/pkg/table/tables/index_test.go +++ b/pkg/table/tables/index_test.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/lightning/backend/encode" lkv "github.com/pingcap/tidb/pkg/lightning/backend/kv" + "github.com/pingcap/tidb/pkg/meta/metabuild" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser" "github.com/pingcap/tidb/pkg/parser/ast" @@ -170,7 +171,7 @@ func TestSingleColumnCommonHandle(t *testing.T) { func buildTableInfo(t *testing.T, sql string) *model.TableInfo { stmt, err := parser.New().ParseOneStmt(sql, "", "") require.NoError(t, err) - tblInfo, err := ddl.BuildTableInfoFromAST(stmt.(*ast.CreateTableStmt)) + tblInfo, err := ddl.BuildTableInfoFromAST(metabuild.NewContext(), stmt.(*ast.CreateTableStmt)) require.NoError(t, err) return tblInfo } diff --git a/pkg/testkit/ddlhelper/BUILD.bazel b/pkg/testkit/ddlhelper/BUILD.bazel deleted file mode 100644 index deaaf3d50ba55..0000000000000 --- a/pkg/testkit/ddlhelper/BUILD.bazel +++ /dev/null @@ -1,13 +0,0 @@ -load("@io_bazel_rules_go//go:def.bzl", "go_library") - -go_library( - name = "ddlhelper", - srcs = ["helper.go"], - importpath = "github.com/pingcap/tidb/pkg/testkit/ddlhelper", - visibility = ["//visibility:public"], - deps = [ - "//pkg/ddl", - "//pkg/meta/model", - "//pkg/parser/ast", - ], -) diff --git a/pkg/testkit/ddlhelper/helper.go b/pkg/testkit/ddlhelper/helper.go deleted file mode 100644 index fcaf79b8faf91..0000000000000 --- a/pkg/testkit/ddlhelper/helper.go +++ /dev/null @@ -1,27 +0,0 @@ -// Copyright 2022 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 ddlhelper - -import ( - "github.com/pingcap/tidb/pkg/ddl" - "github.com/pingcap/tidb/pkg/meta/model" - "github.com/pingcap/tidb/pkg/parser/ast" -) - -// BuildTableInfoFromAST builds model.TableInfo from a SQL statement. -// Note: TableID and PartitionID are left as uninitialized value. -func BuildTableInfoFromAST(s *ast.CreateTableStmt) (*model.TableInfo, error) { - return ddl.BuildTableInfoFromAST(s) -} diff --git a/pkg/util/dbutil/dbutiltest/BUILD.bazel b/pkg/util/dbutil/dbutiltest/BUILD.bazel index 08609c1c0c282..67914e45b010b 100644 --- a/pkg/util/dbutil/dbutiltest/BUILD.bazel +++ b/pkg/util/dbutil/dbutiltest/BUILD.bazel @@ -7,13 +7,13 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/ddl", + "//pkg/meta/metabuild", "//pkg/meta/model", "//pkg/parser", "//pkg/parser/ast", "//pkg/parser/model", "//pkg/planner/core", "//pkg/types", - "//pkg/util/dbutil", "@com_github_pingcap_errors//:errors", ], ) diff --git a/pkg/util/dbutil/dbutiltest/utils.go b/pkg/util/dbutil/dbutiltest/utils.go index c6149398721b7..0cb5afadf230c 100644 --- a/pkg/util/dbutil/dbutiltest/utils.go +++ b/pkg/util/dbutil/dbutiltest/utils.go @@ -17,33 +17,17 @@ package dbutiltest import ( - "context" - "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/ddl" + "github.com/pingcap/tidb/pkg/meta/metabuild" "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" - "github.com/pingcap/tidb/pkg/util/dbutil" ) -// GetTableInfo returns table information. -func GetTableInfo(ctx context.Context, db dbutil.QueryExecutor, schemaName string, tableName string) (*model.TableInfo, error) { - createTableSQL, err := dbutil.GetCreateTableSQL(ctx, db, schemaName, tableName) - if err != nil { - return nil, errors.Trace(err) - } - - parser2, err := dbutil.GetParserForDB(ctx, db) - if err != nil { - return nil, errors.Trace(err) - } - return GetTableInfoBySQL(createTableSQL, parser2) -} - // GetTableInfoBySQL returns table information by given create table sql. func GetTableInfoBySQL(createTableSQL string, parser2 *parser.Parser) (table *model.TableInfo, err error) { stmt, err := parser2.ParseOneStmt(createTableSQL, "", "") @@ -53,7 +37,7 @@ func GetTableInfoBySQL(createTableSQL string, parser2 *parser.Parser) (table *mo s, ok := stmt.(*ast.CreateTableStmt) if ok { - table, err := ddl.BuildTableInfoFromAST(s) + table, err := ddl.BuildTableInfoFromAST(metabuild.NewContext(), s) if err != nil { return nil, errors.Trace(err) }