From 6ee8e0b73df5db8f3b8305a40a2c9b4ce757a05b Mon Sep 17 00:00:00 2001 From: pengdaqian Date: Mon, 16 Nov 2020 00:44:28 +0800 Subject: [PATCH 01/11] fix select cast not check --- executor/adapter.go | 2 +- executor/compiler.go | 2 +- executor/executor_test.go | 24 ++++++++++- executor/metrics_reader_test.go | 2 +- executor/prepared.go | 2 +- expression/integration_test.go | 2 +- expression/typeinfer_test.go | 2 +- planner/core/common_plans.go | 2 +- planner/core/indexmerge_test.go | 2 +- planner/core/logical_plan_test.go | 28 ++++++------- planner/core/preprocess.go | 67 +++++++++++++++++++++++-------- server/conn.go | 2 +- statistics/selectivity_test.go | 6 +-- util/ranger/ranger_test.go | 10 ++--- 14 files changed, 104 insertions(+), 49 deletions(-) diff --git a/executor/adapter.go b/executor/adapter.go index 4cf27e849d4d6..74085510e16ef 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -256,7 +256,7 @@ func (a *ExecStmt) IsReadOnly(vars *variable.SessionVars) bool { func (a *ExecStmt) RebuildPlan(ctx context.Context) (int64, error) { is := infoschema.GetInfoSchema(a.Ctx) a.InfoSchema = is - if err := plannercore.Preprocess(a.Ctx, a.StmtNode, is, plannercore.InTxnRetry); err != nil { + if err := plannercore.Preprocess(a.Ctx, a.StmtNode, a.StmtNode.Text(), is, plannercore.InTxnRetry); err != nil { return 0, err } p, names, err := planner.Optimize(ctx, a.Ctx, a.StmtNode, is) diff --git a/executor/compiler.go b/executor/compiler.go index 49ae9e3e844da..8e8b3bbc5ff4e 100644 --- a/executor/compiler.go +++ b/executor/compiler.go @@ -54,7 +54,7 @@ func (c *Compiler) Compile(ctx context.Context, stmtNode ast.StmtNode) (*ExecStm } infoSchema := infoschema.GetInfoSchema(c.Ctx) - if err := plannercore.Preprocess(c.Ctx, stmtNode, infoSchema); err != nil { + if err := plannercore.Preprocess(c.Ctx, stmtNode, stmtNode.Text(), infoSchema); err != nil { return nil, err } stmtNode = plannercore.TryAddExtraLimit(c.Ctx, stmtNode) diff --git a/executor/executor_test.go b/executor/executor_test.go index 68c197e5b07c6..527cbb03f34f0 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -17,6 +17,7 @@ import ( "context" "flag" "fmt" + "github.com/pingcap/tidb/errno" "io/ioutil" "math" "net" @@ -2259,7 +2260,7 @@ func (s *testSuiteP2) TestIsPointGet(c *C) { for sqlStr, result := range tests { stmtNode, err := s.ParseOneStmt(sqlStr, "", "") c.Check(err, IsNil) - err = plannercore.Preprocess(ctx, stmtNode, infoSchema) + err = plannercore.Preprocess(ctx, stmtNode, stmtNode.Text(), infoSchema) c.Check(err, IsNil) p, _, err := planner.Optimize(context.TODO(), ctx, stmtNode, infoSchema) c.Check(err, IsNil) @@ -2290,7 +2291,7 @@ func (s *testSuiteP2) TestClusteredIndexIsPointGet(c *C) { for sqlStr, result := range tests { stmtNode, err := s.ParseOneStmt(sqlStr, "", "") c.Check(err, IsNil) - err = plannercore.Preprocess(ctx, stmtNode, infoSchema) + err = plannercore.Preprocess(ctx, stmtNode, stmtNode.Text(), infoSchema) c.Check(err, IsNil) p, _, err := planner.Optimize(context.TODO(), ctx, stmtNode, infoSchema) c.Check(err, IsNil) @@ -6849,3 +6850,22 @@ func (s *testSuite) TestIssue19667(c *C) { tk.MustExec("INSERT INTO t VALUES('1988-04-17 01:59:59')") tk.MustQuery(`SELECT DATE_ADD(a, INTERVAL 1 SECOND) FROM t`).Check(testkit.Rows("1988-04-17 02:00:00")) } + +// TestSelectWithCast for issue #21063 +func (s *testSuite) TestSelectWithCast(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec(`use test`) + tk.MustExec(`drop table if exists t`) + defer tk.MustExec(`drop table if exists t`) + tk.MustExec(`create table t(d decimal(10, 5))`) + tk.MustGetErrCode(`select * from t where d = cast(d as decimal(10,20))`, errno.ErrMBiggerThanD) + tk.MustGetErrCode(`select cast(111 as decimal(1,20)) from t`, errno.ErrMBiggerThanD) + tk.MustGetErrCode(`select * from t where d = cast(111 as decimal(1000,20))`, errno.ErrTooBigPrecision) + tk.MustGetErrCode(`select cast(111 as decimal(1000,20)) from t`, errno.ErrTooBigPrecision) + tk.MustGetErrMsg(`select * from t where d = cast(d as decimal(10,20))`, `[types:1427]For float(M,D), double(M,D) or decimal(M,D), M must be >= D (column '').`) + tk.MustGetErrMsg(`select * from t where d = cast("d" as decimal(10,20))`, `[types:1427]For float(M,D), double(M,D) or decimal(M,D), M must be >= D (column '').`) + tk.MustGetErrMsg(`select * from t where d = cast(111 as decimal(1000,20))`, `[types:1426]Too big precision 1000 specified for column '111'. Maximum is 65.`) + tk.MustGetErrMsg(`select * from t where d = cast("abc" as decimal(1000,20))`, `[types:1426]Too big precision 1000 specified for column '"abc"'. Maximum is 65.`) + tk.MustGetErrMsg(`select * from t where d = cast(d as decimal(1000,20))`, `[types:1426]Too big precision 1000 specified for column 'd'. Maximum is 65.`) + tk.MustGetErrMsg(`select * from t where d = cast('d' as decimal(1000,20))`, `[types:1426]Too big precision 1000 specified for column ''d''. Maximum is 65.`) +} diff --git a/executor/metrics_reader_test.go b/executor/metrics_reader_test.go index 19000b4faee5d..057f8b05e79ca 100644 --- a/executor/metrics_reader_test.go +++ b/executor/metrics_reader_test.go @@ -63,7 +63,7 @@ func (s *testSuite7) TestStmtLabel(c *C) { stmtNode, err := parser.New().ParseOneStmt(tt.sql, "", "") c.Check(err, IsNil) is := infoschema.GetInfoSchema(tk.Se) - err = plannercore.Preprocess(tk.Se.(sessionctx.Context), stmtNode, is) + err = plannercore.Preprocess(tk.Se.(sessionctx.Context), stmtNode, stmtNode.Text(), is) c.Assert(err, IsNil) _, _, err = planner.Optimize(context.TODO(), tk.Se, stmtNode, is) c.Assert(err, IsNil) diff --git a/executor/prepared.go b/executor/prepared.go index 278fd84d4dad0..9a2640e8dcc99 100644 --- a/executor/prepared.go +++ b/executor/prepared.go @@ -153,7 +153,7 @@ func (e *PrepareExec) Next(ctx context.Context, req *chunk.Chunk) error { return ErrPsManyParam } - err = plannercore.Preprocess(e.ctx, stmt, e.is, plannercore.InPrepare) + err = plannercore.Preprocess(e.ctx, stmt, stmt.Text(), e.is, plannercore.InPrepare) if err != nil { return err } diff --git a/expression/integration_test.go b/expression/integration_test.go index a5a554cd1b29e..f03f42bea53a7 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -4527,7 +4527,7 @@ func (s *testIntegrationSuite) TestFilterExtractFromDNF(c *C) { c.Assert(err, IsNil, Commentf("error %v, for expr %s", err, tt.exprStr)) c.Assert(stmts, HasLen, 1) is := domain.GetDomain(sctx).InfoSchema() - err = plannercore.Preprocess(sctx, stmts[0], is) + err = plannercore.Preprocess(sctx, stmts[0], stmts[0].Text(), is) c.Assert(err, IsNil, Commentf("error %v, for resolve name, expr %s", err, tt.exprStr)) p, _, err := plannercore.BuildLogicalPlan(ctx, sctx, stmts[0], is) c.Assert(err, IsNil, Commentf("error %v, for build plan, expr %s", err, tt.exprStr)) diff --git a/expression/typeinfer_test.go b/expression/typeinfer_test.go index ea82d891d07b7..04ba05eeef926 100644 --- a/expression/typeinfer_test.go +++ b/expression/typeinfer_test.go @@ -140,7 +140,7 @@ func (s *testInferTypeSuite) TestInferType(c *C) { c.Assert(err, IsNil) is := domain.GetDomain(sctx).InfoSchema() - err = plannercore.Preprocess(sctx, stmt, is) + err = plannercore.Preprocess(sctx, stmt, stmt.Text(), is) c.Assert(err, IsNil, comment) p, _, err := plannercore.BuildLogicalPlan(ctx, sctx, stmt, is) c.Assert(err, IsNil, comment) diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index 107eea6352269..bb219b160e5b6 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -243,7 +243,7 @@ func (e *Execute) OptimizePreparedPlan(ctx context.Context, sctx sessionctx.Cont preparedObj.Executor = nil // If the schema version has changed we need to preprocess it again, // if this time it failed, the real reason for the error is schema changed. - err := Preprocess(sctx, prepared.Stmt, is, InPrepare) + err := Preprocess(sctx, prepared.Stmt, prepared.Stmt.Text(), is, InPrepare) if err != nil { return ErrSchemaChanged.GenWithStack("Schema change caused error: %s", err.Error()) } diff --git a/planner/core/indexmerge_test.go b/planner/core/indexmerge_test.go index 8df97cb27ac28..f866d6301e8eb 100644 --- a/planner/core/indexmerge_test.go +++ b/planner/core/indexmerge_test.go @@ -90,7 +90,7 @@ func (s *testIndexMergeSuite) TestIndexMergePathGeneration(c *C) { comment := Commentf("case:%v sql:%s", i, tc) stmt, err := s.ParseOneStmt(tc, "", "") c.Assert(err, IsNil, comment) - Preprocess(s.ctx, stmt, s.is) + Preprocess(s.ctx, stmt, stmt.Text(), s.is) builder := NewPlanBuilder(MockContext(), s.is, &hint.BlockHintProcessor{}) p, err := builder.Build(ctx, stmt) if err != nil { diff --git a/planner/core/logical_plan_test.go b/planner/core/logical_plan_test.go index 6e8f6e0ef033f..c72fca12c800f 100644 --- a/planner/core/logical_plan_test.go +++ b/planner/core/logical_plan_test.go @@ -442,7 +442,7 @@ func (s *testPlanSuite) TestSubquery(c *C) { stmt, err := s.ParseOneStmt(ca, "", "") c.Assert(err, IsNil, comment) - Preprocess(s.ctx, stmt, s.is) + Preprocess(s.ctx, stmt, stmt.Text(), s.is) p, _, err := BuildLogicalPlan(ctx, s.ctx, stmt, s.is) c.Assert(err, IsNil) if lp, ok := p.(LogicalPlan); ok { @@ -467,7 +467,7 @@ func (s *testPlanSuite) TestPlanBuilder(c *C) { c.Assert(err, IsNil, comment) s.ctx.GetSessionVars().SetHashJoinConcurrency(1) - Preprocess(s.ctx, stmt, s.is) + Preprocess(s.ctx, stmt, stmt.Text(), s.is) p, _, err := BuildLogicalPlan(ctx, s.ctx, stmt, s.is) c.Assert(err, IsNil) if lp, ok := p.(LogicalPlan); ok { @@ -827,7 +827,7 @@ func (s *testPlanSuite) TestValidate(c *C) { comment := Commentf("for %s", sql) stmt, err := s.ParseOneStmt(sql, "", "") c.Assert(err, IsNil, comment) - Preprocess(s.ctx, stmt, s.is) + Preprocess(s.ctx, stmt, stmt.Text(), s.is) _, _, err = BuildLogicalPlan(ctx, s.ctx, stmt, s.is) if tt.err == nil { c.Assert(err, IsNil, comment) @@ -1117,7 +1117,7 @@ func (s *testPlanSuite) TestVisitInfo(c *C) { comment := Commentf("for %s", tt.sql) stmt, err := s.ParseOneStmt(tt.sql, "", "") c.Assert(err, IsNil, comment) - Preprocess(s.ctx, stmt, s.is) + Preprocess(s.ctx, stmt, stmt.Text(), s.is) builder := NewPlanBuilder(MockContext(), s.is, &hint.BlockHintProcessor{}) builder.ctx.GetSessionVars().SetHashJoinConcurrency(1) _, err = builder.Build(context.TODO(), stmt) @@ -1197,7 +1197,7 @@ func (s *testPlanSuite) TestUnion(c *C) { comment := Commentf("case:%v sql:%s", i, tt) stmt, err := s.ParseOneStmt(tt, "", "") c.Assert(err, IsNil, comment) - Preprocess(s.ctx, stmt, s.is) + Preprocess(s.ctx, stmt, stmt.Text(), s.is) builder := NewPlanBuilder(MockContext(), s.is, &hint.BlockHintProcessor{}) plan, err := builder.Build(ctx, stmt) s.testData.OnRecord(func() { @@ -1229,7 +1229,7 @@ func (s *testPlanSuite) TestTopNPushDown(c *C) { comment := Commentf("case:%v sql:%s", i, tt) stmt, err := s.ParseOneStmt(tt, "", "") c.Assert(err, IsNil, comment) - Preprocess(s.ctx, stmt, s.is) + Preprocess(s.ctx, stmt, stmt.Text(), s.is) builder := NewPlanBuilder(MockContext(), s.is, &hint.BlockHintProcessor{}) p, err := builder.Build(ctx, stmt) c.Assert(err, IsNil) @@ -1303,7 +1303,7 @@ func (s *testPlanSuite) TestOuterJoinEliminator(c *C) { comment := Commentf("case:%v sql:%s", i, tt) stmt, err := s.ParseOneStmt(tt, "", "") c.Assert(err, IsNil, comment) - Preprocess(s.ctx, stmt, s.is) + Preprocess(s.ctx, stmt, stmt.Text(), s.is) builder := NewPlanBuilder(MockContext(), s.is, &hint.BlockHintProcessor{}) p, err := builder.Build(ctx, stmt) c.Assert(err, IsNil) @@ -1339,7 +1339,7 @@ func (s *testPlanSuite) TestSelectView(c *C) { comment := Commentf("case:%v sql:%s", i, tt.sql) stmt, err := s.ParseOneStmt(tt.sql, "", "") c.Assert(err, IsNil, comment) - Preprocess(s.ctx, stmt, s.is) + Preprocess(s.ctx, stmt, stmt.Text(), s.is) builder := NewPlanBuilder(MockContext(), s.is, &hint.BlockHintProcessor{}) p, err := builder.Build(ctx, stmt) c.Assert(err, IsNil) @@ -1410,7 +1410,7 @@ func (s *testPlanSuite) optimize(ctx context.Context, sql string) (PhysicalPlan, if err != nil { return nil, nil, err } - err = Preprocess(s.ctx, stmt, s.is) + err = Preprocess(s.ctx, stmt, stmt.Text(), s.is) if err != nil { return nil, nil, err } @@ -1502,7 +1502,7 @@ func (s *testPlanSuite) TestSkylinePruning(c *C) { comment := Commentf("case:%v sql:%s", i, tt.sql) stmt, err := s.ParseOneStmt(tt.sql, "", "") c.Assert(err, IsNil, comment) - Preprocess(s.ctx, stmt, s.is) + Preprocess(s.ctx, stmt, stmt.Text(), s.is) builder := NewPlanBuilder(MockContext(), s.is, &hint.BlockHintProcessor{}) p, err := builder.Build(ctx, stmt) if err != nil { @@ -1572,7 +1572,7 @@ func (s *testPlanSuite) TestFastPlanContextTables(c *C) { for _, tt := range tests { stmt, err := s.ParseOneStmt(tt.sql, "", "") c.Assert(err, IsNil) - Preprocess(s.ctx, stmt, s.is) + Preprocess(s.ctx, stmt, stmt.Text(), s.is) s.ctx.GetSessionVars().StmtCtx.Tables = nil p := TryFastPlan(s.ctx, stmt) if tt.fastPlan { @@ -1603,7 +1603,7 @@ func (s *testPlanSuite) TestUpdateEQCond(c *C) { comment := Commentf("case:%v sql:%s", i, tt.sql) stmt, err := s.ParseOneStmt(tt.sql, "", "") c.Assert(err, IsNil, comment) - Preprocess(s.ctx, stmt, s.is) + Preprocess(s.ctx, stmt, stmt.Text(), s.is) builder := NewPlanBuilder(MockContext(), s.is, &hint.BlockHintProcessor{}) p, err := builder.Build(ctx, stmt) c.Assert(err, IsNil) @@ -1619,7 +1619,7 @@ func (s *testPlanSuite) TestConflictedJoinTypeHints(c *C) { ctx := context.TODO() stmt, err := s.ParseOneStmt(sql, "", "") c.Assert(err, IsNil) - Preprocess(s.ctx, stmt, s.is) + Preprocess(s.ctx, stmt, stmt.Text(), s.is) builder := NewPlanBuilder(MockContext(), s.is, &hint.BlockHintProcessor{}) p, err := builder.Build(ctx, stmt) c.Assert(err, IsNil) @@ -1639,7 +1639,7 @@ func (s *testPlanSuite) TestSimplyOuterJoinWithOnlyOuterExpr(c *C) { ctx := context.TODO() stmt, err := s.ParseOneStmt(sql, "", "") c.Assert(err, IsNil) - Preprocess(s.ctx, stmt, s.is) + Preprocess(s.ctx, stmt, stmt.Text(), s.is) builder := NewPlanBuilder(MockContext(), s.is, &hint.BlockHintProcessor{}) p, err := builder.Build(ctx, stmt) c.Assert(err, IsNil) diff --git a/planner/core/preprocess.go b/planner/core/preprocess.go index 97b436435cfbb..205c7e95b750c 100644 --- a/planner/core/preprocess.go +++ b/planner/core/preprocess.go @@ -78,8 +78,8 @@ func TryAddExtraLimit(ctx sessionctx.Context, node ast.StmtNode) ast.StmtNode { } // Preprocess resolves table names of the node, and checks some statements validation. -func Preprocess(ctx sessionctx.Context, node ast.Node, is infoschema.InfoSchema, preprocessOpt ...PreprocessOpt) error { - v := preprocessor{is: is, ctx: ctx, tableAliasInJoin: make([]map[string]interface{}, 0)} +func Preprocess(ctx sessionctx.Context, node ast.Node, sql string, is infoschema.InfoSchema, preprocessOpt ...PreprocessOpt) error { + v := preprocessor{is: is, ctx: ctx, sql: sql, tableAliasInJoin: make([]map[string]interface{}, 0)} for _, optFn := range preprocessOpt { optFn(&v) } @@ -108,6 +108,7 @@ const ( // preprocessor is an ast.Visitor that preprocess // ast Nodes parsed from parser. type preprocessor struct { + sql string is infoschema.InfoSchema ctx sessionctx.Context err error @@ -195,6 +196,8 @@ func (p *preprocessor) Enter(in ast.Node) (out ast.Node, skipChildren bool) { } case *ast.CreateStatisticsStmt, *ast.DropStatisticsStmt: p.checkStatisticsOpGrammar(in) + case *ast.FuncCastExpr: + p.checkCastGrammar(node) default: p.flag &= ^parentIsJoin } @@ -802,18 +805,21 @@ func checkColumn(colDef *ast.ColumnDef) error { } // Check column type. - tp := colDef.Tp + return checkTp(colDef.Tp, colDef.Name.Name.O, "") +} + +func checkTp(tp *types.FieldType, colName, val string) error { if tp == nil { return nil } if tp.Flen > math.MaxUint32 { - return types.ErrTooBigDisplayWidth.GenWithStack("Display width out of range for column '%s' (max = %d)", colDef.Name.Name.O, math.MaxUint32) + return types.ErrTooBigDisplayWidth.GenWithStack("Display width out of range for column '%s' (max = %d)", colName, math.MaxUint32) } switch tp.Tp { case mysql.TypeString: if tp.Flen != types.UnspecifiedLength && tp.Flen > mysql.MaxFieldCharLength { - return types.ErrTooBigFieldLength.GenWithStack("Column length too big for column '%s' (max = %d); use BLOB or TEXT instead", colDef.Name.Name.O, mysql.MaxFieldCharLength) + return types.ErrTooBigFieldLength.GenWithStack("Column length too big for column '%s' (max = %d); use BLOB or TEXT instead", colName, mysql.MaxFieldCharLength) } case mysql.TypeVarchar: if len(tp.Charset) == 0 { @@ -822,7 +828,7 @@ func checkColumn(colDef *ast.ColumnDef) error { // return nil, to make the check in the ddl.CreateTable. return nil } - err := ddl.IsTooBigFieldLength(colDef.Tp.Flen, colDef.Name.Name.O, tp.Charset) + err := ddl.IsTooBigFieldLength(tp.Flen, colName, tp.Charset) if err != nil { return err } @@ -831,46 +837,54 @@ func checkColumn(colDef *ast.ColumnDef) error { // https://dev.mysql.com/doc/refman/8.0/en/floating-point-types.html if tp.Decimal == -1 { if tp.Tp == mysql.TypeDouble { - if tp.Flen != -1 { + if tp.Flen != -1 && colName != "" { return types.ErrSyntax.GenWithStackByArgs() } } else { if tp.Flen > mysql.MaxDoublePrecisionLength { - return types.ErrWrongFieldSpec.GenWithStackByArgs(colDef.Name.Name.O) + return types.ErrWrongFieldSpec.GenWithStackByArgs(colName) } } } else { if tp.Decimal > mysql.MaxFloatingTypeScale { - return types.ErrTooBigScale.GenWithStackByArgs(tp.Decimal, colDef.Name.Name.O, mysql.MaxFloatingTypeScale) + return types.ErrTooBigScale.GenWithStackByArgs(tp.Decimal, colName, mysql.MaxFloatingTypeScale) } if tp.Flen > mysql.MaxFloatingTypeWidth { - return types.ErrTooBigDisplayWidth.GenWithStackByArgs(colDef.Name.Name.O, mysql.MaxFloatingTypeWidth) + return types.ErrTooBigDisplayWidth.GenWithStackByArgs(colName, mysql.MaxFloatingTypeWidth) } } case mysql.TypeSet: if len(tp.Elems) > mysql.MaxTypeSetMembers { - return types.ErrTooBigSet.GenWithStack("Too many strings for column %s and SET", colDef.Name.Name.O) + return types.ErrTooBigSet.GenWithStack("Too many strings for column %s and SET", colName) } // Check set elements. See https://dev.mysql.com/doc/refman/5.7/en/set.html. - for _, str := range colDef.Tp.Elems { + for _, str := range tp.Elems { if strings.Contains(str, ",") { return types.ErrIllegalValueForType.GenWithStackByArgs(types.TypeStr(tp.Tp), str) } } case mysql.TypeNewDecimal: if tp.Decimal > mysql.MaxDecimalScale { - return types.ErrTooBigScale.GenWithStackByArgs(tp.Decimal, colDef.Name.Name.O, mysql.MaxDecimalScale) + return types.ErrTooBigScale.GenWithStackByArgs(tp.Decimal, colName, mysql.MaxDecimalScale) } if tp.Flen > mysql.MaxDecimalWidth { - return types.ErrTooBigPrecision.GenWithStackByArgs(tp.Flen, colDef.Name.Name.O, mysql.MaxDecimalWidth) + if colName == "" { + return types.ErrTooBigPrecision.GenWithStackByArgs(tp.Flen, val, mysql.MaxDecimalWidth) + } else { + return types.ErrTooBigPrecision.GenWithStackByArgs(tp.Flen, colName, mysql.MaxDecimalWidth) + } + } + + if tp.Flen < tp.Decimal { + return types.ErrMBiggerThanD.GenWithStackByArgs(colName) } case mysql.TypeBit: if tp.Flen <= 0 { - return types.ErrInvalidFieldSize.GenWithStackByArgs(colDef.Name.Name.O) + return types.ErrInvalidFieldSize.GenWithStackByArgs(colName) } if tp.Flen > mysql.MaxBitDisplayWidth { - return types.ErrTooBigDisplayWidth.GenWithStackByArgs(colDef.Name.Name.O, mysql.MaxBitDisplayWidth) + return types.ErrTooBigDisplayWidth.GenWithStackByArgs(colName, mysql.MaxBitDisplayWidth) } default: // TODO: Add more types. @@ -1064,3 +1078,24 @@ func (p *preprocessor) resolveCreateSequenceStmt(stmt *ast.CreateSequenceStmt) { return } } + +func (p *preprocessor) checkCastGrammar(node *ast.FuncCastExpr) { + var val string + switch x := node.Expr.(type) { + case ast.ValueExpr: + val = x.GetDatumString() + if val == "" { + val = fmt.Sprintf("%v", x.GetValue()) + } else { + wrapChar := p.sql[x.OriginTextPosition() : x.OriginTextPosition()+1] + val = wrapChar + val + wrapChar + } + case *ast.ColumnNameExpr: + val = x.Name.Name.O + default: + } + if err := checkTp(node.Tp, "", val); err != nil { + p.err = err + return + } +} diff --git a/server/conn.go b/server/conn.go index 7a63136e0ff3e..405f2988d907d 100644 --- a/server/conn.go +++ b/server/conn.go @@ -1468,7 +1468,7 @@ func (cc *clientConn) prefetchPointPlanKeys(ctx context.Context, stmts []ast.Stm sc := vars.StmtCtx for i, stmt := range stmts { // TODO: the preprocess is run twice, we should find some way to avoid do it again. - if err = plannercore.Preprocess(cc.ctx, stmt, is); err != nil { + if err = plannercore.Preprocess(cc.ctx, stmt, stmt.Text(), is); err != nil { return nil, err } p := plannercore.TryFastPlan(cc.ctx.Session, stmt) diff --git a/statistics/selectivity_test.go b/statistics/selectivity_test.go index 2b3fb3a3876c3..3bb549265d8cb 100644 --- a/statistics/selectivity_test.go +++ b/statistics/selectivity_test.go @@ -293,7 +293,7 @@ func (s *testStatsSuite) TestSelectivity(c *C) { c.Assert(err, IsNil, Commentf("error %v, for expr %s", err, tt.exprs)) c.Assert(stmts, HasLen, 1) - err = plannercore.Preprocess(sctx, stmts[0], is) + err = plannercore.Preprocess(sctx, stmts[0], stmts[0].Text(), is) c.Assert(err, IsNil, comment) p, _, err := plannercore.BuildLogicalPlan(ctx, sctx, stmts[0], is) c.Assert(err, IsNil, Commentf("error %v, for building plan, expr %s", err, tt.exprs)) @@ -549,7 +549,7 @@ func BenchmarkSelectivity(b *testing.B) { stmts, err := session.Parse(sctx, sql) c.Assert(err, IsNil, Commentf("error %v, for expr %s", err, exprs)) c.Assert(stmts, HasLen, 1) - err = plannercore.Preprocess(sctx, stmts[0], is) + err = plannercore.Preprocess(sctx, stmts[0], stmts[0].Text(), is) c.Assert(err, IsNil, comment) p, _, err := plannercore.BuildLogicalPlan(context.Background(), sctx, stmts[0], is) c.Assert(err, IsNil, Commentf("error %v, for building plan, expr %s", err, exprs)) @@ -705,7 +705,7 @@ func (s *testStatsSuite) TestDNFCondSelectivity(c *C) { c.Assert(err, IsNil, Commentf("error %v, for sql %s", err, tt)) c.Assert(stmts, HasLen, 1) - err = plannercore.Preprocess(sctx, stmts[0], is) + err = plannercore.Preprocess(sctx, stmts[0], stmts[0].Text(), is) c.Assert(err, IsNil, Commentf("error %v, for sql %s", err, tt)) p, _, err := plannercore.BuildLogicalPlan(ctx, sctx, stmts[0], is) c.Assert(err, IsNil, Commentf("error %v, for building plan, sql %s", err, tt)) diff --git a/util/ranger/ranger_test.go b/util/ranger/ranger_test.go index 3e9d7b8542210..ce7999e1bde9e 100644 --- a/util/ranger/ranger_test.go +++ b/util/ranger/ranger_test.go @@ -304,7 +304,7 @@ func (s *testRangerSuite) TestTableRange(c *C) { c.Assert(err, IsNil, Commentf("error %v, for expr %s", err, tt.exprStr)) c.Assert(stmts, HasLen, 1) is := domain.GetDomain(sctx).InfoSchema() - err = plannercore.Preprocess(sctx, stmts[0], is) + err = plannercore.Preprocess(sctx, stmts[0], stmts[0].Text(), is) c.Assert(err, IsNil, Commentf("error %v, for resolve name, expr %s", err, tt.exprStr)) p, _, err := plannercore.BuildLogicalPlan(ctx, sctx, stmts[0], is) c.Assert(err, IsNil, Commentf("error %v, for build plan, expr %s", err, tt.exprStr)) @@ -631,7 +631,7 @@ create table t( c.Assert(err, IsNil, Commentf("error %v, for expr %s", err, tt.exprStr)) c.Assert(stmts, HasLen, 1) is := domain.GetDomain(sctx).InfoSchema() - err = plannercore.Preprocess(sctx, stmts[0], is) + err = plannercore.Preprocess(sctx, stmts[0], stmts[0].Text(), is) c.Assert(err, IsNil, Commentf("error %v, for resolve name, expr %s", err, tt.exprStr)) p, _, err := plannercore.BuildLogicalPlan(ctx, sctx, stmts[0], is) c.Assert(err, IsNil, Commentf("error %v, for build plan, expr %s", err, tt.exprStr)) @@ -752,7 +752,7 @@ func (s *testRangerSuite) TestIndexRangeForUnsignedInt(c *C) { c.Assert(err, IsNil, Commentf("error %v, for expr %s", err, tt.exprStr)) c.Assert(stmts, HasLen, 1) is := domain.GetDomain(sctx).InfoSchema() - err = plannercore.Preprocess(sctx, stmts[0], is) + err = plannercore.Preprocess(sctx, stmts[0], stmts[0].Text(), is) c.Assert(err, IsNil, Commentf("error %v, for resolve name, expr %s", err, tt.exprStr)) p, _, err := plannercore.BuildLogicalPlan(ctx, sctx, stmts[0], is) c.Assert(err, IsNil, Commentf("error %v, for build plan, expr %s", err, tt.exprStr)) @@ -1100,7 +1100,7 @@ func (s *testRangerSuite) TestColumnRange(c *C) { c.Assert(err, IsNil, Commentf("error %v, for expr %s", err, tt.exprStr)) c.Assert(stmts, HasLen, 1) is := domain.GetDomain(sctx).InfoSchema() - err = plannercore.Preprocess(sctx, stmts[0], is) + err = plannercore.Preprocess(sctx, stmts[0], stmts[0].Text(), is) c.Assert(err, IsNil, Commentf("error %v, for resolve name, expr %s", err, tt.exprStr)) p, _, err := plannercore.BuildLogicalPlan(ctx, sctx, stmts[0], is) c.Assert(err, IsNil, Commentf("error %v, for build plan, expr %s", err, tt.exprStr)) @@ -1438,7 +1438,7 @@ func (s *testRangerSuite) TestIndexRangeForYear(c *C) { c.Assert(err, IsNil, Commentf("error %v, for expr %s", err, tt.exprStr)) c.Assert(stmts, HasLen, 1) is := domain.GetDomain(sctx).InfoSchema() - err = plannercore.Preprocess(sctx, stmts[0], is) + err = plannercore.Preprocess(sctx, stmts[0], stmts[0].Text(), is) c.Assert(err, IsNil, Commentf("error %v, for resolve name, expr %s", err, tt.exprStr)) p, _, err := plannercore.BuildLogicalPlan(ctx, sctx, stmts[0], is) c.Assert(err, IsNil, Commentf("error %v, for build plan, expr %s", err, tt.exprStr)) From b65013fe78b35d34d3fa7c9800f4aba5f6f8e2f7 Mon Sep 17 00:00:00 2001 From: pengdaqian Date: Mon, 16 Nov 2020 00:58:24 +0800 Subject: [PATCH 02/11] fix test error --- planner/core/cbo_test.go | 10 +++++----- planner/core/physical_plan_test.go | 4 ++-- planner/core/point_get_plan_test.go | 2 +- planner/core/preprocess_test.go | 2 +- planner/core/stats_test.go | 2 +- 5 files changed, 10 insertions(+), 10 deletions(-) diff --git a/planner/core/cbo_test.go b/planner/core/cbo_test.go index 4394a6e006a38..40ab9ee7edc55 100644 --- a/planner/core/cbo_test.go +++ b/planner/core/cbo_test.go @@ -277,7 +277,7 @@ func (s *testAnalyzeSuite) TestIndexRead(c *C) { c.Assert(stmts, HasLen, 1) stmt := stmts[0] is := domain.GetDomain(ctx).InfoSchema() - err = core.Preprocess(ctx, stmt, is) + err = core.Preprocess(ctx, stmt, stmt.Text(), is) c.Assert(err, IsNil) p, _, err := planner.Optimize(context.TODO(), ctx, stmt, is) c.Assert(err, IsNil) @@ -312,7 +312,7 @@ func (s *testAnalyzeSuite) TestEmptyTable(c *C) { c.Assert(stmts, HasLen, 1) stmt := stmts[0] is := domain.GetDomain(ctx).InfoSchema() - err = core.Preprocess(ctx, stmt, is) + err = core.Preprocess(ctx, stmt, stmt.Text(), is) c.Assert(err, IsNil) p, _, err := planner.Optimize(context.TODO(), ctx, stmt, is) c.Assert(err, IsNil) @@ -383,7 +383,7 @@ func (s *testAnalyzeSuite) TestAnalyze(c *C) { err = executor.ResetContextOfStmt(ctx, stmt) c.Assert(err, IsNil) is := domain.GetDomain(ctx).InfoSchema() - err = core.Preprocess(ctx, stmt, is) + err = core.Preprocess(ctx, stmt, stmt.Text(), is) c.Assert(err, IsNil) p, _, err := planner.Optimize(context.TODO(), ctx, stmt, is) c.Assert(err, IsNil) @@ -463,7 +463,7 @@ func (s *testAnalyzeSuite) TestPreparedNullParam(c *C) { stmt := stmts[0] is := domain.GetDomain(ctx).InfoSchema() - err = core.Preprocess(ctx, stmt, is, core.InPrepare) + err = core.Preprocess(ctx, stmt, stmt.Text(), is, core.InPrepare) c.Assert(err, IsNil) p, _, err := planner.Optimize(context.TODO(), ctx, stmt, is) c.Assert(err, IsNil) @@ -692,7 +692,7 @@ func BenchmarkOptimize(b *testing.B) { c.Assert(stmts, HasLen, 1) stmt := stmts[0] is := domain.GetDomain(ctx).InfoSchema() - err = core.Preprocess(ctx, stmt, is) + err = core.Preprocess(ctx, stmt, stmt.Text(), is) c.Assert(err, IsNil) b.Run(tt.sql, func(b *testing.B) { diff --git a/planner/core/physical_plan_test.go b/planner/core/physical_plan_test.go index 62519e84ba530..80d65f06bfbe3 100644 --- a/planner/core/physical_plan_test.go +++ b/planner/core/physical_plan_test.go @@ -248,7 +248,7 @@ func (s *testPlanSuite) TestDAGPlanBuilderBasePhysicalPlan(c *C) { stmt, err := s.ParseOneStmt(tt, "", "") c.Assert(err, IsNil, comment) - core.Preprocess(se, stmt, s.is) + core.Preprocess(se, stmt, stmt.Text(), s.is) p, _, err := planner.Optimize(context.TODO(), se, stmt, s.is) c.Assert(err, IsNil) s.testData.OnRecord(func() { @@ -1371,7 +1371,7 @@ func (s *testPlanSuite) TestDAGPlanBuilderSplitAvg(c *C) { stmt, err := s.ParseOneStmt(tt.sql, "", "") c.Assert(err, IsNil, comment) - core.Preprocess(se, stmt, s.is) + core.Preprocess(se, stmt, stmt.Text(), s.is) p, _, err := planner.Optimize(context.TODO(), se, stmt, s.is) c.Assert(err, IsNil, comment) diff --git a/planner/core/point_get_plan_test.go b/planner/core/point_get_plan_test.go index c74a1140fc4a2..d257a3c214c85 100644 --- a/planner/core/point_get_plan_test.go +++ b/planner/core/point_get_plan_test.go @@ -307,7 +307,7 @@ func (s *testPointGetSuite) TestPointGetId(c *C) { c.Assert(stmts, HasLen, 1) stmt := stmts[0] is := domain.GetDomain(ctx).InfoSchema() - err = core.Preprocess(ctx, stmt, is) + err = core.Preprocess(ctx, stmt, stmt.Text(), is) c.Assert(err, IsNil) p, _, err := planner.Optimize(context.TODO(), ctx, stmt, is) c.Assert(err, IsNil) diff --git a/planner/core/preprocess_test.go b/planner/core/preprocess_test.go index 442dcee025b30..c7c931fa9fcb0 100644 --- a/planner/core/preprocess_test.go +++ b/planner/core/preprocess_test.go @@ -65,7 +65,7 @@ func (s *testValidatorSuite) runSQL(c *C, sql string, inPrepare bool, terr error if inPrepare { opts = append(opts, core.InPrepare) } - err := core.Preprocess(s.ctx, stmt, s.is, opts...) + err := core.Preprocess(s.ctx, stmt, stmt.Text(), s.is, opts...) c.Assert(terror.ErrorEqual(err, terr), IsTrue, Commentf("sql: %s, err:%v", sql, err)) } diff --git a/planner/core/stats_test.go b/planner/core/stats_test.go index 0b3176eb87b60..9eebf2a010b88 100644 --- a/planner/core/stats_test.go +++ b/planner/core/stats_test.go @@ -75,7 +75,7 @@ func (s *testStatsSuite) TestGroupNDVs(c *C) { comment := Commentf("case:%v sql: %s", i, tt) stmt, err := s.ParseOneStmt(tt, "", "") c.Assert(err, IsNil, comment) - core.Preprocess(tk.Se, stmt, is) + core.Preprocess(tk.Se, stmt, stmt.Text(), is) builder := core.NewPlanBuilder(tk.Se, is, &hint.BlockHintProcessor{}) p, err := builder.Build(ctx, stmt) c.Assert(err, IsNil, comment) From 89260ac793e4d1effce79374f81cebbce7e1b9f3 Mon Sep 17 00:00:00 2001 From: pengdaqian Date: Mon, 16 Nov 2020 01:23:26 +0800 Subject: [PATCH 03/11] fix test error --- expression/integration_test.go | 9 +-------- 1 file changed, 1 insertion(+), 8 deletions(-) diff --git a/expression/integration_test.go b/expression/integration_test.go index f03f42bea53a7..5db37a77b0491 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -2315,7 +2315,6 @@ func (s *testIntegrationSuite2) TestBuiltin(c *C) { defer s.cleanEnv(c) tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") - ctx := context.Background() // for is true && is false tk.MustExec("drop table if exists t") @@ -2754,13 +2753,7 @@ func (s *testIntegrationSuite2) TestBuiltin(c *C) { c.Assert(err, NotNil) // test case decimal precision less than the scale. - rs, err := tk.Exec("select cast(12.1 as decimal(3, 4));") - c.Assert(err, IsNil) - _, err = session.GetRows4Test(ctx, tk.Se, rs) - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[types:1427]For float(M,D), double(M,D) or decimal(M,D), M must be >= D (column '').") - c.Assert(rs.Close(), IsNil) - + tk.MustGetErrMsg("select cast(12.1 as decimal(3, 4));", "[types:1427]For float(M,D), double(M,D) or decimal(M,D), M must be >= D (column '').") // test unhex and hex result = tk.MustQuery("select unhex('4D7953514C')") result.Check(testkit.Rows("MySQL")) From 62c68ed298a21ebc4d77003b6c27cd2e10bb8dac Mon Sep 17 00:00:00 2001 From: pengdaqian Date: Mon, 16 Nov 2020 01:40:24 +0800 Subject: [PATCH 04/11] fix fmt error --- ddl/db_integration_test.go | 19 +++++++++++++++++++ executor/executor_test.go | 20 -------------------- 2 files changed, 19 insertions(+), 20 deletions(-) diff --git a/ddl/db_integration_test.go b/ddl/db_integration_test.go index fe20bc54c028f..35a0616d9b629 100644 --- a/ddl/db_integration_test.go +++ b/ddl/db_integration_test.go @@ -2577,3 +2577,22 @@ func (s *testIntegrationSuite3) TestEnumAndSetDefaultValue(c *C) { c.Assert(tbl.Meta().Columns[0].DefaultValue, Equals, "a") c.Assert(tbl.Meta().Columns[1].DefaultValue, Equals, "a") } + +// TestSelectWithCast for issue #21063 +func (s *testIntegrationSuite) TestSelectWithCast(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec(`use test`) + tk.MustExec(`drop table if exists t`) + defer tk.MustExec(`drop table if exists t`) + tk.MustExec(`create table t(d decimal(10, 5))`) + tk.MustGetErrCode(`select * from t where d = cast(d as decimal(10,20))`, errno.ErrMBiggerThanD) + tk.MustGetErrCode(`select cast(111 as decimal(1,20)) from t`, errno.ErrMBiggerThanD) + tk.MustGetErrCode(`select * from t where d = cast(111 as decimal(1000,20))`, errno.ErrTooBigPrecision) + tk.MustGetErrCode(`select cast(111 as decimal(1000,20)) from t`, errno.ErrTooBigPrecision) + tk.MustGetErrMsg(`select * from t where d = cast(d as decimal(10,20))`, `[types:1427]For float(M,D), double(M,D) or decimal(M,D), M must be >= D (column '').`) + tk.MustGetErrMsg(`select * from t where d = cast("d" as decimal(10,20))`, `[types:1427]For float(M,D), double(M,D) or decimal(M,D), M must be >= D (column '').`) + tk.MustGetErrMsg(`select * from t where d = cast(111 as decimal(1000,20))`, `[types:1426]Too big precision 1000 specified for column '111'. Maximum is 65.`) + tk.MustGetErrMsg(`select * from t where d = cast("abc" as decimal(1000,20))`, `[types:1426]Too big precision 1000 specified for column '"abc"'. Maximum is 65.`) + tk.MustGetErrMsg(`select * from t where d = cast(d as decimal(1000,20))`, `[types:1426]Too big precision 1000 specified for column 'd'. Maximum is 65.`) + tk.MustGetErrMsg(`select * from t where d = cast('d' as decimal(1000,20))`, `[types:1426]Too big precision 1000 specified for column ''d''. Maximum is 65.`) +} diff --git a/executor/executor_test.go b/executor/executor_test.go index 527cbb03f34f0..972af5187a918 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -17,7 +17,6 @@ import ( "context" "flag" "fmt" - "github.com/pingcap/tidb/errno" "io/ioutil" "math" "net" @@ -6850,22 +6849,3 @@ func (s *testSuite) TestIssue19667(c *C) { tk.MustExec("INSERT INTO t VALUES('1988-04-17 01:59:59')") tk.MustQuery(`SELECT DATE_ADD(a, INTERVAL 1 SECOND) FROM t`).Check(testkit.Rows("1988-04-17 02:00:00")) } - -// TestSelectWithCast for issue #21063 -func (s *testSuite) TestSelectWithCast(c *C) { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec(`use test`) - tk.MustExec(`drop table if exists t`) - defer tk.MustExec(`drop table if exists t`) - tk.MustExec(`create table t(d decimal(10, 5))`) - tk.MustGetErrCode(`select * from t where d = cast(d as decimal(10,20))`, errno.ErrMBiggerThanD) - tk.MustGetErrCode(`select cast(111 as decimal(1,20)) from t`, errno.ErrMBiggerThanD) - tk.MustGetErrCode(`select * from t where d = cast(111 as decimal(1000,20))`, errno.ErrTooBigPrecision) - tk.MustGetErrCode(`select cast(111 as decimal(1000,20)) from t`, errno.ErrTooBigPrecision) - tk.MustGetErrMsg(`select * from t where d = cast(d as decimal(10,20))`, `[types:1427]For float(M,D), double(M,D) or decimal(M,D), M must be >= D (column '').`) - tk.MustGetErrMsg(`select * from t where d = cast("d" as decimal(10,20))`, `[types:1427]For float(M,D), double(M,D) or decimal(M,D), M must be >= D (column '').`) - tk.MustGetErrMsg(`select * from t where d = cast(111 as decimal(1000,20))`, `[types:1426]Too big precision 1000 specified for column '111'. Maximum is 65.`) - tk.MustGetErrMsg(`select * from t where d = cast("abc" as decimal(1000,20))`, `[types:1426]Too big precision 1000 specified for column '"abc"'. Maximum is 65.`) - tk.MustGetErrMsg(`select * from t where d = cast(d as decimal(1000,20))`, `[types:1426]Too big precision 1000 specified for column 'd'. Maximum is 65.`) - tk.MustGetErrMsg(`select * from t where d = cast('d' as decimal(1000,20))`, `[types:1426]Too big precision 1000 specified for column ''d''. Maximum is 65.`) -} From 4b938ce32c83fd007d18455248e5a24ca73c8ca1 Mon Sep 17 00:00:00 2001 From: pengdaqian Date: Mon, 16 Nov 2020 02:34:26 +0800 Subject: [PATCH 05/11] fix lint error --- planner/core/preprocess.go | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/planner/core/preprocess.go b/planner/core/preprocess.go index 205c7e95b750c..f33aa11c1cc57 100644 --- a/planner/core/preprocess.go +++ b/planner/core/preprocess.go @@ -869,11 +869,13 @@ func checkTp(tp *types.FieldType, colName, val string) error { } if tp.Flen > mysql.MaxDecimalWidth { + var arg string if colName == "" { - return types.ErrTooBigPrecision.GenWithStackByArgs(tp.Flen, val, mysql.MaxDecimalWidth) + arg = val } else { - return types.ErrTooBigPrecision.GenWithStackByArgs(tp.Flen, colName, mysql.MaxDecimalWidth) + arg = colName } + return types.ErrTooBigPrecision.GenWithStackByArgs(tp.Flen, arg, mysql.MaxDecimalWidth) } if tp.Flen < tp.Decimal { From 397c807918642c08507c0943f95cf0fcf57b6c8f Mon Sep 17 00:00:00 2001 From: pengdaqian Date: Mon, 16 Nov 2020 02:42:32 +0800 Subject: [PATCH 06/11] fix lint error --- ddl/db_integration_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ddl/db_integration_test.go b/ddl/db_integration_test.go index 35a0616d9b629..06b9bf5bb2f91 100644 --- a/ddl/db_integration_test.go +++ b/ddl/db_integration_test.go @@ -2579,7 +2579,7 @@ func (s *testIntegrationSuite3) TestEnumAndSetDefaultValue(c *C) { } // TestSelectWithCast for issue #21063 -func (s *testIntegrationSuite) TestSelectWithCast(c *C) { +func (s *testIntegrationSuite3) TestSelectWithCast(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec(`use test`) tk.MustExec(`drop table if exists t`) From 8c20fcf094b4fdf566ae5d9300f818a78af2f986 Mon Sep 17 00:00:00 2001 From: pengdaqian Date: Mon, 16 Nov 2020 08:47:48 +0800 Subject: [PATCH 07/11] change preprocess function sign --- executor/adapter.go | 2 +- executor/compiler.go | 2 +- executor/executor_test.go | 4 ++-- executor/metrics_reader_test.go | 2 +- executor/prepared.go | 2 +- expression/integration_test.go | 2 +- expression/typeinfer_test.go | 2 +- planner/core/cbo_test.go | 10 +++++----- planner/core/common_plans.go | 2 +- planner/core/indexmerge_test.go | 2 +- planner/core/logical_plan_test.go | 28 ++++++++++++++-------------- planner/core/physical_plan_test.go | 4 ++-- planner/core/point_get_plan_test.go | 2 +- planner/core/preprocess.go | 4 ++-- planner/core/preprocess_test.go | 2 +- planner/core/stats_test.go | 2 +- server/conn.go | 2 +- statistics/selectivity_test.go | 6 +++--- util/ranger/ranger_test.go | 10 +++++----- 19 files changed, 45 insertions(+), 45 deletions(-) diff --git a/executor/adapter.go b/executor/adapter.go index 74085510e16ef..4cf27e849d4d6 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -256,7 +256,7 @@ func (a *ExecStmt) IsReadOnly(vars *variable.SessionVars) bool { func (a *ExecStmt) RebuildPlan(ctx context.Context) (int64, error) { is := infoschema.GetInfoSchema(a.Ctx) a.InfoSchema = is - if err := plannercore.Preprocess(a.Ctx, a.StmtNode, a.StmtNode.Text(), is, plannercore.InTxnRetry); err != nil { + if err := plannercore.Preprocess(a.Ctx, a.StmtNode, is, plannercore.InTxnRetry); err != nil { return 0, err } p, names, err := planner.Optimize(ctx, a.Ctx, a.StmtNode, is) diff --git a/executor/compiler.go b/executor/compiler.go index 8e8b3bbc5ff4e..49ae9e3e844da 100644 --- a/executor/compiler.go +++ b/executor/compiler.go @@ -54,7 +54,7 @@ func (c *Compiler) Compile(ctx context.Context, stmtNode ast.StmtNode) (*ExecStm } infoSchema := infoschema.GetInfoSchema(c.Ctx) - if err := plannercore.Preprocess(c.Ctx, stmtNode, stmtNode.Text(), infoSchema); err != nil { + if err := plannercore.Preprocess(c.Ctx, stmtNode, infoSchema); err != nil { return nil, err } stmtNode = plannercore.TryAddExtraLimit(c.Ctx, stmtNode) diff --git a/executor/executor_test.go b/executor/executor_test.go index 972af5187a918..68c197e5b07c6 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -2259,7 +2259,7 @@ func (s *testSuiteP2) TestIsPointGet(c *C) { for sqlStr, result := range tests { stmtNode, err := s.ParseOneStmt(sqlStr, "", "") c.Check(err, IsNil) - err = plannercore.Preprocess(ctx, stmtNode, stmtNode.Text(), infoSchema) + err = plannercore.Preprocess(ctx, stmtNode, infoSchema) c.Check(err, IsNil) p, _, err := planner.Optimize(context.TODO(), ctx, stmtNode, infoSchema) c.Check(err, IsNil) @@ -2290,7 +2290,7 @@ func (s *testSuiteP2) TestClusteredIndexIsPointGet(c *C) { for sqlStr, result := range tests { stmtNode, err := s.ParseOneStmt(sqlStr, "", "") c.Check(err, IsNil) - err = plannercore.Preprocess(ctx, stmtNode, stmtNode.Text(), infoSchema) + err = plannercore.Preprocess(ctx, stmtNode, infoSchema) c.Check(err, IsNil) p, _, err := planner.Optimize(context.TODO(), ctx, stmtNode, infoSchema) c.Check(err, IsNil) diff --git a/executor/metrics_reader_test.go b/executor/metrics_reader_test.go index 057f8b05e79ca..19000b4faee5d 100644 --- a/executor/metrics_reader_test.go +++ b/executor/metrics_reader_test.go @@ -63,7 +63,7 @@ func (s *testSuite7) TestStmtLabel(c *C) { stmtNode, err := parser.New().ParseOneStmt(tt.sql, "", "") c.Check(err, IsNil) is := infoschema.GetInfoSchema(tk.Se) - err = plannercore.Preprocess(tk.Se.(sessionctx.Context), stmtNode, stmtNode.Text(), is) + err = plannercore.Preprocess(tk.Se.(sessionctx.Context), stmtNode, is) c.Assert(err, IsNil) _, _, err = planner.Optimize(context.TODO(), tk.Se, stmtNode, is) c.Assert(err, IsNil) diff --git a/executor/prepared.go b/executor/prepared.go index 9a2640e8dcc99..278fd84d4dad0 100644 --- a/executor/prepared.go +++ b/executor/prepared.go @@ -153,7 +153,7 @@ func (e *PrepareExec) Next(ctx context.Context, req *chunk.Chunk) error { return ErrPsManyParam } - err = plannercore.Preprocess(e.ctx, stmt, stmt.Text(), e.is, plannercore.InPrepare) + err = plannercore.Preprocess(e.ctx, stmt, e.is, plannercore.InPrepare) if err != nil { return err } diff --git a/expression/integration_test.go b/expression/integration_test.go index 5db37a77b0491..c47d29e59f580 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -4520,7 +4520,7 @@ func (s *testIntegrationSuite) TestFilterExtractFromDNF(c *C) { c.Assert(err, IsNil, Commentf("error %v, for expr %s", err, tt.exprStr)) c.Assert(stmts, HasLen, 1) is := domain.GetDomain(sctx).InfoSchema() - err = plannercore.Preprocess(sctx, stmts[0], stmts[0].Text(), is) + err = plannercore.Preprocess(sctx, stmts[0], is) c.Assert(err, IsNil, Commentf("error %v, for resolve name, expr %s", err, tt.exprStr)) p, _, err := plannercore.BuildLogicalPlan(ctx, sctx, stmts[0], is) c.Assert(err, IsNil, Commentf("error %v, for build plan, expr %s", err, tt.exprStr)) diff --git a/expression/typeinfer_test.go b/expression/typeinfer_test.go index 04ba05eeef926..ea82d891d07b7 100644 --- a/expression/typeinfer_test.go +++ b/expression/typeinfer_test.go @@ -140,7 +140,7 @@ func (s *testInferTypeSuite) TestInferType(c *C) { c.Assert(err, IsNil) is := domain.GetDomain(sctx).InfoSchema() - err = plannercore.Preprocess(sctx, stmt, stmt.Text(), is) + err = plannercore.Preprocess(sctx, stmt, is) c.Assert(err, IsNil, comment) p, _, err := plannercore.BuildLogicalPlan(ctx, sctx, stmt, is) c.Assert(err, IsNil, comment) diff --git a/planner/core/cbo_test.go b/planner/core/cbo_test.go index 40ab9ee7edc55..4394a6e006a38 100644 --- a/planner/core/cbo_test.go +++ b/planner/core/cbo_test.go @@ -277,7 +277,7 @@ func (s *testAnalyzeSuite) TestIndexRead(c *C) { c.Assert(stmts, HasLen, 1) stmt := stmts[0] is := domain.GetDomain(ctx).InfoSchema() - err = core.Preprocess(ctx, stmt, stmt.Text(), is) + err = core.Preprocess(ctx, stmt, is) c.Assert(err, IsNil) p, _, err := planner.Optimize(context.TODO(), ctx, stmt, is) c.Assert(err, IsNil) @@ -312,7 +312,7 @@ func (s *testAnalyzeSuite) TestEmptyTable(c *C) { c.Assert(stmts, HasLen, 1) stmt := stmts[0] is := domain.GetDomain(ctx).InfoSchema() - err = core.Preprocess(ctx, stmt, stmt.Text(), is) + err = core.Preprocess(ctx, stmt, is) c.Assert(err, IsNil) p, _, err := planner.Optimize(context.TODO(), ctx, stmt, is) c.Assert(err, IsNil) @@ -383,7 +383,7 @@ func (s *testAnalyzeSuite) TestAnalyze(c *C) { err = executor.ResetContextOfStmt(ctx, stmt) c.Assert(err, IsNil) is := domain.GetDomain(ctx).InfoSchema() - err = core.Preprocess(ctx, stmt, stmt.Text(), is) + err = core.Preprocess(ctx, stmt, is) c.Assert(err, IsNil) p, _, err := planner.Optimize(context.TODO(), ctx, stmt, is) c.Assert(err, IsNil) @@ -463,7 +463,7 @@ func (s *testAnalyzeSuite) TestPreparedNullParam(c *C) { stmt := stmts[0] is := domain.GetDomain(ctx).InfoSchema() - err = core.Preprocess(ctx, stmt, stmt.Text(), is, core.InPrepare) + err = core.Preprocess(ctx, stmt, is, core.InPrepare) c.Assert(err, IsNil) p, _, err := planner.Optimize(context.TODO(), ctx, stmt, is) c.Assert(err, IsNil) @@ -692,7 +692,7 @@ func BenchmarkOptimize(b *testing.B) { c.Assert(stmts, HasLen, 1) stmt := stmts[0] is := domain.GetDomain(ctx).InfoSchema() - err = core.Preprocess(ctx, stmt, stmt.Text(), is) + err = core.Preprocess(ctx, stmt, is) c.Assert(err, IsNil) b.Run(tt.sql, func(b *testing.B) { diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index bb219b160e5b6..107eea6352269 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -243,7 +243,7 @@ func (e *Execute) OptimizePreparedPlan(ctx context.Context, sctx sessionctx.Cont preparedObj.Executor = nil // If the schema version has changed we need to preprocess it again, // if this time it failed, the real reason for the error is schema changed. - err := Preprocess(sctx, prepared.Stmt, prepared.Stmt.Text(), is, InPrepare) + err := Preprocess(sctx, prepared.Stmt, is, InPrepare) if err != nil { return ErrSchemaChanged.GenWithStack("Schema change caused error: %s", err.Error()) } diff --git a/planner/core/indexmerge_test.go b/planner/core/indexmerge_test.go index f866d6301e8eb..8df97cb27ac28 100644 --- a/planner/core/indexmerge_test.go +++ b/planner/core/indexmerge_test.go @@ -90,7 +90,7 @@ func (s *testIndexMergeSuite) TestIndexMergePathGeneration(c *C) { comment := Commentf("case:%v sql:%s", i, tc) stmt, err := s.ParseOneStmt(tc, "", "") c.Assert(err, IsNil, comment) - Preprocess(s.ctx, stmt, stmt.Text(), s.is) + Preprocess(s.ctx, stmt, s.is) builder := NewPlanBuilder(MockContext(), s.is, &hint.BlockHintProcessor{}) p, err := builder.Build(ctx, stmt) if err != nil { diff --git a/planner/core/logical_plan_test.go b/planner/core/logical_plan_test.go index c72fca12c800f..6e8f6e0ef033f 100644 --- a/planner/core/logical_plan_test.go +++ b/planner/core/logical_plan_test.go @@ -442,7 +442,7 @@ func (s *testPlanSuite) TestSubquery(c *C) { stmt, err := s.ParseOneStmt(ca, "", "") c.Assert(err, IsNil, comment) - Preprocess(s.ctx, stmt, stmt.Text(), s.is) + Preprocess(s.ctx, stmt, s.is) p, _, err := BuildLogicalPlan(ctx, s.ctx, stmt, s.is) c.Assert(err, IsNil) if lp, ok := p.(LogicalPlan); ok { @@ -467,7 +467,7 @@ func (s *testPlanSuite) TestPlanBuilder(c *C) { c.Assert(err, IsNil, comment) s.ctx.GetSessionVars().SetHashJoinConcurrency(1) - Preprocess(s.ctx, stmt, stmt.Text(), s.is) + Preprocess(s.ctx, stmt, s.is) p, _, err := BuildLogicalPlan(ctx, s.ctx, stmt, s.is) c.Assert(err, IsNil) if lp, ok := p.(LogicalPlan); ok { @@ -827,7 +827,7 @@ func (s *testPlanSuite) TestValidate(c *C) { comment := Commentf("for %s", sql) stmt, err := s.ParseOneStmt(sql, "", "") c.Assert(err, IsNil, comment) - Preprocess(s.ctx, stmt, stmt.Text(), s.is) + Preprocess(s.ctx, stmt, s.is) _, _, err = BuildLogicalPlan(ctx, s.ctx, stmt, s.is) if tt.err == nil { c.Assert(err, IsNil, comment) @@ -1117,7 +1117,7 @@ func (s *testPlanSuite) TestVisitInfo(c *C) { comment := Commentf("for %s", tt.sql) stmt, err := s.ParseOneStmt(tt.sql, "", "") c.Assert(err, IsNil, comment) - Preprocess(s.ctx, stmt, stmt.Text(), s.is) + Preprocess(s.ctx, stmt, s.is) builder := NewPlanBuilder(MockContext(), s.is, &hint.BlockHintProcessor{}) builder.ctx.GetSessionVars().SetHashJoinConcurrency(1) _, err = builder.Build(context.TODO(), stmt) @@ -1197,7 +1197,7 @@ func (s *testPlanSuite) TestUnion(c *C) { comment := Commentf("case:%v sql:%s", i, tt) stmt, err := s.ParseOneStmt(tt, "", "") c.Assert(err, IsNil, comment) - Preprocess(s.ctx, stmt, stmt.Text(), s.is) + Preprocess(s.ctx, stmt, s.is) builder := NewPlanBuilder(MockContext(), s.is, &hint.BlockHintProcessor{}) plan, err := builder.Build(ctx, stmt) s.testData.OnRecord(func() { @@ -1229,7 +1229,7 @@ func (s *testPlanSuite) TestTopNPushDown(c *C) { comment := Commentf("case:%v sql:%s", i, tt) stmt, err := s.ParseOneStmt(tt, "", "") c.Assert(err, IsNil, comment) - Preprocess(s.ctx, stmt, stmt.Text(), s.is) + Preprocess(s.ctx, stmt, s.is) builder := NewPlanBuilder(MockContext(), s.is, &hint.BlockHintProcessor{}) p, err := builder.Build(ctx, stmt) c.Assert(err, IsNil) @@ -1303,7 +1303,7 @@ func (s *testPlanSuite) TestOuterJoinEliminator(c *C) { comment := Commentf("case:%v sql:%s", i, tt) stmt, err := s.ParseOneStmt(tt, "", "") c.Assert(err, IsNil, comment) - Preprocess(s.ctx, stmt, stmt.Text(), s.is) + Preprocess(s.ctx, stmt, s.is) builder := NewPlanBuilder(MockContext(), s.is, &hint.BlockHintProcessor{}) p, err := builder.Build(ctx, stmt) c.Assert(err, IsNil) @@ -1339,7 +1339,7 @@ func (s *testPlanSuite) TestSelectView(c *C) { comment := Commentf("case:%v sql:%s", i, tt.sql) stmt, err := s.ParseOneStmt(tt.sql, "", "") c.Assert(err, IsNil, comment) - Preprocess(s.ctx, stmt, stmt.Text(), s.is) + Preprocess(s.ctx, stmt, s.is) builder := NewPlanBuilder(MockContext(), s.is, &hint.BlockHintProcessor{}) p, err := builder.Build(ctx, stmt) c.Assert(err, IsNil) @@ -1410,7 +1410,7 @@ func (s *testPlanSuite) optimize(ctx context.Context, sql string) (PhysicalPlan, if err != nil { return nil, nil, err } - err = Preprocess(s.ctx, stmt, stmt.Text(), s.is) + err = Preprocess(s.ctx, stmt, s.is) if err != nil { return nil, nil, err } @@ -1502,7 +1502,7 @@ func (s *testPlanSuite) TestSkylinePruning(c *C) { comment := Commentf("case:%v sql:%s", i, tt.sql) stmt, err := s.ParseOneStmt(tt.sql, "", "") c.Assert(err, IsNil, comment) - Preprocess(s.ctx, stmt, stmt.Text(), s.is) + Preprocess(s.ctx, stmt, s.is) builder := NewPlanBuilder(MockContext(), s.is, &hint.BlockHintProcessor{}) p, err := builder.Build(ctx, stmt) if err != nil { @@ -1572,7 +1572,7 @@ func (s *testPlanSuite) TestFastPlanContextTables(c *C) { for _, tt := range tests { stmt, err := s.ParseOneStmt(tt.sql, "", "") c.Assert(err, IsNil) - Preprocess(s.ctx, stmt, stmt.Text(), s.is) + Preprocess(s.ctx, stmt, s.is) s.ctx.GetSessionVars().StmtCtx.Tables = nil p := TryFastPlan(s.ctx, stmt) if tt.fastPlan { @@ -1603,7 +1603,7 @@ func (s *testPlanSuite) TestUpdateEQCond(c *C) { comment := Commentf("case:%v sql:%s", i, tt.sql) stmt, err := s.ParseOneStmt(tt.sql, "", "") c.Assert(err, IsNil, comment) - Preprocess(s.ctx, stmt, stmt.Text(), s.is) + Preprocess(s.ctx, stmt, s.is) builder := NewPlanBuilder(MockContext(), s.is, &hint.BlockHintProcessor{}) p, err := builder.Build(ctx, stmt) c.Assert(err, IsNil) @@ -1619,7 +1619,7 @@ func (s *testPlanSuite) TestConflictedJoinTypeHints(c *C) { ctx := context.TODO() stmt, err := s.ParseOneStmt(sql, "", "") c.Assert(err, IsNil) - Preprocess(s.ctx, stmt, stmt.Text(), s.is) + Preprocess(s.ctx, stmt, s.is) builder := NewPlanBuilder(MockContext(), s.is, &hint.BlockHintProcessor{}) p, err := builder.Build(ctx, stmt) c.Assert(err, IsNil) @@ -1639,7 +1639,7 @@ func (s *testPlanSuite) TestSimplyOuterJoinWithOnlyOuterExpr(c *C) { ctx := context.TODO() stmt, err := s.ParseOneStmt(sql, "", "") c.Assert(err, IsNil) - Preprocess(s.ctx, stmt, stmt.Text(), s.is) + Preprocess(s.ctx, stmt, s.is) builder := NewPlanBuilder(MockContext(), s.is, &hint.BlockHintProcessor{}) p, err := builder.Build(ctx, stmt) c.Assert(err, IsNil) diff --git a/planner/core/physical_plan_test.go b/planner/core/physical_plan_test.go index 80d65f06bfbe3..62519e84ba530 100644 --- a/planner/core/physical_plan_test.go +++ b/planner/core/physical_plan_test.go @@ -248,7 +248,7 @@ func (s *testPlanSuite) TestDAGPlanBuilderBasePhysicalPlan(c *C) { stmt, err := s.ParseOneStmt(tt, "", "") c.Assert(err, IsNil, comment) - core.Preprocess(se, stmt, stmt.Text(), s.is) + core.Preprocess(se, stmt, s.is) p, _, err := planner.Optimize(context.TODO(), se, stmt, s.is) c.Assert(err, IsNil) s.testData.OnRecord(func() { @@ -1371,7 +1371,7 @@ func (s *testPlanSuite) TestDAGPlanBuilderSplitAvg(c *C) { stmt, err := s.ParseOneStmt(tt.sql, "", "") c.Assert(err, IsNil, comment) - core.Preprocess(se, stmt, stmt.Text(), s.is) + core.Preprocess(se, stmt, s.is) p, _, err := planner.Optimize(context.TODO(), se, stmt, s.is) c.Assert(err, IsNil, comment) diff --git a/planner/core/point_get_plan_test.go b/planner/core/point_get_plan_test.go index d257a3c214c85..c74a1140fc4a2 100644 --- a/planner/core/point_get_plan_test.go +++ b/planner/core/point_get_plan_test.go @@ -307,7 +307,7 @@ func (s *testPointGetSuite) TestPointGetId(c *C) { c.Assert(stmts, HasLen, 1) stmt := stmts[0] is := domain.GetDomain(ctx).InfoSchema() - err = core.Preprocess(ctx, stmt, stmt.Text(), is) + err = core.Preprocess(ctx, stmt, is) c.Assert(err, IsNil) p, _, err := planner.Optimize(context.TODO(), ctx, stmt, is) c.Assert(err, IsNil) diff --git a/planner/core/preprocess.go b/planner/core/preprocess.go index f33aa11c1cc57..a6a4854f47e5d 100644 --- a/planner/core/preprocess.go +++ b/planner/core/preprocess.go @@ -78,8 +78,8 @@ func TryAddExtraLimit(ctx sessionctx.Context, node ast.StmtNode) ast.StmtNode { } // Preprocess resolves table names of the node, and checks some statements validation. -func Preprocess(ctx sessionctx.Context, node ast.Node, sql string, is infoschema.InfoSchema, preprocessOpt ...PreprocessOpt) error { - v := preprocessor{is: is, ctx: ctx, sql: sql, tableAliasInJoin: make([]map[string]interface{}, 0)} +func Preprocess(ctx sessionctx.Context, node ast.Node, is infoschema.InfoSchema, preprocessOpt ...PreprocessOpt) error { + v := preprocessor{is: is, ctx: ctx, sql: node.Text(), tableAliasInJoin: make([]map[string]interface{}, 0)} for _, optFn := range preprocessOpt { optFn(&v) } diff --git a/planner/core/preprocess_test.go b/planner/core/preprocess_test.go index c7c931fa9fcb0..442dcee025b30 100644 --- a/planner/core/preprocess_test.go +++ b/planner/core/preprocess_test.go @@ -65,7 +65,7 @@ func (s *testValidatorSuite) runSQL(c *C, sql string, inPrepare bool, terr error if inPrepare { opts = append(opts, core.InPrepare) } - err := core.Preprocess(s.ctx, stmt, stmt.Text(), s.is, opts...) + err := core.Preprocess(s.ctx, stmt, s.is, opts...) c.Assert(terror.ErrorEqual(err, terr), IsTrue, Commentf("sql: %s, err:%v", sql, err)) } diff --git a/planner/core/stats_test.go b/planner/core/stats_test.go index 9eebf2a010b88..0b3176eb87b60 100644 --- a/planner/core/stats_test.go +++ b/planner/core/stats_test.go @@ -75,7 +75,7 @@ func (s *testStatsSuite) TestGroupNDVs(c *C) { comment := Commentf("case:%v sql: %s", i, tt) stmt, err := s.ParseOneStmt(tt, "", "") c.Assert(err, IsNil, comment) - core.Preprocess(tk.Se, stmt, stmt.Text(), is) + core.Preprocess(tk.Se, stmt, is) builder := core.NewPlanBuilder(tk.Se, is, &hint.BlockHintProcessor{}) p, err := builder.Build(ctx, stmt) c.Assert(err, IsNil, comment) diff --git a/server/conn.go b/server/conn.go index 405f2988d907d..7a63136e0ff3e 100644 --- a/server/conn.go +++ b/server/conn.go @@ -1468,7 +1468,7 @@ func (cc *clientConn) prefetchPointPlanKeys(ctx context.Context, stmts []ast.Stm sc := vars.StmtCtx for i, stmt := range stmts { // TODO: the preprocess is run twice, we should find some way to avoid do it again. - if err = plannercore.Preprocess(cc.ctx, stmt, stmt.Text(), is); err != nil { + if err = plannercore.Preprocess(cc.ctx, stmt, is); err != nil { return nil, err } p := plannercore.TryFastPlan(cc.ctx.Session, stmt) diff --git a/statistics/selectivity_test.go b/statistics/selectivity_test.go index 3bb549265d8cb..2b3fb3a3876c3 100644 --- a/statistics/selectivity_test.go +++ b/statistics/selectivity_test.go @@ -293,7 +293,7 @@ func (s *testStatsSuite) TestSelectivity(c *C) { c.Assert(err, IsNil, Commentf("error %v, for expr %s", err, tt.exprs)) c.Assert(stmts, HasLen, 1) - err = plannercore.Preprocess(sctx, stmts[0], stmts[0].Text(), is) + err = plannercore.Preprocess(sctx, stmts[0], is) c.Assert(err, IsNil, comment) p, _, err := plannercore.BuildLogicalPlan(ctx, sctx, stmts[0], is) c.Assert(err, IsNil, Commentf("error %v, for building plan, expr %s", err, tt.exprs)) @@ -549,7 +549,7 @@ func BenchmarkSelectivity(b *testing.B) { stmts, err := session.Parse(sctx, sql) c.Assert(err, IsNil, Commentf("error %v, for expr %s", err, exprs)) c.Assert(stmts, HasLen, 1) - err = plannercore.Preprocess(sctx, stmts[0], stmts[0].Text(), is) + err = plannercore.Preprocess(sctx, stmts[0], is) c.Assert(err, IsNil, comment) p, _, err := plannercore.BuildLogicalPlan(context.Background(), sctx, stmts[0], is) c.Assert(err, IsNil, Commentf("error %v, for building plan, expr %s", err, exprs)) @@ -705,7 +705,7 @@ func (s *testStatsSuite) TestDNFCondSelectivity(c *C) { c.Assert(err, IsNil, Commentf("error %v, for sql %s", err, tt)) c.Assert(stmts, HasLen, 1) - err = plannercore.Preprocess(sctx, stmts[0], stmts[0].Text(), is) + err = plannercore.Preprocess(sctx, stmts[0], is) c.Assert(err, IsNil, Commentf("error %v, for sql %s", err, tt)) p, _, err := plannercore.BuildLogicalPlan(ctx, sctx, stmts[0], is) c.Assert(err, IsNil, Commentf("error %v, for building plan, sql %s", err, tt)) diff --git a/util/ranger/ranger_test.go b/util/ranger/ranger_test.go index ce7999e1bde9e..3e9d7b8542210 100644 --- a/util/ranger/ranger_test.go +++ b/util/ranger/ranger_test.go @@ -304,7 +304,7 @@ func (s *testRangerSuite) TestTableRange(c *C) { c.Assert(err, IsNil, Commentf("error %v, for expr %s", err, tt.exprStr)) c.Assert(stmts, HasLen, 1) is := domain.GetDomain(sctx).InfoSchema() - err = plannercore.Preprocess(sctx, stmts[0], stmts[0].Text(), is) + err = plannercore.Preprocess(sctx, stmts[0], is) c.Assert(err, IsNil, Commentf("error %v, for resolve name, expr %s", err, tt.exprStr)) p, _, err := plannercore.BuildLogicalPlan(ctx, sctx, stmts[0], is) c.Assert(err, IsNil, Commentf("error %v, for build plan, expr %s", err, tt.exprStr)) @@ -631,7 +631,7 @@ create table t( c.Assert(err, IsNil, Commentf("error %v, for expr %s", err, tt.exprStr)) c.Assert(stmts, HasLen, 1) is := domain.GetDomain(sctx).InfoSchema() - err = plannercore.Preprocess(sctx, stmts[0], stmts[0].Text(), is) + err = plannercore.Preprocess(sctx, stmts[0], is) c.Assert(err, IsNil, Commentf("error %v, for resolve name, expr %s", err, tt.exprStr)) p, _, err := plannercore.BuildLogicalPlan(ctx, sctx, stmts[0], is) c.Assert(err, IsNil, Commentf("error %v, for build plan, expr %s", err, tt.exprStr)) @@ -752,7 +752,7 @@ func (s *testRangerSuite) TestIndexRangeForUnsignedInt(c *C) { c.Assert(err, IsNil, Commentf("error %v, for expr %s", err, tt.exprStr)) c.Assert(stmts, HasLen, 1) is := domain.GetDomain(sctx).InfoSchema() - err = plannercore.Preprocess(sctx, stmts[0], stmts[0].Text(), is) + err = plannercore.Preprocess(sctx, stmts[0], is) c.Assert(err, IsNil, Commentf("error %v, for resolve name, expr %s", err, tt.exprStr)) p, _, err := plannercore.BuildLogicalPlan(ctx, sctx, stmts[0], is) c.Assert(err, IsNil, Commentf("error %v, for build plan, expr %s", err, tt.exprStr)) @@ -1100,7 +1100,7 @@ func (s *testRangerSuite) TestColumnRange(c *C) { c.Assert(err, IsNil, Commentf("error %v, for expr %s", err, tt.exprStr)) c.Assert(stmts, HasLen, 1) is := domain.GetDomain(sctx).InfoSchema() - err = plannercore.Preprocess(sctx, stmts[0], stmts[0].Text(), is) + err = plannercore.Preprocess(sctx, stmts[0], is) c.Assert(err, IsNil, Commentf("error %v, for resolve name, expr %s", err, tt.exprStr)) p, _, err := plannercore.BuildLogicalPlan(ctx, sctx, stmts[0], is) c.Assert(err, IsNil, Commentf("error %v, for build plan, expr %s", err, tt.exprStr)) @@ -1438,7 +1438,7 @@ func (s *testRangerSuite) TestIndexRangeForYear(c *C) { c.Assert(err, IsNil, Commentf("error %v, for expr %s", err, tt.exprStr)) c.Assert(stmts, HasLen, 1) is := domain.GetDomain(sctx).InfoSchema() - err = plannercore.Preprocess(sctx, stmts[0], stmts[0].Text(), is) + err = plannercore.Preprocess(sctx, stmts[0], is) c.Assert(err, IsNil, Commentf("error %v, for resolve name, expr %s", err, tt.exprStr)) p, _, err := plannercore.BuildLogicalPlan(ctx, sctx, stmts[0], is) c.Assert(err, IsNil, Commentf("error %v, for build plan, expr %s", err, tt.exprStr)) From dc602b6362557f714f61ed9bdb04626a7e5b4cbb Mon Sep 17 00:00:00 2001 From: pengdaqian Date: Mon, 16 Nov 2020 22:25:37 +0800 Subject: [PATCH 08/11] delete dup code --- expression/integration_test.go | 2 -- 1 file changed, 2 deletions(-) diff --git a/expression/integration_test.go b/expression/integration_test.go index 4363dd21d251f..86bfcc4d46264 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -2752,8 +2752,6 @@ func (s *testIntegrationSuite2) TestBuiltin(c *C) { _, err = tk.Exec("insert into t values(-9223372036854775809)") c.Assert(err, NotNil) - // test case decimal precision less than the scale. - tk.MustGetErrMsg("select cast(12.1 as decimal(3, 4));", "[types:1427]For float(M,D), double(M,D) or decimal(M,D), M must be >= D (column '').") // test unhex and hex result = tk.MustQuery("select unhex('4D7953514C')") result.Check(testkit.Rows("MySQL")) From 6d98ccd59b4b25038405503b332ad6c206119f38 Mon Sep 17 00:00:00 2001 From: pengdaqian Date: Tue, 17 Nov 2020 11:55:28 +0800 Subject: [PATCH 09/11] change preprocess --- planner/core/preprocess.go | 80 +++++++++++++++++++++++++-------- planner/core/preprocess_test.go | 26 ++++++++++- 2 files changed, 87 insertions(+), 19 deletions(-) diff --git a/planner/core/preprocess.go b/planner/core/preprocess.go index 11b44d83e038d..145af99891e19 100644 --- a/planner/core/preprocess.go +++ b/planner/core/preprocess.go @@ -79,7 +79,7 @@ func TryAddExtraLimit(ctx sessionctx.Context, node ast.StmtNode) ast.StmtNode { // Preprocess resolves table names of the node, and checks some statements validation. func Preprocess(ctx sessionctx.Context, node ast.Node, is infoschema.InfoSchema, preprocessOpt ...PreprocessOpt) error { - v := preprocessor{is: is, ctx: ctx, tableAliasInJoin: make([]map[string]interface{}, 0)} + v := preprocessor{is: is, ctx: ctx, sql: node.Text(), tableAliasInJoin: make([]map[string]interface{}, 0)} for _, optFn := range preprocessOpt { optFn(&v) } @@ -110,6 +110,7 @@ const ( type preprocessor struct { is infoschema.InfoSchema ctx sessionctx.Context + sql string err error flag preprocessorFlag @@ -184,6 +185,8 @@ func (p *preprocessor) Enter(in ast.Node) (out ast.Node, skipChildren bool) { if node.FnName.L == ast.NextVal || node.FnName.L == ast.LastVal || node.FnName.L == ast.SetVal { p.flag |= inSequenceFunction } + case *ast.FuncCastExpr: + p.checkCastGrammar(node) case *ast.BRIEStmt: if node.Kind == ast.BRIEKindRestore { p.flag |= inCreateOrDropTable @@ -802,18 +805,21 @@ func checkColumn(colDef *ast.ColumnDef) error { } // Check column type. - tp := colDef.Tp + return checkTp(colDef.Tp, colDef.Name.Name.O, "") +} + +func checkTp(tp *types.FieldType, colName, val string) error { if tp == nil { return nil } if tp.Flen > math.MaxUint32 { - return types.ErrTooBigDisplayWidth.GenWithStack("Display width out of range for column '%s' (max = %d)", colDef.Name.Name.O, math.MaxUint32) + return types.ErrTooBigDisplayWidth.GenWithStack("Display width out of range for column '%s' (max = %d)", colName, math.MaxUint32) } switch tp.Tp { case mysql.TypeString: if tp.Flen != types.UnspecifiedLength && tp.Flen > mysql.MaxFieldCharLength { - return types.ErrTooBigFieldLength.GenWithStack("Column length too big for column '%s' (max = %d); use BLOB or TEXT instead", colDef.Name.Name.O, mysql.MaxFieldCharLength) + return types.ErrTooBigFieldLength.GenWithStack("Column length too big for column '%s' (max = %d); use BLOB or TEXT instead", colName, mysql.MaxFieldCharLength) } case mysql.TypeVarchar: if len(tp.Charset) == 0 { @@ -822,7 +828,7 @@ func checkColumn(colDef *ast.ColumnDef) error { // return nil, to make the check in the ddl.CreateTable. return nil } - err := ddl.IsTooBigFieldLength(colDef.Tp.Flen, colDef.Name.Name.O, tp.Charset) + err := ddl.IsTooBigFieldLength(tp.Flen, colName, tp.Charset) if err != nil { return err } @@ -830,46 +836,64 @@ func checkColumn(colDef *ast.ColumnDef) error { // For FLOAT, the SQL standard permits an optional specification of the precision. // https://dev.mysql.com/doc/refman/8.0/en/floating-point-types.html if tp.Decimal == -1 { - switch tp.Tp { - case mysql.TypeDouble: - // For Double type Flen and Decimal check is moved to parser component - default: + if tp.Tp == mysql.TypeDouble { + if tp.Flen != -1 && colName != "" { + return types.ErrSyntax.GenWithStackByArgs() + } + } else { if tp.Flen > mysql.MaxDoublePrecisionLength { - return types.ErrWrongFieldSpec.GenWithStackByArgs(colDef.Name.Name.O) + return types.ErrWrongFieldSpec.GenWithStackByArgs(colName) } } } else { if tp.Decimal > mysql.MaxFloatingTypeScale { - return types.ErrTooBigScale.GenWithStackByArgs(tp.Decimal, colDef.Name.Name.O, mysql.MaxFloatingTypeScale) + return types.ErrTooBigScale.GenWithStackByArgs(tp.Decimal, colName, mysql.MaxFloatingTypeScale) } if tp.Flen > mysql.MaxFloatingTypeWidth { - return types.ErrTooBigDisplayWidth.GenWithStackByArgs(colDef.Name.Name.O, mysql.MaxFloatingTypeWidth) + return types.ErrTooBigDisplayWidth.GenWithStackByArgs(colName, mysql.MaxFloatingTypeWidth) } } case mysql.TypeSet: if len(tp.Elems) > mysql.MaxTypeSetMembers { - return types.ErrTooBigSet.GenWithStack("Too many strings for column %s and SET", colDef.Name.Name.O) + return types.ErrTooBigSet.GenWithStack("Too many strings for column %s and SET", colName) } // Check set elements. See https://dev.mysql.com/doc/refman/5.7/en/set.html. - for _, str := range colDef.Tp.Elems { + for _, str := range tp.Elems { if strings.Contains(str, ",") { return types.ErrIllegalValueForType.GenWithStackByArgs(types.TypeStr(tp.Tp), str) } } case mysql.TypeNewDecimal: if tp.Decimal > mysql.MaxDecimalScale { - return types.ErrTooBigScale.GenWithStackByArgs(tp.Decimal, colDef.Name.Name.O, mysql.MaxDecimalScale) + var arg string + if colName == "" { + arg = val + } else { + arg = colName + } + return types.ErrTooBigScale.GenWithStackByArgs(tp.Decimal, arg, mysql.MaxDecimalScale) + } if tp.Flen > mysql.MaxDecimalWidth { - return types.ErrTooBigPrecision.GenWithStackByArgs(tp.Flen, colDef.Name.Name.O, mysql.MaxDecimalWidth) + var arg string + if colName == "" { + arg = val + } else { + arg = colName + } + return types.ErrTooBigPrecision.GenWithStackByArgs(tp.Flen, arg, mysql.MaxDecimalWidth) + } + + if tp.Flen < tp.Decimal { + return types.ErrMBiggerThanD.GenWithStackByArgs(colName) } case mysql.TypeBit: if tp.Flen <= 0 { - return types.ErrInvalidFieldSize.GenWithStackByArgs(colDef.Name.Name.O) + return types.ErrInvalidFieldSize.GenWithStackByArgs(colName) } if tp.Flen > mysql.MaxBitDisplayWidth { - return types.ErrTooBigDisplayWidth.GenWithStackByArgs(colDef.Name.Name.O, mysql.MaxBitDisplayWidth) + return types.ErrTooBigDisplayWidth.GenWithStackByArgs(colName, mysql.MaxBitDisplayWidth) } default: // TODO: Add more types. @@ -1063,3 +1087,23 @@ func (p *preprocessor) resolveCreateSequenceStmt(stmt *ast.CreateSequenceStmt) { return } } +func (p *preprocessor) checkCastGrammar(node *ast.FuncCastExpr) { + var val string + switch x := node.Expr.(type) { + case ast.ValueExpr: + val = x.GetDatumString() + if val == "" { + val = fmt.Sprintf("%v", x.GetValue()) + } else { + wrapChar := p.sql[x.OriginTextPosition() : x.OriginTextPosition()+1] + val = wrapChar + val + wrapChar + } + case *ast.ColumnNameExpr: + val = x.Name.Name.O + default: + } + if err := checkTp(node.Tp, "", val); err != nil { + p.err = err + return + } +} diff --git a/planner/core/preprocess_test.go b/planner/core/preprocess_test.go index 4f6d90422a391..8aa924ec3ee77 100644 --- a/planner/core/preprocess_test.go +++ b/planner/core/preprocess_test.go @@ -15,7 +15,6 @@ package core_test import ( "context" - . "github.com/pingcap/check" "github.com/pingcap/errors" "github.com/pingcap/parser" @@ -269,6 +268,31 @@ func (s *testValidatorSuite) TestValidator(c *C) { {"CREATE TABLE t (a int, index(a));", false, nil}, {"CREATE INDEX `` on t (a);", true, errors.New("[ddl:1280]Incorrect index name ''")}, {"CREATE INDEX `` on t ((lower(a)));", true, errors.New("[ddl:1280]Incorrect index name ''")}, + + // for ErrTooBigScale + {`select convert('0.0', Decimal(41, 40))`, false, errors.New(`[types:1425]Too big scale 40 specified for column ''0.0''. Maximum is 30.`)}, + // for cast decimal ErrTooBigPrecision + {`select * from t where d = cast(d as decimal(1000,20))`, false, errors.New(`[types:1426]Too big precision 1000 specified for column 'd'. Maximum is 65.`)}, + {`select * from t where d = cast(111 as decimal(1000,20))`, false, errors.New(`[types:1426]Too big precision 1000 specified for column '111'. Maximum is 65.`)}, + {`select * from t where d = cast("abc" as decimal(1000,20))`, false, errors.New(`[types:1426]Too big precision 1000 specified for column '"abc"'. Maximum is 65.`)}, + {`select * from t where d = cast('d' as decimal(1000,20))`, false, errors.New(`[types:1426]Too big precision 1000 specified for column ''d''. Maximum is 65.`)}, + {`select cast(d as decimal(1000,20))`, false, errors.New(`[types:1426]Too big precision 1000 specified for column 'd'. Maximum is 65.`)}, + {`select cast(111 as decimal(1000,20))`, false, errors.New(`[types:1426]Too big precision 1000 specified for column '111'. Maximum is 65.`)}, + {`select cast("abc" as decimal(1000,20))`, false, errors.New(`[types:1426]Too big precision 1000 specified for column '"abc"'. Maximum is 65.`)}, + {`select cast("'d'" as decimal(1000,20))`, false, errors.New(`[types:1426]Too big precision 1000 specified for column '"'d'"'. Maximum is 65.`)}, + // for cast decimal ErrMBiggerThanD + {`select * from t where d = cast(d as decimal(10,20))`, false, errors.New(`[types:1427]For float(M,D), double(M,D) or decimal(M,D), M must be >= D (column '').`)}, + {`select * from t where d = cast("d" as decimal(10,20))`, false, errors.New(`[types:1427]For float(M,D), double(M,D) or decimal(M,D), M must be >= D (column '').`)}, + {`select * from t where d = cast("'d'" as decimal(10,20))`, false, errors.New(`[types:1427]For float(M,D), double(M,D) or decimal(M,D), M must be >= D (column '').`)}, + // for convert decimal ErrTooBigPrecision + {`select * from t where d = convert(d, decimal(1000,20))`, false, errors.New(`[types:1426]Too big precision 1000 specified for column 'd'. Maximum is 65.`)}, + {`select * from t where d = convert(111, decimal(1000,20))`, false, errors.New(`[types:1426]Too big precision 1000 specified for column '111'. Maximum is 65.`)}, + {`select * from t where d = convert("abc", decimal(1000,20))`, false, errors.New(`[types:1426]Too big precision 1000 specified for column '"abc"'. Maximum is 65.`)}, + {`select * from t where d = convert('d', decimal(1000,20))`, false, errors.New(`[types:1426]Too big precision 1000 specified for column ''d''. Maximum is 65.`)}, + // for convert decimal ErrMBiggerThanD + {`select * from t where d = convert(d , decimal(10,20))`, false, errors.New(`[types:1427]For float(M,D), double(M,D) or decimal(M,D), M must be >= D (column '').`)}, + {`select * from t where d = convert("d", decimal(10,20))`, false, errors.New(`[types:1427]For float(M,D), double(M,D) or decimal(M,D), M must be >= D (column '').`)}, + {`select * from t where d = convert("'d'", decimal(10,20))`, false, errors.New(`[types:1427]For float(M,D), double(M,D) or decimal(M,D), M must be >= D (column '').`)}, } _, err := s.se.Execute(context.Background(), "use test") From 5abf42c2471916ee8f586a679ce223adc7ab5b54 Mon Sep 17 00:00:00 2001 From: pengdaqian Date: Tue, 17 Nov 2020 12:15:54 +0800 Subject: [PATCH 10/11] fix lint error --- planner/core/preprocess_test.go | 39 +++++++++++++++++---------------- 1 file changed, 20 insertions(+), 19 deletions(-) diff --git a/planner/core/preprocess_test.go b/planner/core/preprocess_test.go index 8aa924ec3ee77..b76af468efb8b 100644 --- a/planner/core/preprocess_test.go +++ b/planner/core/preprocess_test.go @@ -15,6 +15,7 @@ package core_test import ( "context" + . "github.com/pingcap/check" "github.com/pingcap/errors" "github.com/pingcap/parser" @@ -270,29 +271,29 @@ func (s *testValidatorSuite) TestValidator(c *C) { {"CREATE INDEX `` on t ((lower(a)));", true, errors.New("[ddl:1280]Incorrect index name ''")}, // for ErrTooBigScale - {`select convert('0.0', Decimal(41, 40))`, false, errors.New(`[types:1425]Too big scale 40 specified for column ''0.0''. Maximum is 30.`)}, + {`select convert('0.0', Decimal(41, 40))`, false, errors.New(`[types:1425]Too big scale 40 specified for column ''0.0''. Maximum is 30` + ".")}, // for cast decimal ErrTooBigPrecision - {`select * from t where d = cast(d as decimal(1000,20))`, false, errors.New(`[types:1426]Too big precision 1000 specified for column 'd'. Maximum is 65.`)}, - {`select * from t where d = cast(111 as decimal(1000,20))`, false, errors.New(`[types:1426]Too big precision 1000 specified for column '111'. Maximum is 65.`)}, - {`select * from t where d = cast("abc" as decimal(1000,20))`, false, errors.New(`[types:1426]Too big precision 1000 specified for column '"abc"'. Maximum is 65.`)}, - {`select * from t where d = cast('d' as decimal(1000,20))`, false, errors.New(`[types:1426]Too big precision 1000 specified for column ''d''. Maximum is 65.`)}, - {`select cast(d as decimal(1000,20))`, false, errors.New(`[types:1426]Too big precision 1000 specified for column 'd'. Maximum is 65.`)}, - {`select cast(111 as decimal(1000,20))`, false, errors.New(`[types:1426]Too big precision 1000 specified for column '111'. Maximum is 65.`)}, - {`select cast("abc" as decimal(1000,20))`, false, errors.New(`[types:1426]Too big precision 1000 specified for column '"abc"'. Maximum is 65.`)}, - {`select cast("'d'" as decimal(1000,20))`, false, errors.New(`[types:1426]Too big precision 1000 specified for column '"'d'"'. Maximum is 65.`)}, + {`select * from t where d = cast(d as decimal(1000,20))`, false, errors.New(`[types:1426]Too big precision 1000 specified for column 'd'. Maximum is 65` + ".")}, + {`select * from t where d = cast(111 as decimal(1000,20))`, false, errors.New(`[types:1426]Too big precision 1000 specified for column '111'. Maximum is 65` + ".")}, + {`select * from t where d = cast("abc" as decimal(1000,20))`, false, errors.New(`[types:1426]Too big precision 1000 specified for column '"abc"'. Maximum is 65` + ".")}, + {`select * from t where d = cast('d' as decimal(1000,20))`, false, errors.New(`[types:1426]Too big precision 1000 specified for column ''d''. Maximum is 65` + ".")}, + {`select cast(d as decimal(1000,20))`, false, errors.New(`[types:1426]Too big precision 1000 specified for column 'd'. Maximum is 65` + ".")}, + {`select cast(111 as decimal(1000,20))`, false, errors.New(`[types:1426]Too big precision 1000 specified for column '111'. Maximum is 65` + ".")}, + {`select cast("abc" as decimal(1000,20))`, false, errors.New(`[types:1426]Too big precision 1000 specified for column '"abc"'. Maximum is 65` + ".")}, + {`select cast("'d'" as decimal(1000,20))`, false, errors.New(`[types:1426]Too big precision 1000 specified for column '"'d'"'. Maximum is 65` + ".")}, // for cast decimal ErrMBiggerThanD - {`select * from t where d = cast(d as decimal(10,20))`, false, errors.New(`[types:1427]For float(M,D), double(M,D) or decimal(M,D), M must be >= D (column '').`)}, - {`select * from t where d = cast("d" as decimal(10,20))`, false, errors.New(`[types:1427]For float(M,D), double(M,D) or decimal(M,D), M must be >= D (column '').`)}, - {`select * from t where d = cast("'d'" as decimal(10,20))`, false, errors.New(`[types:1427]For float(M,D), double(M,D) or decimal(M,D), M must be >= D (column '').`)}, + {`select * from t where d = cast(d as decimal(10,20))`, false, errors.New(`[types:1427]For float(M,D), double(M,D) or decimal(M,D), M must be >= D (column '')` + ".")}, + {`select * from t where d = cast("d" as decimal(10,20))`, false, errors.New(`[types:1427]For float(M,D), double(M,D) or decimal(M,D), M must be >= D (column '')` + ".")}, + {`select * from t where d = cast("'d'" as decimal(10,20))`, false, errors.New(`[types:1427]For float(M,D), double(M,D) or decimal(M,D), M must be >= D (column '')` + ".")}, // for convert decimal ErrTooBigPrecision - {`select * from t where d = convert(d, decimal(1000,20))`, false, errors.New(`[types:1426]Too big precision 1000 specified for column 'd'. Maximum is 65.`)}, - {`select * from t where d = convert(111, decimal(1000,20))`, false, errors.New(`[types:1426]Too big precision 1000 specified for column '111'. Maximum is 65.`)}, - {`select * from t where d = convert("abc", decimal(1000,20))`, false, errors.New(`[types:1426]Too big precision 1000 specified for column '"abc"'. Maximum is 65.`)}, - {`select * from t where d = convert('d', decimal(1000,20))`, false, errors.New(`[types:1426]Too big precision 1000 specified for column ''d''. Maximum is 65.`)}, + {`select * from t where d = convert(d, decimal(1000,20))`, false, errors.New(`[types:1426]Too big precision 1000 specified for column 'd'. Maximum is 65` + ".")}, + {`select * from t where d = convert(111, decimal(1000,20))`, false, errors.New(`[types:1426]Too big precision 1000 specified for column '111'. Maximum is 65` + ".")}, + {`select * from t where d = convert("abc", decimal(1000,20))`, false, errors.New(`[types:1426]Too big precision 1000 specified for column '"abc"'. Maximum is 65` + ".")}, + {`select * from t where d = convert('d', decimal(1000,20))`, false, errors.New(`[types:1426]Too big precision 1000 specified for column ''d''. Maximum is 65` + ".")}, // for convert decimal ErrMBiggerThanD - {`select * from t where d = convert(d , decimal(10,20))`, false, errors.New(`[types:1427]For float(M,D), double(M,D) or decimal(M,D), M must be >= D (column '').`)}, - {`select * from t where d = convert("d", decimal(10,20))`, false, errors.New(`[types:1427]For float(M,D), double(M,D) or decimal(M,D), M must be >= D (column '').`)}, - {`select * from t where d = convert("'d'", decimal(10,20))`, false, errors.New(`[types:1427]For float(M,D), double(M,D) or decimal(M,D), M must be >= D (column '').`)}, + {`select * from t where d = convert(d , decimal(10,20))`, false, errors.New(`[types:1427]For float(M,D), double(M,D) or decimal(M,D), M must be >= D (column '')` + ".")}, + {`select * from t where d = convert("d", decimal(10,20))`, false, errors.New(`[types:1427]For float(M,D), double(M,D) or decimal(M,D), M must be >= D (column '')` + ".")}, + {`select * from t where d = convert("'d'", decimal(10,20))`, false, errors.New(`[types:1427]For float(M,D), double(M,D) or decimal(M,D), M must be >= D (column '')` + ".")}, } _, err := s.se.Execute(context.Background(), "use test") From 57e3be22be5ea0037a587cc45fe9ad685a403cdc Mon Sep 17 00:00:00 2001 From: pengdaqian Date: Tue, 17 Nov 2020 13:29:44 +0800 Subject: [PATCH 11/11] fix error --- planner/core/preprocess.go | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/planner/core/preprocess.go b/planner/core/preprocess.go index 145af99891e19..4c5374bfff7f1 100644 --- a/planner/core/preprocess.go +++ b/planner/core/preprocess.go @@ -836,11 +836,10 @@ func checkTp(tp *types.FieldType, colName, val string) error { // For FLOAT, the SQL standard permits an optional specification of the precision. // https://dev.mysql.com/doc/refman/8.0/en/floating-point-types.html if tp.Decimal == -1 { - if tp.Tp == mysql.TypeDouble { - if tp.Flen != -1 && colName != "" { - return types.ErrSyntax.GenWithStackByArgs() - } - } else { + switch tp.Tp { + case mysql.TypeDouble: + // For Double type Flen and Decimal check is moved to parser component + default: if tp.Flen > mysql.MaxDoublePrecisionLength { return types.ErrWrongFieldSpec.GenWithStackByArgs(colName) }