Skip to content

Commit

Permalink
*: remove mock.NewContext() usage when building table meta in produ…
Browse files Browse the repository at this point in the history
…ction code (#56348)

ref #53388
  • Loading branch information
lcwangchao authored Sep 29, 2024
1 parent 65d740f commit 8f0baf4
Show file tree
Hide file tree
Showing 18 changed files with 46 additions and 39 deletions.
2 changes: 1 addition & 1 deletion cmd/importer/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ go_library(
visibility = ["//visibility:private"],
deps = [
"//pkg/ddl",
"//pkg/meta/metabuild",
"//pkg/meta/model",
"//pkg/parser",
"//pkg/parser/ast",
Expand All @@ -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",
Expand Down
5 changes: 3 additions & 2 deletions cmd/importer/parser.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
)

Expand Down Expand Up @@ -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)
}
Expand Down
3 changes: 2 additions & 1 deletion lightning/pkg/importer/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand All @@ -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",
Expand Down Expand Up @@ -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",
Expand Down
3 changes: 2 additions & 1 deletion lightning/pkg/importer/chunk_process_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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

Expand Down
6 changes: 3 additions & 3 deletions lightning/pkg/importer/get_pre_info.go
Original file line number Diff line number Diff line change
Expand Up @@ -43,14 +43,14 @@ 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"
_ "github.com/pingcap/tidb/pkg/planner/core" // to setup expression.EvalAstExpr. Otherwise we cannot parse the default value
"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"
)
Expand Down Expand Up @@ -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
}
Expand Down
5 changes: 2 additions & 3 deletions pkg/ddl/create_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
)
Expand Down Expand Up @@ -392,8 +391,8 @@ func findTableIDFromStore(t *meta.Mutator, schemaID int64, tableName string) (in

// 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.
Expand Down
1 change: 1 addition & 0 deletions pkg/infoschema/perfschema/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@ go_library(
"//pkg/infoschema",
"//pkg/kv",
"//pkg/meta/autoid",
"//pkg/meta/metabuild",
"//pkg/meta/model",
"//pkg/parser",
"//pkg/parser/ast",
Expand Down
5 changes: 4 additions & 1 deletion pkg/infoschema/perfschema/init.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand All @@ -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)
}
Expand All @@ -62,6 +64,7 @@ func Init() {
c.ID = int64(i) + 1
}
meta.DBID = dbID
meta.State = model.StatePublic
}
dbInfo := &model.DBInfo{
ID: dbID,
Expand Down
11 changes: 11 additions & 0 deletions pkg/meta/metabuild/context.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
4 changes: 2 additions & 2 deletions pkg/planner/core/partition_pruning_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 := ddlhelper.BuildTableInfoFromASTForTest(stmt.(*ast.CreateTableStmt))
if err != nil {
return nil
}
Expand Down Expand Up @@ -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 := ddlhelper.BuildTableInfoFromASTForTest(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)
Expand Down
1 change: 1 addition & 0 deletions pkg/session/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,7 @@ go_library(
"//pkg/infoschema/context",
"//pkg/kv",
"//pkg/meta",
"//pkg/meta/metabuild",
"//pkg/meta/model",
"//pkg/metrics",
"//pkg/owner",
Expand Down
5 changes: 3 additions & 2 deletions pkg/session/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -3253,7 +3254,7 @@ func createAndSplitTables(store kv.Storage, t *meta.Mutator, dbID int64, tables
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)
}
Expand Down Expand Up @@ -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)
}
Expand Down
1 change: 1 addition & 0 deletions pkg/table/tables/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand Down
3 changes: 2 additions & 1 deletion pkg/table/tables/index_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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
}
Expand Down
1 change: 1 addition & 0 deletions pkg/testkit/ddlhelper/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -7,6 +7,7 @@ go_library(
visibility = ["//visibility:public"],
deps = [
"//pkg/ddl",
"//pkg/meta/metabuild",
"//pkg/meta/model",
"//pkg/parser/ast",
],
Expand Down
7 changes: 4 additions & 3 deletions pkg/testkit/ddlhelper/helper.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,12 +16,13 @@ package ddlhelper

import (
"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/ast"
)

// BuildTableInfoFromAST builds model.TableInfo from a SQL statement.
// BuildTableInfoFromASTForTest 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)
func BuildTableInfoFromASTForTest(s *ast.CreateTableStmt) (*model.TableInfo, error) {
return ddl.BuildTableInfoFromAST(metabuild.NewContext(), s)
}
2 changes: 1 addition & 1 deletion pkg/util/dbutil/dbutiltest/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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",
],
)
20 changes: 2 additions & 18 deletions pkg/util/dbutil/dbutiltest/utils.go
Original file line number Diff line number Diff line change
Expand Up @@ -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, "", "")
Expand All @@ -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)
}
Expand Down

0 comments on commit 8f0baf4

Please sign in to comment.