Skip to content

Commit

Permalink
Merge branch 'master' into flushPlanCache
Browse files Browse the repository at this point in the history
  • Loading branch information
Reminiscent committed Dec 17, 2021
2 parents 955d189 + 1cf2a6d commit 228f6e9
Show file tree
Hide file tree
Showing 11 changed files with 363 additions and 189 deletions.
37 changes: 37 additions & 0 deletions ddl/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -47,6 +47,7 @@ import (
"github.com/pingcap/tidb/table"
goutil "github.com/pingcap/tidb/util"
"github.com/pingcap/tidb/util/admin"
"github.com/pingcap/tidb/util/gcutil"
"github.com/pingcap/tidb/util/logutil"
"go.etcd.io/etcd/clientv3"
"go.uber.org/zap"
Expand Down Expand Up @@ -731,3 +732,39 @@ func init() {
RunInGoTest = true
}
}

// GetDropOrTruncateTableInfoFromJobsByStore implements GetDropOrTruncateTableInfoFromJobs
func GetDropOrTruncateTableInfoFromJobsByStore(jobs []*model.Job, gcSafePoint uint64, getTable func(uint64, int64, int64) (*model.TableInfo, error), fn func(*model.Job, *model.TableInfo) (bool, error)) (bool, error) {
for _, job := range jobs {
// Check GC safe point for getting snapshot infoSchema.
err := gcutil.ValidateSnapshotWithGCSafePoint(job.StartTS, gcSafePoint)
if err != nil {
return false, err
}
if job.Type != model.ActionDropTable && job.Type != model.ActionTruncateTable {
continue
}

tbl, err := getTable(job.StartTS, job.SchemaID, job.TableID)
if err != nil {
if meta.ErrDBNotExists.Equal(err) {
// The dropped/truncated DDL maybe execute failed that caused by the parallel DDL execution,
// then can't find the table from the snapshot info-schema. Should just ignore error here,
// see more in TestParallelDropSchemaAndDropTable.
continue
}
return false, err
}
if tbl == nil {
// The dropped/truncated DDL maybe execute failed that caused by the parallel DDL execution,
// then can't find the table from the snapshot info-schema. Should just ignore error here,
// see more in TestParallelDropSchemaAndDropTable.
continue
}
finish, err := fn(job, tbl)
if err != nil || finish {
return finish, err
}
}
return false, nil
}
39 changes: 6 additions & 33 deletions executor/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -710,42 +710,15 @@ func (e *DDLExec) getRecoverTableByJobID(s *ast.RecoverTableStmt, t *meta.Meta,
// GetDropOrTruncateTableInfoFromJobs gets the dropped/truncated table information from DDL jobs,
// it will use the `start_ts` of DDL job as snapshot to get the dropped/truncated table information.
func GetDropOrTruncateTableInfoFromJobs(jobs []*model.Job, gcSafePoint uint64, dom *domain.Domain, fn func(*model.Job, *model.TableInfo) (bool, error)) (bool, error) {
for _, job := range jobs {
// Check GC safe point for getting snapshot infoSchema.
err := gcutil.ValidateSnapshotWithGCSafePoint(job.StartTS, gcSafePoint)
getTable := func(StartTS uint64, SchemaID int64, TableID int64) (*model.TableInfo, error) {
snapMeta, err := dom.GetSnapshotMeta(StartTS)
if err != nil {
return false, err
}
if job.Type != model.ActionDropTable && job.Type != model.ActionTruncateTable {
continue
}

snapMeta, err := dom.GetSnapshotMeta(job.StartTS)
if err != nil {
return false, err
}
tbl, err := snapMeta.GetTable(job.SchemaID, job.TableID)
if err != nil {
if meta.ErrDBNotExists.Equal(err) {
// The dropped/truncated DDL maybe execute failed that caused by the parallel DDL execution,
// then can't find the table from the snapshot info-schema. Should just ignore error here,
// see more in TestParallelDropSchemaAndDropTable.
continue
}
return false, err
}
if tbl == nil {
// The dropped/truncated DDL maybe execute failed that caused by the parallel DDL execution,
// then can't find the table from the snapshot info-schema. Should just ignore error here,
// see more in TestParallelDropSchemaAndDropTable.
continue
}
finish, err := fn(job, tbl)
if err != nil || finish {
return finish, err
return nil, err
}
tbl, err := snapMeta.GetTable(SchemaID, TableID)
return tbl, err
}
return false, nil
return ddl.GetDropOrTruncateTableInfoFromJobsByStore(jobs, gcSafePoint, getTable, fn)
}

func (e *DDLExec) getRecoverTableByTableName(tableName *ast.TableName) (*model.Job, *model.TableInfo, error) {
Expand Down
38 changes: 38 additions & 0 deletions executor/executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -9522,6 +9522,44 @@ func (s *testSerialSuite) TestIssue30289(c *C) {
c.Assert(err.Error(), Matches, "issue30289 build return error")
}

func (s *testSerialSuite) TestIssue29498(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec("DROP TABLE IF EXISTS t1;")
tk.MustExec("CREATE TABLE t1 (t3 TIME(3), d DATE, t TIME);")
tk.MustExec("INSERT INTO t1 VALUES ('00:00:00.567', '2002-01-01', '00:00:02');")

res := tk.MustQuery("SELECT CONCAT(IFNULL(t3, d)) AS col1 FROM t1;")
row := res.Rows()[0][0].(string)
c.Assert(len(row), Equals, mysql.MaxDatetimeWidthNoFsp+3+1)
c.Assert(row[len(row)-12:], Equals, "00:00:00.567")

res = tk.MustQuery("SELECT IFNULL(t3, d) AS col1 FROM t1;")
row = res.Rows()[0][0].(string)
c.Assert(len(row), Equals, mysql.MaxDatetimeWidthNoFsp+3+1)
c.Assert(row[len(row)-12:], Equals, "00:00:00.567")

res = tk.MustQuery("SELECT CONCAT(IFNULL(t, d)) AS col1 FROM t1;")
row = res.Rows()[0][0].(string)
c.Assert(len(row), Equals, mysql.MaxDatetimeWidthNoFsp)
c.Assert(row[len(row)-8:], Equals, "00:00:02")

res = tk.MustQuery("SELECT IFNULL(t, d) AS col1 FROM t1;")
row = res.Rows()[0][0].(string)
c.Assert(len(row), Equals, mysql.MaxDatetimeWidthNoFsp)
c.Assert(row[len(row)-8:], Equals, "00:00:02")

res = tk.MustQuery("SELECT CONCAT(xx) FROM (SELECT t3 AS xx FROM t1 UNION SELECT d FROM t1) x ORDER BY -xx LIMIT 1;")
row = res.Rows()[0][0].(string)
c.Assert(len(row), Equals, mysql.MaxDatetimeWidthNoFsp+3+1)
c.Assert(row[len(row)-12:], Equals, "00:00:00.567")

res = tk.MustQuery("SELECT CONCAT(CASE WHEN d IS NOT NULL THEN t3 ELSE d END) AS col1 FROM t1;")
row = res.Rows()[0][0].(string)
c.Assert(len(row), Equals, mysql.MaxDatetimeWidthNoFsp+3+1)
c.Assert(row[len(row)-12:], Equals, "00:00:00.567")
}

// Test invoke Close without invoking Open before for each operators.
func (s *testSerialSuite) TestUnreasonablyClose(c *C) {
defer testleak.AfterTest(c)()
Expand Down
3 changes: 3 additions & 0 deletions expression/builtin_control.go
Original file line number Diff line number Diff line change
Expand Up @@ -155,6 +155,8 @@ func InferType4ControlFuncs(ctx sessionctx.Context, funcName string, lexp, rexp
if resultFieldType.Tp == mysql.TypeEnum || resultFieldType.Tp == mysql.TypeSet {
resultFieldType.Tp = mysql.TypeVarchar
}
} else if resultFieldType.Tp == mysql.TypeDatetime {
types.TryToFixFlenOfDatetime(resultFieldType)
}
return resultFieldType, nil
}
Expand Down Expand Up @@ -204,6 +206,7 @@ func (c *caseWhenFunctionClass) getFunction(ctx sessionctx.Context, args []Expre
decimal = 0
}
fieldTp.Decimal, fieldTp.Flen = decimal, flen
types.TryToFixFlenOfDatetime(fieldTp)
if fieldTp.EvalType().IsStringKind() && !isBinaryStr {
fieldTp.Charset, fieldTp.Collate = DeriveCollationFromExprs(ctx, args...)
if fieldTp.Charset == charset.CharsetBin && fieldTp.Collate == charset.CollationBin {
Expand Down
4 changes: 4 additions & 0 deletions expression/typeinfer_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -824,6 +824,8 @@ func (s *InferTypeSuite) createTestCase4ControlFuncs() []typeInferTestCase {
{"ifnull(null, null)", mysql.TypeNull, charset.CharsetBin, mysql.BinaryFlag, 0, 0},
{"ifnull(c_double_d, c_timestamp_d)", mysql.TypeVarchar, charset.CharsetUTF8MB4, 0, 22, types.UnspecifiedLength},
{"ifnull(c_json, c_decimal)", mysql.TypeLongBlob, charset.CharsetUTF8MB4, 0, math.MaxUint32, types.UnspecifiedLength},
{"ifnull(c_time, c_date)", mysql.TypeDatetime, charset.CharsetUTF8MB4, 0, mysql.MaxDatetimeWidthNoFsp + 3 + 1, 3},
{"ifnull(c_time_d, c_date)", mysql.TypeDatetime, charset.CharsetUTF8MB4, 0, mysql.MaxDatetimeWidthNoFsp, 0},
{"if(c_int_d, c_decimal, c_int_d)", mysql.TypeNewDecimal, charset.CharsetBin, mysql.BinaryFlag, 14, 3},
{"if(c_int_d, c_char, c_int_d)", mysql.TypeString, charset.CharsetUTF8MB4, mysql.BinaryFlag, 20, types.UnspecifiedLength},
{"if(c_int_d, c_binary, c_int_d)", mysql.TypeString, charset.CharsetBin, mysql.BinaryFlag, 20, types.UnspecifiedLength},
Expand All @@ -838,6 +840,8 @@ func (s *InferTypeSuite) createTestCase4ControlFuncs() []typeInferTestCase {
{"case when c_int_d > 1 then c_double_d else c_bchar end", mysql.TypeString, charset.CharsetUTF8MB4, mysql.BinaryFlag, 22, types.UnspecifiedLength},
{"case when c_int_d > 2 then c_double_d when c_int_d < 1 then c_decimal else c_double_d end", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, 22, 3},
{"case when c_double_d > 2 then c_decimal else 1 end", mysql.TypeNewDecimal, charset.CharsetBin, mysql.BinaryFlag, 6, 3},
{"case when c_time is not null then c_time else c_date end", mysql.TypeDatetime, charset.CharsetUTF8MB4, mysql.BinaryFlag, mysql.MaxDatetimeWidthNoFsp + 3 + 1, 3},
{"case when c_time_d is not null then c_time_d else c_date end", mysql.TypeDatetime, charset.CharsetUTF8MB4, mysql.BinaryFlag, mysql.MaxDatetimeWidthNoFsp, 0},
{"case when null then null else null end", mysql.TypeNull, charset.CharsetBin, mysql.BinaryFlag, 0, types.UnspecifiedLength},
}
}
Expand Down
1 change: 1 addition & 0 deletions planner/core/logical_plan_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -1368,6 +1368,7 @@ func unionJoinFieldType(a, b *types.FieldType) *types.FieldType {
resultTp.Decimal = mathutil.Max(a.Decimal, b.Decimal)
// `Flen - Decimal` is the fraction before '.'
resultTp.Flen = mathutil.Max(a.Flen-a.Decimal, b.Flen-b.Decimal) + resultTp.Decimal
types.TryToFixFlenOfDatetime(resultTp)
if resultTp.EvalType() != types.ETInt && (a.EvalType() == types.ETInt || b.EvalType() == types.ETInt) && resultTp.Flen < mysql.MaxIntWidth {
resultTp.Flen = mysql.MaxIntWidth
}
Expand Down
Loading

0 comments on commit 228f6e9

Please sign in to comment.