From f1e587704c0f8aa46e4c5bb33c390718e9e577f4 Mon Sep 17 00:00:00 2001 From: wjHuang Date: Tue, 21 Dec 2021 12:19:46 +0800 Subject: [PATCH 01/44] expression: fix wrong retType for reverse function (#30829) close pingcap/tidb#30809 --- .../r/explain_generate_column_substitute.result | 7 +++++++ .../t/explain_generate_column_substitute.test | 7 +++++++ ddl/ddl_api.go | 5 +++++ expression/builtin_string.go | 14 ++++++++------ 4 files changed, 27 insertions(+), 6 deletions(-) diff --git a/cmd/explaintest/r/explain_generate_column_substitute.result b/cmd/explaintest/r/explain_generate_column_substitute.result index 5162adb37f49c..3cab89b08dfc5 100644 --- a/cmd/explaintest/r/explain_generate_column_substitute.result +++ b/cmd/explaintest/r/explain_generate_column_substitute.result @@ -570,3 +570,10 @@ a select * from t004 ignore index (eidx) where timestampadd(microsecond, 1, a) = timestampadd(microsecond, 1, '2021-08-20'); a 2021-08-20 +drop table if exists t; +create table t ( c_int int, c_str varchar(40) character set utf8 collate utf8_general_ci, primary key(c_int, c_str(9)) clustered, key idx((reverse(c_str)))); +replace into t (c_int, c_str) values (9, "beautiful hermann"); +select reverse(c_str) from t use index(idx); +reverse(c_str) +nnamreh lufituaeb +drop table t; diff --git a/cmd/explaintest/t/explain_generate_column_substitute.test b/cmd/explaintest/t/explain_generate_column_substitute.test index 152d2c2a6dc7f..4e47ce34607ae 100644 --- a/cmd/explaintest/t/explain_generate_column_substitute.test +++ b/cmd/explaintest/t/explain_generate_column_substitute.test @@ -259,3 +259,10 @@ select * from t004 where timestampadd(microsecond, 1, a) = timestampadd(microsec alter table t004 add index eidx ((timestampadd(microsecond, 1, a))); select * from t004 use index(eidx) where timestampadd(microsecond, 1, a) = timestampadd(microsecond, 1, '2021-08-20'); select * from t004 ignore index (eidx) where timestampadd(microsecond, 1, a) = timestampadd(microsecond, 1, '2021-08-20'); + +drop table if exists t; +create table t ( c_int int, c_str varchar(40) character set utf8 collate utf8_general_ci, primary key(c_int, c_str(9)) clustered, key idx((reverse(c_str)))); +replace into t (c_int, c_str) values (9, "beautiful hermann"); +select reverse(c_str) from t use index(idx); + +drop table t; diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 957ba6e68386d..23cc306cd77b1 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -5360,6 +5360,11 @@ func buildHiddenColumnInfo(ctx sessionctx.Context, indexPartSpecifications []*as Hidden: true, FieldType: *expr.GetType(), } + // Reset some flag, it may be caused by wrong type infer. But it's not easy to fix them all, so reset them here for safety. + colInfo.Flag &= ^mysql.PriKeyFlag + colInfo.Flag &= ^mysql.UniqueKeyFlag + colInfo.Flag &= ^mysql.AutoIncrementFlag + if colInfo.Tp == mysql.TypeDatetime || colInfo.Tp == mysql.TypeDate || colInfo.Tp == mysql.TypeTimestamp || colInfo.Tp == mysql.TypeDuration { if colInfo.FieldType.Decimal == types.UnspecifiedLength { colInfo.FieldType.Decimal = int(types.MaxFsp) diff --git a/expression/builtin_string.go b/expression/builtin_string.go index 925aecb3dc33a..c494d9fcb5c10 100644 --- a/expression/builtin_string.go +++ b/expression/builtin_string.go @@ -173,10 +173,12 @@ func reverseRunes(origin []rune) []rune { // SetBinFlagOrBinStr sets resTp to binary string if argTp is a binary string, // if not, sets the binary flag of resTp to true if argTp has binary flag. +// We need to check if the tp is enum or set, if so, don't add binary flag directly unless it has binary flag. func SetBinFlagOrBinStr(argTp *types.FieldType, resTp *types.FieldType) { + nonEnumOrSet := !(argTp.Tp == mysql.TypeEnum || argTp.Tp == mysql.TypeSet) if types.IsBinaryStr(argTp) { types.SetBinChsClnFlag(resTp) - } else if mysql.HasBinaryFlag(argTp.Flag) || !types.IsNonBinaryStr(argTp) { + } else if mysql.HasBinaryFlag(argTp.Flag) || (!types.IsNonBinaryStr(argTp) && nonEnumOrSet) { resTp.Flag |= mysql.BinaryFlag } } @@ -765,12 +767,12 @@ func (c *reverseFunctionClass) getFunction(ctx sessionctx.Context, args []Expres if err != nil { return nil, err } - retTp := *args[0].GetType() - retTp.Tp = mysql.TypeVarString - retTp.Decimal = types.UnspecifiedLength - bf.tp = &retTp + + argTp := args[0].GetType() + bf.tp.Flen = args[0].GetType().Flen + addBinFlag(bf.tp) var sig builtinFunc - if types.IsBinaryStr(bf.tp) { + if types.IsBinaryStr(argTp) { sig = &builtinReverseSig{bf} sig.setPbCode(tipb.ScalarFuncSig_Reverse) } else { From 06c1e8c93420ba18b6b1d3c2459c4bb77fc1632e Mon Sep 17 00:00:00 2001 From: Song Gao Date: Tue, 21 Dec 2021 13:21:46 +0800 Subject: [PATCH 02/44] planner: support trace topn push down (#30800) ref pingcap/tidb#29661 --- planner/core/logical_plan_trace_test.go | 46 ++++++++ planner/core/plan.go | 2 +- planner/core/rule_topn_push_down.go | 133 ++++++++++++++++++------ 3 files changed, 147 insertions(+), 34 deletions(-) diff --git a/planner/core/logical_plan_trace_test.go b/planner/core/logical_plan_trace_test.go index 58348bd7712de..0d4577bc1f107 100644 --- a/planner/core/logical_plan_trace_test.go +++ b/planner/core/logical_plan_trace_test.go @@ -86,6 +86,52 @@ func (s *testPlanSuite) TestSingleRuleTraceStep(c *C) { assertRuleName string assertRuleSteps []assertTraceStep }{ + { + sql: "select * from t as t1 left join t as t2 on t1.a = t2.a order by t1.a limit 10;", + flags: []uint64{flagPrunColumns, flagBuildKeyInfo, flagPushDownTopN}, + assertRuleName: "topn_push_down", + assertRuleSteps: []assertTraceStep{ + { + assertAction: "Limit_6 is converted into TopN_7", + assertReason: "", + }, + { + assertAction: "Sort_5 passes ByItems[test.t.a] to TopN_7", + assertReason: "TopN_7 is Limit originally", + }, + { + assertAction: "TopN_8 is added and pushed into Join_3's left table", + assertReason: "Join_3's joinType is left outer join, and all ByItems[test.t.a] contained in left table", + }, + { + assertAction: "TopN_8 is added as DataSource_1's parent", + assertReason: "TopN is pushed down", + }, + { + assertAction: "TopN_7 is added as Join_3's parent", + assertReason: "TopN is pushed down", + }, + }, + }, + { + sql: "select * from t order by a limit 10", + flags: []uint64{flagPrunColumns, flagBuildKeyInfo, flagPushDownTopN}, + assertRuleName: "topn_push_down", + assertRuleSteps: []assertTraceStep{ + { + assertAction: "Limit_4 is converted into TopN_5", + assertReason: "", + }, + { + assertAction: "Sort_3 passes ByItems[test.t.a] to TopN_5", + assertReason: "TopN_5 is Limit originally", + }, + { + assertAction: "TopN_5 is added as DataSource_1's parent", + assertReason: "TopN is pushed down", + }, + }, + }, { sql: "select * from pt3 where ptn > 3;", flags: []uint64{flagPartitionProcessor, flagPredicatePushDown, flagBuildKeyInfo, flagPrunColumns}, diff --git a/planner/core/plan.go b/planner/core/plan.go index 3515f44e91750..bd9d19a7de4e8 100644 --- a/planner/core/plan.go +++ b/planner/core/plan.go @@ -259,7 +259,7 @@ type LogicalPlan interface { BuildKeyInfo(selfSchema *expression.Schema, childSchema []*expression.Schema) // pushDownTopN will push down the topN or limit operator during logical optimization. - pushDownTopN(topN *LogicalTopN) LogicalPlan + pushDownTopN(topN *LogicalTopN, opt *logicalOptimizeOp) LogicalPlan // recursiveDeriveStats derives statistic info between plans. recursiveDeriveStats(colGroups [][]*expression.Column) (*property.StatsInfo, error) diff --git a/planner/core/rule_topn_push_down.go b/planner/core/rule_topn_push_down.go index e6234bbc3f3dc..73ed5749bc57c 100644 --- a/planner/core/rule_topn_push_down.go +++ b/planner/core/rule_topn_push_down.go @@ -15,7 +15,9 @@ package core import ( + "bytes" "context" + "fmt" "github.com/cznic/mathutil" "github.com/pingcap/tidb/expression" @@ -27,22 +29,22 @@ type pushDownTopNOptimizer struct { } func (s *pushDownTopNOptimizer) optimize(ctx context.Context, p LogicalPlan, opt *logicalOptimizeOp) (LogicalPlan, error) { - return p.pushDownTopN(nil), nil + return p.pushDownTopN(nil, opt), nil } -func (s *baseLogicalPlan) pushDownTopN(topN *LogicalTopN) LogicalPlan { +func (s *baseLogicalPlan) pushDownTopN(topN *LogicalTopN, opt *logicalOptimizeOp) LogicalPlan { p := s.self for i, child := range p.Children() { - p.Children()[i] = child.pushDownTopN(nil) + p.Children()[i] = child.pushDownTopN(nil, opt) } if topN != nil { - return topN.setChild(p) + return topN.setChild(p, opt) } return p } // setChild set p as topn's child. -func (lt *LogicalTopN) setChild(p LogicalPlan) LogicalPlan { +func (lt *LogicalTopN) setChild(p LogicalPlan, opt *logicalOptimizeOp) LogicalPlan { // Remove this TopN if its child is a TableDual. dual, isDual := p.(*LogicalTableDual) if isDual { @@ -62,37 +64,43 @@ func (lt *LogicalTopN) setChild(p LogicalPlan) LogicalPlan { limitHints: lt.limitHints, }.Init(lt.ctx, lt.blockOffset) limit.SetChildren(p) + appendTopNPushDownTraceStep(limit, p, opt) return limit } // Then lt must be topN. lt.SetChildren(p) + appendTopNPushDownTraceStep(lt, p, opt) return lt } -func (ls *LogicalSort) pushDownTopN(topN *LogicalTopN) LogicalPlan { +func (ls *LogicalSort) pushDownTopN(topN *LogicalTopN, opt *logicalOptimizeOp) LogicalPlan { if topN == nil { - return ls.baseLogicalPlan.pushDownTopN(nil) + return ls.baseLogicalPlan.pushDownTopN(nil, opt) } else if topN.isLimit() { topN.ByItems = ls.ByItems - return ls.children[0].pushDownTopN(topN) + appendSortPassByItemsTraceStep(ls, topN, opt) + return ls.children[0].pushDownTopN(topN, opt) } // If a TopN is pushed down, this sort is useless. - return ls.children[0].pushDownTopN(topN) + return ls.children[0].pushDownTopN(topN, opt) } -func (p *LogicalLimit) convertToTopN() *LogicalTopN { - return LogicalTopN{Offset: p.Offset, Count: p.Count, limitHints: p.limitHints}.Init(p.ctx, p.blockOffset) +func (p *LogicalLimit) convertToTopN(opt *logicalOptimizeOp) *LogicalTopN { + topn := LogicalTopN{Offset: p.Offset, Count: p.Count, limitHints: p.limitHints}.Init(p.ctx, p.blockOffset) + opt.appendStepToCurrent(topn.ID(), topn.TP(), "", fmt.Sprintf("%v_%v is converted into %v_%v", + p.TP(), p.ID(), topn.TP(), topn.ID())) + return topn } -func (p *LogicalLimit) pushDownTopN(topN *LogicalTopN) LogicalPlan { - child := p.children[0].pushDownTopN(p.convertToTopN()) +func (p *LogicalLimit) pushDownTopN(topN *LogicalTopN, opt *logicalOptimizeOp) LogicalPlan { + child := p.children[0].pushDownTopN(p.convertToTopN(opt), opt) if topN != nil { - return topN.setChild(child) + return topN.setChild(child, opt) } return child } -func (p *LogicalUnionAll) pushDownTopN(topN *LogicalTopN) LogicalPlan { +func (p *LogicalUnionAll) pushDownTopN(topN *LogicalTopN, opt *logicalOptimizeOp) LogicalPlan { for i, child := range p.children { var newTopN *LogicalTopN if topN != nil { @@ -100,19 +108,23 @@ func (p *LogicalUnionAll) pushDownTopN(topN *LogicalTopN) LogicalPlan { for _, by := range topN.ByItems { newTopN.ByItems = append(newTopN.ByItems, &util.ByItems{Expr: by.Expr, Desc: by.Desc}) } + // newTopN to push down Union's child + opt.appendStepToCurrent(newTopN.ID(), newTopN.TP(), "", + fmt.Sprintf("%v_%v is added and pushed down across %v_%v", + newTopN.TP(), newTopN.ID(), p.TP(), p.ID())) } - p.children[i] = child.pushDownTopN(newTopN) + p.children[i] = child.pushDownTopN(newTopN, opt) } if topN != nil { - return topN.setChild(p) + return topN.setChild(p, opt) } return p } -func (p *LogicalProjection) pushDownTopN(topN *LogicalTopN) LogicalPlan { +func (p *LogicalProjection) pushDownTopN(topN *LogicalTopN, opt *logicalOptimizeOp) LogicalPlan { for _, expr := range p.Exprs { if expression.HasAssignSetVarFunc(expr) { - return p.baseLogicalPlan.pushDownTopN(topN) + return p.baseLogicalPlan.pushDownTopN(topN, opt) } } if topN != nil { @@ -128,28 +140,28 @@ func (p *LogicalProjection) pushDownTopN(topN *LogicalTopN) LogicalPlan { } } } - p.children[0] = p.children[0].pushDownTopN(topN) + p.children[0] = p.children[0].pushDownTopN(topN, opt) return p } -func (p *LogicalLock) pushDownTopN(topN *LogicalTopN) LogicalPlan { +func (p *LogicalLock) pushDownTopN(topN *LogicalTopN, opt *logicalOptimizeOp) LogicalPlan { if topN != nil { - p.children[0] = p.children[0].pushDownTopN(topN) + p.children[0] = p.children[0].pushDownTopN(topN, opt) } return p.self } // pushDownTopNToChild will push a topN to one child of join. The idx stands for join child index. 0 is for left child. -func (p *LogicalJoin) pushDownTopNToChild(topN *LogicalTopN, idx int) LogicalPlan { +func (p *LogicalJoin) pushDownTopNToChild(topN *LogicalTopN, idx int, opt *logicalOptimizeOp) LogicalPlan { if topN == nil { - return p.children[idx].pushDownTopN(nil) + return p.children[idx].pushDownTopN(nil, opt) } for _, by := range topN.ByItems { cols := expression.ExtractColumns(by.Expr) for _, col := range cols { if !p.children[idx].Schema().Contains(col) { - return p.children[idx].pushDownTopN(nil) + return p.children[idx].pushDownTopN(nil, opt) } } } @@ -162,24 +174,25 @@ func (p *LogicalJoin) pushDownTopNToChild(topN *LogicalTopN, idx int) LogicalPla for i := range topN.ByItems { newTopN.ByItems[i] = topN.ByItems[i].Clone() } - return p.children[idx].pushDownTopN(newTopN) + appendTopNPushDownJoinTraceStep(p, newTopN, idx, opt) + return p.children[idx].pushDownTopN(newTopN, opt) } -func (p *LogicalJoin) pushDownTopN(topN *LogicalTopN) LogicalPlan { +func (p *LogicalJoin) pushDownTopN(topN *LogicalTopN, opt *logicalOptimizeOp) LogicalPlan { switch p.JoinType { case LeftOuterJoin, LeftOuterSemiJoin, AntiLeftOuterSemiJoin: - p.children[0] = p.pushDownTopNToChild(topN, 0) - p.children[1] = p.children[1].pushDownTopN(nil) + p.children[0] = p.pushDownTopNToChild(topN, 0, opt) + p.children[1] = p.children[1].pushDownTopN(nil, opt) case RightOuterJoin: - p.children[1] = p.pushDownTopNToChild(topN, 1) - p.children[0] = p.children[0].pushDownTopN(nil) + p.children[1] = p.pushDownTopNToChild(topN, 1, opt) + p.children[0] = p.children[0].pushDownTopN(nil, opt) default: - return p.baseLogicalPlan.pushDownTopN(topN) + return p.baseLogicalPlan.pushDownTopN(topN, opt) } // The LogicalJoin may be also a LogicalApply. So we must use self to set parents. if topN != nil { - return topN.setChild(p.self) + return topN.setChild(p.self, opt) } return p.self } @@ -187,3 +200,57 @@ func (p *LogicalJoin) pushDownTopN(topN *LogicalTopN) LogicalPlan { func (*pushDownTopNOptimizer) name() string { return "topn_push_down" } + +func appendTopNPushDownTraceStep(parent LogicalPlan, child LogicalPlan, opt *logicalOptimizeOp) { + action := fmt.Sprintf("%v_%v is added as %v_%v's parent", parent.TP(), parent.ID(), child.TP(), child.ID()) + reason := fmt.Sprintf("%v is pushed down", parent.TP()) + opt.appendStepToCurrent(parent.ID(), parent.TP(), reason, action) +} + +func appendTopNPushDownJoinTraceStep(p *LogicalJoin, topN *LogicalTopN, idx int, opt *logicalOptimizeOp) { + action := func() string { + buffer := bytes.NewBufferString(fmt.Sprintf("%v_%v is added and pushed into %v_%v's ", + topN.TP(), topN.ID(), p.TP(), p.ID())) + if idx == 0 { + buffer.WriteString("left ") + } else { + buffer.WriteString("right ") + } + buffer.WriteString("table") + return buffer.String() + }() + reason := func() string { + buffer := bytes.NewBufferString(fmt.Sprintf("%v_%v's joinType is %v, and all ByItems[", p.TP(), p.ID(), p.JoinType.String())) + for i, item := range topN.ByItems { + if i > 0 { + buffer.WriteString(",") + } + buffer.WriteString(item.String()) + } + buffer.WriteString("] contained in ") + if idx == 0 { + buffer.WriteString("left ") + } else { + buffer.WriteString("right ") + } + buffer.WriteString("table") + return buffer.String() + }() + opt.appendStepToCurrent(p.ID(), p.TP(), reason, action) +} + +func appendSortPassByItemsTraceStep(sort *LogicalSort, topN *LogicalTopN, opt *logicalOptimizeOp) { + action := func() string { + buffer := bytes.NewBufferString(fmt.Sprintf("%v_%v passes ByItems[", sort.TP(), sort.ID())) + for i, item := range sort.ByItems { + if i > 0 { + buffer.WriteString(",") + } + buffer.WriteString(item.String()) + } + buffer.WriteString(fmt.Sprintf("] to %v_%v", topN.TP(), topN.ID())) + return buffer.String() + }() + reason := fmt.Sprintf("%v_%v is Limit originally", topN.TP(), topN.ID()) + opt.appendStepToCurrent(sort.ID(), sort.TP(), reason, action) +} From 77b4e4028cff7c25c1db0582acf0e5685251ff67 Mon Sep 17 00:00:00 2001 From: Xiang Zhang Date: Tue, 21 Dec 2021 15:23:47 +0800 Subject: [PATCH 03/44] github: add issue requirement to pull request template (#30817) close pingcap/tidb#30814 --- .github/pull_request_template.md | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/.github/pull_request_template.md b/.github/pull_request_template.md index 3eef4ea0a6587..b8f7a3b6eeca5 100644 --- a/.github/pull_request_template.md +++ b/.github/pull_request_template.md @@ -9,6 +9,16 @@ PR Title Format: --> ### What problem does this PR solve? + Issue Number: close #xxx From fe1aaf2fd730870d962e8b188e57c19ad6a305e3 Mon Sep 17 00:00:00 2001 From: Yexiang Zhang Date: Tue, 21 Dec 2021 15:43:47 +0800 Subject: [PATCH 04/44] topsql: introduce stmtstats and sql execution count (#30277) --- distsql/distsql.go | 24 +- distsql/distsql_test.go | 2 +- executor/adapter.go | 31 +++ executor/analyze.go | 6 +- executor/batch_point_get.go | 1 + executor/checksum.go | 2 +- executor/executor.go | 8 + executor/insert.go | 1 + executor/point_get.go | 1 + executor/replace.go | 1 + executor/update.go | 4 + go.mod | 2 +- go.sum | 4 +- kv/option.go | 4 +- session/session.go | 7 + sessionctx/stmtctx/stmtctx.go | 7 + sessionctx/variable/session.go | 12 +- store/driver/txn/snapshot.go | 3 + store/driver/txn/txn_driver.go | 3 + util/topsql/stmtstats/aggregator.go | 156 +++++++++++++ util/topsql/stmtstats/aggregator_test.go | 93 ++++++++ util/topsql/stmtstats/kv_exec_count.go | 73 ++++++ util/topsql/stmtstats/kv_exec_count_test.go | 43 ++++ util/topsql/stmtstats/main_test.go | 27 +++ util/topsql/stmtstats/stmtstats.go | 219 ++++++++++++++++++ util/topsql/stmtstats/stmtstats_test.go | 187 +++++++++++++++ .../stmtstats/stmtstatstest/main_test.go | 31 +++ .../stmtstats/stmtstatstest/stmtstats_test.go | 150 ++++++++++++ util/topsql/topsql.go | 3 + 29 files changed, 1092 insertions(+), 13 deletions(-) create mode 100644 util/topsql/stmtstats/aggregator.go create mode 100644 util/topsql/stmtstats/aggregator_test.go create mode 100644 util/topsql/stmtstats/kv_exec_count.go create mode 100644 util/topsql/stmtstats/kv_exec_count_test.go create mode 100644 util/topsql/stmtstats/main_test.go create mode 100644 util/topsql/stmtstats/stmtstats.go create mode 100644 util/topsql/stmtstats/stmtstats_test.go create mode 100644 util/topsql/stmtstats/stmtstatstest/main_test.go create mode 100644 util/topsql/stmtstats/stmtstatstest/stmtstats_test.go diff --git a/distsql/distsql.go b/distsql/distsql.go index 2f952da2a7d3c..77b75efc480fd 100644 --- a/distsql/distsql.go +++ b/distsql/distsql.go @@ -23,17 +23,20 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/sessionctx/stmtctx" + "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/logutil" - "github.com/pingcap/tidb/util/memory" "github.com/pingcap/tidb/util/trxevents" "github.com/pingcap/tipb/go-tipb" + "github.com/tikv/client-go/v2/tikvrpc/interceptor" "go.uber.org/zap" ) // DispatchMPPTasks dispatches all tasks and returns an iterator. func DispatchMPPTasks(ctx context.Context, sctx sessionctx.Context, tasks []*kv.MPPDispatchRequest, fieldTypes []*types.FieldType, planIDs []int, rootID int) (SelectResult, error) { + ctx = WithSQLKvExecCounterInterceptor(ctx, sctx.GetSessionVars().StmtCtx) _, allowTiFlashFallback := sctx.GetSessionVars().AllowFallbackToTiKV[kv.TiFlash] resp := sctx.GetMPPClient().DispatchMPPTasks(ctx, sctx.GetSessionVars().KVVars, tasks, allowTiFlashFallback) if resp == nil { @@ -88,6 +91,8 @@ func Select(ctx context.Context, sctx sessionctx.Context, kvReq *kv.Request, fie zap.String("stmt", originalSQL)) } } + + ctx = WithSQLKvExecCounterInterceptor(ctx, sctx.GetSessionVars().StmtCtx) resp := sctx.GetClient().Send(ctx, kvReq, sctx.GetSessionVars().KVVars, sctx.GetSessionVars().StmtCtx.MemTracker, enabledRateLimitAction, eventCb) if resp == nil { return nil, errors.New("client returns nil response") @@ -149,8 +154,9 @@ func SelectWithRuntimeStats(ctx context.Context, sctx sessionctx.Context, kvReq // Analyze do a analyze request. func Analyze(ctx context.Context, client kv.Client, kvReq *kv.Request, vars interface{}, - isRestrict bool, sessionMemTracker *memory.Tracker) (SelectResult, error) { - resp := client.Send(ctx, kvReq, vars, sessionMemTracker, false, nil) + isRestrict bool, stmtCtx *stmtctx.StatementContext) (SelectResult, error) { + ctx = WithSQLKvExecCounterInterceptor(ctx, stmtCtx) + resp := client.Send(ctx, kvReq, vars, stmtCtx.MemTracker, false, nil) if resp == nil { return nil, errors.New("client returns nil response") } @@ -244,3 +250,15 @@ func init() { systemEndian = tipb.Endian_LittleEndian } } + +// WithSQLKvExecCounterInterceptor binds an interceptor for client-go to count the +// number of SQL executions of each TiKV (if any). +func WithSQLKvExecCounterInterceptor(ctx context.Context, stmtCtx *stmtctx.StatementContext) context.Context { + if variable.TopSQLEnabled() && stmtCtx.KvExecCounter != nil { + // Unlike calling Transaction or Snapshot interface, in distsql package we directly + // face tikv Request. So we need to manually bind RPCInterceptor to ctx. Instead of + // calling SetRPCInterceptor on Transaction or Snapshot. + return interceptor.WithRPCInterceptor(ctx, stmtCtx.KvExecCounter.RPCInterceptor()) + } + return ctx +} diff --git a/distsql/distsql_test.go b/distsql/distsql_test.go index 6ff4a5cb284e5..9be2738da2251 100644 --- a/distsql/distsql_test.go +++ b/distsql/distsql_test.go @@ -191,7 +191,7 @@ func TestAnalyze(t *testing.T) { Build() require.NoError(t, err) - response, err := Analyze(context.TODO(), sctx.GetClient(), request, tikvstore.DefaultVars, true, sctx.GetSessionVars().StmtCtx.MemTracker) + response, err := Analyze(context.TODO(), sctx.GetClient(), request, tikvstore.DefaultVars, true, sctx.GetSessionVars().StmtCtx) require.NoError(t, err) result, ok := response.(*selectResult) diff --git a/executor/adapter.go b/executor/adapter.go index 87f87a9712516..39e660099ed3f 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -233,6 +233,7 @@ func (a *ExecStmt) PointGet(ctx context.Context, is infoschema.InfoSchema) (*rec ctx = opentracing.ContextWithSpan(ctx, span1) } ctx = a.setPlanLabelForTopSQL(ctx) + a.observeStmtBeginForTopSQL() startTs := uint64(math.MaxUint64) err := a.Ctx.InitTxnWithStartTS(startTs) if err != nil { @@ -383,6 +384,7 @@ func (a *ExecStmt) Exec(ctx context.Context) (_ sqlexec.RecordSet, err error) { } // ExecuteExec will rewrite `a.Plan`, so set plan label should be executed after `a.buildExecutor`. ctx = a.setPlanLabelForTopSQL(ctx) + a.observeStmtBeginForTopSQL() if err = e.Open(ctx); err != nil { terror.Call(e.Close) @@ -896,6 +898,7 @@ func (a *ExecStmt) FinishExecuteStmt(txnTS uint64, err error, hasMoreResults boo // `LowSlowQuery` and `SummaryStmt` must be called before recording `PrevStmt`. a.LogSlowQuery(txnTS, succ, hasMoreResults) a.SummaryStmt(succ) + a.observeStmtFinishedForTopSQL() if sessVars.StmtCtx.IsTiFlash.Load() { if succ { totalTiFlashQuerySuccCounter.Inc() @@ -1247,3 +1250,31 @@ func (a *ExecStmt) GetTextToLog() string { } return sql } + +func (a *ExecStmt) observeStmtBeginForTopSQL() { + if vars := a.Ctx.GetSessionVars(); variable.TopSQLEnabled() && vars.StmtStats != nil { + sqlDigest, planDigest := a.getSQLPlanDigest() + vars.StmtStats.OnExecutionBegin(sqlDigest, planDigest) + // This is a special logic prepared for TiKV's SQLExecCount. + vars.StmtCtx.KvExecCounter = vars.StmtStats.CreateKvExecCounter(sqlDigest, planDigest) + } +} + +func (a *ExecStmt) observeStmtFinishedForTopSQL() { + if vars := a.Ctx.GetSessionVars(); variable.TopSQLEnabled() && vars.StmtStats != nil { + sqlDigest, planDigest := a.getSQLPlanDigest() + vars.StmtStats.OnExecutionFinished(sqlDigest, planDigest) + } +} + +func (a *ExecStmt) getSQLPlanDigest() ([]byte, []byte) { + var sqlDigest, planDigest []byte + vars := a.Ctx.GetSessionVars() + if _, d := vars.StmtCtx.SQLDigest(); d != nil { + sqlDigest = d.Bytes() + } + if _, d := vars.StmtCtx.GetPlanDigest(); d != nil { + planDigest = d.Bytes() + } + return sqlDigest, planDigest +} diff --git a/executor/analyze.go b/executor/analyze.go index 5397c1ee0608c..c8f91ccc13f63 100644 --- a/executor/analyze.go +++ b/executor/analyze.go @@ -400,7 +400,7 @@ func (e *AnalyzeIndexExec) fetchAnalyzeResult(ranges []*ranger.Range, isNullRang return err } ctx := context.TODO() - result, err := distsql.Analyze(ctx, e.ctx.GetClient(), kvReq, e.ctx.GetSessionVars().KVVars, e.ctx.GetSessionVars().InRestrictedSQL, e.ctx.GetSessionVars().StmtCtx.MemTracker) + result, err := distsql.Analyze(ctx, e.ctx.GetClient(), kvReq, e.ctx.GetSessionVars().KVVars, e.ctx.GetSessionVars().InRestrictedSQL, e.ctx.GetSessionVars().StmtCtx) if err != nil { return err } @@ -763,7 +763,7 @@ func (e *AnalyzeColumnsExec) buildResp(ranges []*ranger.Range) (distsql.SelectRe return nil, err } ctx := context.TODO() - result, err := distsql.Analyze(ctx, e.ctx.GetClient(), kvReq, e.ctx.GetSessionVars().KVVars, e.ctx.GetSessionVars().InRestrictedSQL, e.ctx.GetSessionVars().StmtCtx.MemTracker) + result, err := distsql.Analyze(ctx, e.ctx.GetClient(), kvReq, e.ctx.GetSessionVars().KVVars, e.ctx.GetSessionVars().InRestrictedSQL, e.ctx.GetSessionVars().StmtCtx) if err != nil { return nil, err } @@ -1854,6 +1854,7 @@ func (e *AnalyzeFastExec) handleScanTasks(bo *tikv.Backoffer) (keysSize int, err snapshot.SetOption(kv.ReplicaRead, kv.ReplicaReadFollower) } setResourceGroupTaggerForTxn(e.ctx.GetSessionVars().StmtCtx, snapshot) + setRPCInterceptorOfExecCounterForTxn(e.ctx.GetSessionVars(), snapshot) for _, t := range e.scanTasks { iter, err := snapshot.Iter(kv.Key(t.StartKey), kv.Key(t.EndKey)) if err != nil { @@ -1875,6 +1876,7 @@ func (e *AnalyzeFastExec) handleSampTasks(workID int, step uint32, err *error) { snapshot.SetOption(kv.IsolationLevel, kv.SI) snapshot.SetOption(kv.Priority, kv.PriorityLow) setResourceGroupTaggerForTxn(e.ctx.GetSessionVars().StmtCtx, snapshot) + setRPCInterceptorOfExecCounterForTxn(e.ctx.GetSessionVars(), snapshot) readReplicaType := e.ctx.GetSessionVars().GetReplicaRead() if readReplicaType.IsFollowerRead() { snapshot.SetOption(kv.ReplicaRead, readReplicaType) diff --git a/executor/batch_point_get.go b/executor/batch_point_get.go index c30bf507d6d9d..f642cf92d56ff 100644 --- a/executor/batch_point_get.go +++ b/executor/batch_point_get.go @@ -150,6 +150,7 @@ func (e *BatchPointGetExec) Open(context.Context) error { }) } setResourceGroupTaggerForTxn(stmtCtx, snapshot) + setRPCInterceptorOfExecCounterForTxn(sessVars, snapshot) var batchGetter kv.BatchGetter = snapshot if txn.Valid() { lock := e.tblInfo.Lock diff --git a/executor/checksum.go b/executor/checksum.go index 69fd6ed319e75..013fd3be2226f 100644 --- a/executor/checksum.go +++ b/executor/checksum.go @@ -128,7 +128,7 @@ func (e *ChecksumTableExec) checksumWorker(taskCh <-chan *checksumTask, resultCh } func (e *ChecksumTableExec) handleChecksumRequest(req *kv.Request) (resp *tipb.ChecksumResponse, err error) { - ctx := context.TODO() + ctx := distsql.WithSQLKvExecCounterInterceptor(context.TODO(), e.ctx.GetSessionVars().StmtCtx) res, err := distsql.Checksum(ctx, e.ctx.GetClient(), req, e.ctx.GetSessionVars().KVVars) if err != nil { return nil, err diff --git a/executor/executor.go b/executor/executor.go index eee07f8774ed0..4338529cea8ac 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -1914,3 +1914,11 @@ func setResourceGroupTaggerForTxn(sc *stmtctx.StatementContext, snapshot kv.Snap snapshot.SetOption(kv.ResourceGroupTagger, sc.GetResourceGroupTagger()) } } + +// setRPCInterceptorOfExecCounterForTxn binds an interceptor for client-go to count +// the number of SQL executions of each TiKV. +func setRPCInterceptorOfExecCounterForTxn(vars *variable.SessionVars, snapshot kv.Snapshot) { + if snapshot != nil && variable.TopSQLEnabled() && vars.StmtCtx.KvExecCounter != nil { + snapshot.SetOption(kv.RPCInterceptor, vars.StmtCtx.KvExecCounter.RPCInterceptor()) + } +} diff --git a/executor/insert.go b/executor/insert.go index 2862416ddf04a..7b0758c7ff076 100644 --- a/executor/insert.go +++ b/executor/insert.go @@ -67,6 +67,7 @@ func (e *InsertExec) exec(ctx context.Context, rows [][]types.Datum) error { return err } setResourceGroupTaggerForTxn(sessVars.StmtCtx, txn) + setRPCInterceptorOfExecCounterForTxn(sessVars, txn) txnSize := txn.Size() sessVars.StmtCtx.AddRecordRows(uint64(len(rows))) // If you use the IGNORE keyword, duplicate-key error that occurs while executing the INSERT statement are ignored. diff --git a/executor/point_get.go b/executor/point_get.go index 45f3fa76e263f..698626b4e1403 100644 --- a/executor/point_get.go +++ b/executor/point_get.go @@ -191,6 +191,7 @@ func (e *PointGetExecutor) Open(context.Context) error { } }) setResourceGroupTaggerForTxn(e.ctx.GetSessionVars().StmtCtx, e.snapshot) + setRPCInterceptorOfExecCounterForTxn(e.ctx.GetSessionVars(), e.snapshot) return nil } diff --git a/executor/replace.go b/executor/replace.go index 78e0085aa520e..fe1930639f446 100644 --- a/executor/replace.go +++ b/executor/replace.go @@ -223,6 +223,7 @@ func (e *ReplaceExec) exec(ctx context.Context, newRows [][]types.Datum) error { } } setResourceGroupTaggerForTxn(e.ctx.GetSessionVars().StmtCtx, txn) + setRPCInterceptorOfExecCounterForTxn(e.ctx.GetSessionVars(), txn) prefetchStart := time.Now() // Use BatchGet to fill cache. // It's an optimization and could be removed without affecting correctness. diff --git a/executor/update.go b/executor/update.go index 7df144b28196c..16024bc403fa1 100644 --- a/executor/update.go +++ b/executor/update.go @@ -275,6 +275,10 @@ func (e *UpdateExec) updateRows(ctx context.Context) (int, error) { txn, err := e.ctx.Txn(true) if err == nil { txn.SetOption(kv.ResourceGroupTagger, e.ctx.GetSessionVars().StmtCtx.GetResourceGroupTagger()) + if e.ctx.GetSessionVars().StmtCtx.KvExecCounter != nil { + // Bind an interceptor for client-go to count the number of SQL executions of each TiKV. + txn.SetOption(kv.RPCInterceptor, e.ctx.GetSessionVars().StmtCtx.KvExecCounter.RPCInterceptor()) + } } } for rowIdx := 0; rowIdx < chk.NumRows(); rowIdx++ { diff --git a/go.mod b/go.mod index 3c2868df8b118..6f2a22ed5dd5d 100644 --- a/go.mod +++ b/go.mod @@ -65,7 +65,7 @@ require ( github.com/spf13/pflag v1.0.5 github.com/stretchr/testify v1.7.0 github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2 - github.com/tikv/client-go/v2 v2.0.0-rc.0.20211214093715-605f49d3ba50 + github.com/tikv/client-go/v2 v2.0.0-rc.0.20211218050306-6165dbaa95d0 github.com/tikv/pd v1.1.0-beta.0.20211118054146-02848d2660ee github.com/twmb/murmur3 v1.1.3 github.com/uber/jaeger-client-go v2.22.1+incompatible diff --git a/go.sum b/go.sum index a6b2d60e60a01..36c8604668b6b 100644 --- a/go.sum +++ b/go.sum @@ -712,8 +712,8 @@ github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2/go.mod h1:2PfK github.com/tidwall/gjson v1.3.5/go.mod h1:P256ACg0Mn+j1RXIDXoss50DeIABTYK1PULOJHhxOls= github.com/tidwall/match v1.0.1/go.mod h1:LujAq0jyVjBy028G1WhWfIzbpQfMO8bBZ6Tyb0+pL9E= github.com/tidwall/pretty v1.0.0/go.mod h1:XNkn88O1ChpSDQmQeStsy+sBenx6DDtFZJxhVysOjyk= -github.com/tikv/client-go/v2 v2.0.0-rc.0.20211214093715-605f49d3ba50 h1:B+cAIm2P1/SNsVV1vL9/mRaGUVl/vdgV8MU03O0vY28= -github.com/tikv/client-go/v2 v2.0.0-rc.0.20211214093715-605f49d3ba50/go.mod h1:wRuh+W35daKTiYBld0oBlT6PSkzEVr+pB/vChzJZk+8= +github.com/tikv/client-go/v2 v2.0.0-rc.0.20211218050306-6165dbaa95d0 h1:38Jst/O36MKXAt7aD1Ipnx4nKwclG66ifkcmi4f0NZ4= +github.com/tikv/client-go/v2 v2.0.0-rc.0.20211218050306-6165dbaa95d0/go.mod h1:wRuh+W35daKTiYBld0oBlT6PSkzEVr+pB/vChzJZk+8= github.com/tikv/pd v1.1.0-beta.0.20211029083450-e65f0c55b6ae/go.mod h1:varH0IE0jJ9E9WN2Ei/N6pajMlPkcXdDEf7f5mmsUVQ= github.com/tikv/pd v1.1.0-beta.0.20211118054146-02848d2660ee h1:rAAdvQ8Hh36syHr92g0VmZEpkH+40RGQBpFL2121xMs= github.com/tikv/pd v1.1.0-beta.0.20211118054146-02848d2660ee/go.mod h1:lRbwxBAhnTQR5vqbTzeI/Bj62bD2OvYYuFezo2vrmeI= diff --git a/kv/option.go b/kv/option.go index 2a7a17fedcb6c..f3247b5b31b2c 100644 --- a/kv/option.go +++ b/kv/option.go @@ -68,12 +68,14 @@ const ( ResourceGroupTagger // KVFilter indicates the filter to ignore key-values in the transaction's memory buffer. KVFilter - // SnapInterceptor is used for setting the interceptor for snapshot SnapInterceptor // CommitTSUpperBoundChec is used by cached table // The commitTS must be greater than all the write lock lease of the visited cached table. CommitTSUpperBoundCheck + // RPCInterceptor is interceptor.RPCInterceptor on Transaction or Snapshot, used to decorate + // additional logic before and after the underlying client-go RPC request. + RPCInterceptor ) // ReplicaReadType is the type of replica to read data from diff --git a/session/session.go b/session/session.go index a7976a5001bee..b7171a79c9525 100644 --- a/session/session.go +++ b/session/session.go @@ -548,6 +548,10 @@ func (s *session) doCommit(ctx context.Context) error { s.txn.SetOption(kv.EnableAsyncCommit, sessVars.EnableAsyncCommit) s.txn.SetOption(kv.Enable1PC, sessVars.Enable1PC) s.txn.SetOption(kv.ResourceGroupTagger, sessVars.StmtCtx.GetResourceGroupTagger()) + if sessVars.StmtCtx.KvExecCounter != nil { + // Bind an interceptor for client-go to count the number of SQL executions of each TiKV. + s.txn.SetOption(kv.RPCInterceptor, sessVars.StmtCtx.KvExecCounter.RPCInterceptor()) + } // priority of the sysvar is lower than `start transaction with causal consistency only` if val := s.txn.GetOption(kv.GuaranteeLinearizability); val == nil || val.(bool) { // We needn't ask the TiKV client to guarantee linearizability for auto-commit transactions @@ -2311,6 +2315,9 @@ func (s *session) Close() { s.RollbackTxn(ctx) if s.sessionVars != nil { s.sessionVars.WithdrawAllPreparedStmt() + if s.sessionVars.StmtStats != nil { + s.sessionVars.StmtStats.SetFinished() + } } s.ClearDiskFullOpt() } diff --git a/sessionctx/stmtctx/stmtctx.go b/sessionctx/stmtctx/stmtctx.go index e41eb4766b47b..a064f891dd854 100644 --- a/sessionctx/stmtctx/stmtctx.go +++ b/sessionctx/stmtctx/stmtctx.go @@ -29,6 +29,7 @@ import ( "github.com/pingcap/tidb/util/execdetails" "github.com/pingcap/tidb/util/memory" "github.com/pingcap/tidb/util/resourcegrouptag" + "github.com/pingcap/tidb/util/topsql/stmtstats" "github.com/pingcap/tidb/util/tracing" "github.com/tikv/client-go/v2/tikvrpc" "github.com/tikv/client-go/v2/util" @@ -207,6 +208,12 @@ type StatementContext struct { // WaitLockLeaseTime is the duration of cached table read lease expiration time. WaitLockLeaseTime time.Duration + + // KvExecCounter is created from SessionVars.StmtStats to count the number of SQL + // executions of the kv layer during the current execution of the statement. + // Its life cycle is limited to this execution, and a new KvExecCounter is + // always created during each statement execution. + KvExecCounter *stmtstats.KvExecCounter } // StmtHints are SessionVars related sql hints. diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 8169eaa5c2d66..d03cbdce86fde 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -29,8 +29,6 @@ import ( "sync/atomic" "time" - utilMath "github.com/pingcap/tidb/util/math" - "github.com/pingcap/errors" pumpcli "github.com/pingcap/tidb-tools/tidb-binlog/pump_client" "github.com/pingcap/tidb/config" @@ -48,10 +46,12 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/execdetails" + utilMath "github.com/pingcap/tidb/util/math" "github.com/pingcap/tidb/util/rowcodec" "github.com/pingcap/tidb/util/stringutil" "github.com/pingcap/tidb/util/tableutil" "github.com/pingcap/tidb/util/timeutil" + "github.com/pingcap/tidb/util/topsql/stmtstats" tikvstore "github.com/tikv/client-go/v2/kv" "github.com/tikv/client-go/v2/oracle" "github.com/twmb/murmur3" @@ -969,6 +969,13 @@ type SessionVars struct { // EnablePaging indicates whether enable paging in coprocessor requests. EnablePaging bool + + // StmtStats is used to count various indicators of each SQL in this session + // at each point in time. These data will be periodically taken away by the + // background goroutine. The background goroutine will continue to aggregate + // all the local data in each session, and finally report them to the remote + // regularly. + StmtStats *stmtstats.StatementStats } // InitStatementContext initializes a StatementContext, the object is reused to reduce allocation. @@ -1203,6 +1210,7 @@ func NewSessionVars() *SessionVars { MPPStoreFailTTL: DefTiDBMPPStoreFailTTL, EnablePlacementChecks: DefEnablePlacementCheck, Rng: utilMath.NewWithTime(), + StmtStats: stmtstats.CreateStatementStats(), } vars.KVVars = tikvstore.NewVariables(&vars.Killed) vars.Concurrency = Concurrency{ diff --git a/store/driver/txn/snapshot.go b/store/driver/txn/snapshot.go index 3c372bae83725..28b73e15e3228 100644 --- a/store/driver/txn/snapshot.go +++ b/store/driver/txn/snapshot.go @@ -23,6 +23,7 @@ import ( derr "github.com/pingcap/tidb/store/driver/error" "github.com/pingcap/tidb/store/driver/options" "github.com/tikv/client-go/v2/tikvrpc" + "github.com/tikv/client-go/v2/tikvrpc/interceptor" "github.com/tikv/client-go/v2/txnkv/txnsnapshot" "github.com/tikv/client-go/v2/txnkv/txnutil" ) @@ -120,6 +121,8 @@ func (s *tikvSnapshot) SetOption(opt int, val interface{}) { s.KVSnapshot.SetReadReplicaScope(val.(string)) case kv.SnapInterceptor: s.interceptor = val.(kv.SnapshotInterceptor) + case kv.RPCInterceptor: + s.KVSnapshot.SetRPCInterceptor(val.(interceptor.RPCInterceptor)) } } diff --git a/store/driver/txn/txn_driver.go b/store/driver/txn/txn_driver.go index 717bf3b154761..bb9e38a4f3c03 100644 --- a/store/driver/txn/txn_driver.go +++ b/store/driver/txn/txn_driver.go @@ -32,6 +32,7 @@ import ( tikvstore "github.com/tikv/client-go/v2/kv" "github.com/tikv/client-go/v2/tikv" "github.com/tikv/client-go/v2/tikvrpc" + "github.com/tikv/client-go/v2/tikvrpc/interceptor" "github.com/tikv/client-go/v2/txnkv/txnsnapshot" ) @@ -232,6 +233,8 @@ func (txn *tikvTxn) SetOption(opt int, val interface{}) { txn.snapshotInterceptor = val.(kv.SnapshotInterceptor) case kv.CommitTSUpperBoundCheck: txn.KVTxn.SetCommitTSUpperBoundCheck(val.(func(commitTS uint64) bool)) + case kv.RPCInterceptor: + txn.KVTxn.SetRPCInterceptor(val.(interceptor.RPCInterceptor)) } } diff --git a/util/topsql/stmtstats/aggregator.go b/util/topsql/stmtstats/aggregator.go new file mode 100644 index 0000000000000..d78ed7b62dafb --- /dev/null +++ b/util/topsql/stmtstats/aggregator.go @@ -0,0 +1,156 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package stmtstats + +import ( + "context" + "sync" + "time" + + "go.uber.org/atomic" +) + +// globalAggregator is global *aggregator. +var globalAggregator = newAggregator() + +// StatementStatsRecord is the merged StatementStatsMap with timestamp. +type StatementStatsRecord struct { + Timestamp int64 + Data StatementStatsMap +} + +// aggregator is used to collect and aggregate data from all StatementStats. +// It is responsible for collecting data from all StatementStats, aggregating +// them together, uploading them and regularly cleaning up the closed StatementStats. +type aggregator struct { + ctx context.Context + cancel context.CancelFunc + statsSet sync.Map // map[*StatementStats]struct{} + collectors sync.Map // map[Collector]struct{} + running *atomic.Bool +} + +// newAggregator creates an empty aggregator. +func newAggregator() *aggregator { + return &aggregator{running: atomic.NewBool(false)} +} + +// run will block the current goroutine and execute the main loop of aggregator. +func (m *aggregator) run() { + m.ctx, m.cancel = context.WithCancel(context.Background()) + m.running.Store(true) + defer func() { + m.running.Store(false) + }() + tick := time.NewTicker(time.Second) + defer tick.Stop() + for { + select { + case <-m.ctx.Done(): + return + case <-tick.C: + m.aggregate() + } + } +} + +// aggregate data from all associated StatementStats. +// If StatementStats has been closed, collect will remove it from the map. +func (m *aggregator) aggregate() { + r := StatementStatsRecord{ + Timestamp: time.Now().Unix(), + Data: StatementStatsMap{}, + } + m.statsSet.Range(func(statsR, _ interface{}) bool { + stats := statsR.(*StatementStats) + if stats.Finished() { + m.unregister(stats) + } + r.Data.Merge(stats.Take()) + return true + }) + m.collectors.Range(func(c, _ interface{}) bool { + c.(Collector).CollectStmtStatsRecords([]StatementStatsRecord{r}) + return true + }) +} + +// register binds StatementStats to aggregator. +// register is thread-safe. +func (m *aggregator) register(stats *StatementStats) { + m.statsSet.Store(stats, struct{}{}) +} + +// unregister removes StatementStats from aggregator. +// unregister is thread-safe. +func (m *aggregator) unregister(stats *StatementStats) { + m.statsSet.Delete(stats) +} + +// registerCollector binds a Collector to aggregator. +// registerCollector is thread-safe. +func (m *aggregator) registerCollector(collector Collector) { + m.collectors.Store(collector, struct{}{}) +} + +// unregisterCollector removes Collector from aggregator. +// unregisterCollector is thread-safe. +func (m *aggregator) unregisterCollector(collector Collector) { + m.collectors.Delete(collector) +} + +// close ends the execution of the current aggregator. +func (m *aggregator) close() { + m.cancel() +} + +// closed returns whether the aggregator has been closed. +func (m *aggregator) closed() bool { + return !m.running.Load() +} + +// SetupAggregator is used to initialize the background aggregator goroutine of the stmtstats module. +// SetupAggregator is **not** thread-safe. +func SetupAggregator() { + if globalAggregator.closed() { + go globalAggregator.run() + } +} + +// CloseAggregator is used to stop the background aggregator goroutine of the stmtstats module. +// SetupAggregator is **not** thread-safe. +func CloseAggregator() { + if !globalAggregator.closed() { + globalAggregator.close() + } +} + +// RegisterCollector binds a Collector to globalAggregator. +// RegisterCollector is thread-safe. +func RegisterCollector(collector Collector) { + globalAggregator.registerCollector(collector) +} + +// UnregisterCollector removes Collector from globalAggregator. +// UnregisterCollector is thread-safe. +func UnregisterCollector(collector Collector) { + globalAggregator.unregisterCollector(collector) +} + +// Collector is used to collect StatementStatsRecord. +type Collector interface { + // CollectStmtStatsRecords is used to collect list of StatementStatsRecord. + CollectStmtStatsRecords([]StatementStatsRecord) +} diff --git a/util/topsql/stmtstats/aggregator_test.go b/util/topsql/stmtstats/aggregator_test.go new file mode 100644 index 0000000000000..24a72bb89131d --- /dev/null +++ b/util/topsql/stmtstats/aggregator_test.go @@ -0,0 +1,93 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package stmtstats + +import ( + "sync" + "testing" + "time" + + "github.com/stretchr/testify/assert" + "go.uber.org/atomic" +) + +func Test_SetupCloseAggregator(t *testing.T) { + for n := 0; n < 3; n++ { + SetupAggregator() + time.Sleep(100 * time.Millisecond) + assert.False(t, globalAggregator.closed()) + CloseAggregator() + time.Sleep(100 * time.Millisecond) + assert.True(t, globalAggregator.closed()) + } +} + +func Test_RegisterUnregisterCollector(t *testing.T) { + SetupAggregator() + defer CloseAggregator() + time.Sleep(100 * time.Millisecond) + collector := newMockCollector(func(records []StatementStatsRecord) {}) + RegisterCollector(collector) + _, ok := globalAggregator.collectors.Load(collector) + assert.True(t, ok) + UnregisterCollector(collector) + _, ok = globalAggregator.collectors.Load(collector) + assert.False(t, ok) +} + +func Test_aggregator_register_collect(t *testing.T) { + a := newAggregator() + stats := &StatementStats{ + data: StatementStatsMap{}, + finished: atomic.NewBool(false), + } + a.register(stats) + stats.OnExecutionBegin([]byte("SQL-1"), []byte("")) + var records []StatementStatsRecord + a.registerCollector(newMockCollector(func(rs []StatementStatsRecord) { + records = append(records, rs...) + })) + a.aggregate() + assert.NotEmpty(t, records) + assert.Equal(t, uint64(1), records[0].Data[SQLPlanDigest{SQLDigest: "SQL-1"}].ExecCount) +} + +func Test_aggregator_run_close(t *testing.T) { + wg := sync.WaitGroup{} + a := newAggregator() + assert.True(t, a.closed()) + wg.Add(1) + go func() { + a.run() + wg.Done() + }() + time.Sleep(100 * time.Millisecond) + assert.False(t, a.closed()) + a.close() + wg.Wait() + assert.True(t, a.closed()) +} + +type mockCollector struct { + f func(records []StatementStatsRecord) +} + +func newMockCollector(f func(records []StatementStatsRecord)) Collector { + return &mockCollector{f: f} +} + +func (c *mockCollector) CollectStmtStatsRecords(records []StatementStatsRecord) { + c.f(records) +} diff --git a/util/topsql/stmtstats/kv_exec_count.go b/util/topsql/stmtstats/kv_exec_count.go new file mode 100644 index 0000000000000..7da4dc8eebdcd --- /dev/null +++ b/util/topsql/stmtstats/kv_exec_count.go @@ -0,0 +1,73 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package stmtstats + +import ( + "sync" + + "github.com/tikv/client-go/v2/tikvrpc" + "github.com/tikv/client-go/v2/tikvrpc/interceptor" +) + +// CreateKvExecCounter creates an associated KvExecCounter from StatementStats. +// The created KvExecCounter can only be used during a single statement execution +// and cannot be reused. +func (s *StatementStats) CreateKvExecCounter(sqlDigest, planDigest []byte) *KvExecCounter { + return &KvExecCounter{ + stats: s, + digest: SQLPlanDigest{SQLDigest: BinaryDigest(sqlDigest), PlanDigest: BinaryDigest(planDigest)}, + marked: map[string]struct{}{}, + } +} + +// KvExecCounter is used to count the number of SQL executions of the kv layer. +// It internally calls addKvExecCount of StatementStats at the right time, to +// ensure the semantic of "SQL execution count of TiKV". +type KvExecCounter struct { + stats *StatementStats + digest SQLPlanDigest + mu sync.Mutex + marked map[string]struct{} // HashSet +} + +// RPCInterceptor returns an interceptor.RPCInterceptor for client-go. +// The returned interceptor is generally expected to be bind to transaction or +// snapshot. In this way, the logic preset by KvExecCounter will be executed before +// each RPC request is initiated, in order to count the number of SQL executions of +// the TiKV dimension. +func (c *KvExecCounter) RPCInterceptor() interceptor.RPCInterceptor { + return func(next interceptor.RPCInterceptorFunc) interceptor.RPCInterceptorFunc { + return func(target string, req *tikvrpc.Request) (*tikvrpc.Response, error) { + c.mark(target) + return next(target, req) + } + } +} + +// mark this target during the current execution of statement. +// If this target is marked for the first time, then increase the number of execution. +// mark is thread-safe. +func (c *KvExecCounter) mark(target string) { + firstMark := false + c.mu.Lock() + if _, ok := c.marked[target]; !ok { + c.marked[target] = struct{}{} + firstMark = true + } + c.mu.Unlock() + if firstMark { + c.stats.addKvExecCount([]byte(c.digest.SQLDigest), []byte(c.digest.PlanDigest), target, 1) + } +} diff --git a/util/topsql/stmtstats/kv_exec_count_test.go b/util/topsql/stmtstats/kv_exec_count_test.go new file mode 100644 index 0000000000000..c55a5300c0891 --- /dev/null +++ b/util/topsql/stmtstats/kv_exec_count_test.go @@ -0,0 +1,43 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package stmtstats + +import ( + "testing" + + "github.com/stretchr/testify/assert" + "github.com/tikv/client-go/v2/tikvrpc" +) + +func TestKvExecCounter(t *testing.T) { + stats := CreateStatementStats() + counter := stats.CreateKvExecCounter([]byte("SQL-1"), []byte("")) + interceptor := counter.RPCInterceptor() + for n := 0; n < 10; n++ { + _, _ = interceptor(func(target string, req *tikvrpc.Request) (*tikvrpc.Response, error) { + return nil, nil + })("TIKV-1", nil) + } + for n := 0; n < 10; n++ { + _, _ = interceptor(func(target string, req *tikvrpc.Request) (*tikvrpc.Response, error) { + return nil, nil + })("TIKV-2", nil) + } + assert.Len(t, counter.marked, 2) + assert.Contains(t, counter.marked, "TIKV-1") + assert.Contains(t, counter.marked, "TIKV-2") + assert.NotNil(t, stats.data[SQLPlanDigest{SQLDigest: "SQL-1"}]) + assert.Equal(t, uint64(1), stats.data[SQLPlanDigest{SQLDigest: "SQL-1"}].KvStatsItem.KvExecCount["TIKV-1"]) +} diff --git a/util/topsql/stmtstats/main_test.go b/util/topsql/stmtstats/main_test.go new file mode 100644 index 0000000000000..24f6c2574c522 --- /dev/null +++ b/util/topsql/stmtstats/main_test.go @@ -0,0 +1,27 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package stmtstats + +import ( + "testing" + + "github.com/pingcap/tidb/util/testbridge" + "go.uber.org/goleak" +) + +func TestMain(m *testing.M) { + testbridge.WorkaroundGoCheckFlags() + goleak.VerifyTestMain(m) +} diff --git a/util/topsql/stmtstats/stmtstats.go b/util/topsql/stmtstats/stmtstats.go new file mode 100644 index 0000000000000..24faa93899cce --- /dev/null +++ b/util/topsql/stmtstats/stmtstats.go @@ -0,0 +1,219 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package stmtstats + +import ( + "sync" + + "go.uber.org/atomic" +) + +var _ StatementObserver = &StatementStats{} + +// StatementObserver is an abstract interface as a callback to the corresponding +// position of TiDB's SQL statement execution process. StatementStats implements +// StatementObserver and performs counting such as SQLExecCount/SQLDuration internally. +// The caller only needs to be responsible for calling different methods at the +// corresponding locations, without paying attention to implementation details. +type StatementObserver interface { + // OnExecutionBegin should be called before statement execution. + OnExecutionBegin(sqlDigest, planDigest []byte) + + // OnExecutionFinished should be called after the statement is executed. + OnExecutionFinished(sqlDigest, planDigest []byte) +} + +// StatementStats is a counter used locally in each session. +// We can use StatementStats to count data such as "the number of SQL executions", +// and it is expected that these statistics will eventually be collected and merged +// in the background. +type StatementStats struct { + mu sync.Mutex + data StatementStatsMap + finished *atomic.Bool +} + +// CreateStatementStats try to create and register an StatementStats. +func CreateStatementStats() *StatementStats { + stats := &StatementStats{ + data: StatementStatsMap{}, + finished: atomic.NewBool(false), + } + globalAggregator.register(stats) + return stats +} + +// OnExecutionBegin implements StatementObserver.OnExecutionBegin. +func (s *StatementStats) OnExecutionBegin(sqlDigest, planDigest []byte) { + s.mu.Lock() + defer s.mu.Unlock() + item := s.GetOrCreateStatementStatsItem(sqlDigest, planDigest) + + item.ExecCount++ + // Count more data here. +} + +// OnExecutionFinished implements StatementObserver.OnExecutionFinished. +func (s *StatementStats) OnExecutionFinished(sqlDigest, planDigest []byte) { + // Count more data here. +} + +// GetOrCreateStatementStatsItem creates the corresponding StatementStatsItem +// for the specified SQLPlanDigest and timestamp if it does not exist before. +// GetOrCreateStatementStatsItem is just a helper function, not responsible for +// concurrency control, so GetOrCreateStatementStatsItem is **not** thread-safe. +func (s *StatementStats) GetOrCreateStatementStatsItem(sqlDigest, planDigest []byte) *StatementStatsItem { + key := SQLPlanDigest{SQLDigest: BinaryDigest(sqlDigest), PlanDigest: BinaryDigest(planDigest)} + item, ok := s.data[key] + if !ok { + s.data[key] = NewStatementStatsItem() + item = s.data[key] + } + return item +} + +// addKvExecCount is used to count the number of executions of a certain SQLPlanDigest for a certain target. +// addKvExecCount is thread-safe. +func (s *StatementStats) addKvExecCount(sqlDigest, planDigest []byte, target string, n uint64) { + s.mu.Lock() + defer s.mu.Unlock() + item := s.GetOrCreateStatementStatsItem(sqlDigest, planDigest) + item.KvStatsItem.KvExecCount[target] += n +} + +// Take takes out all existing StatementStatsMap data from StatementStats. +// Take is thread-safe. +func (s *StatementStats) Take() StatementStatsMap { + s.mu.Lock() + defer s.mu.Unlock() + data := s.data + s.data = StatementStatsMap{} + return data +} + +// SetFinished marks this StatementStats as "finished" and no more counting or +// aggregation should happen. Associated resources will be cleaned up, like background +// aggregators. +// Generally, as the StatementStats is created when a session starts, SetFinished +// should be called when the session ends. +func (s *StatementStats) SetFinished() { + s.finished.Store(true) +} + +// Finished returns whether the StatementStats has been finished. +func (s *StatementStats) Finished() bool { + return s.finished.Load() +} + +// BinaryDigest is converted from parser.Digest.Bytes(), and the purpose +// is to be used as the key of the map. +type BinaryDigest string + +// SQLPlanDigest is used as the key of StatementStatsMap to +// distinguish different sql. +type SQLPlanDigest struct { + SQLDigest BinaryDigest + PlanDigest BinaryDigest +} + +// StatementStatsMap is the local data type of StatementStats. +type StatementStatsMap map[SQLPlanDigest]*StatementStatsItem + +// Merge merges other into StatementStatsMap. +// Values with the same SQLPlanDigest will be merged. +// +// After executing Merge, some pointers in other may be referenced +// by m. So after calling Merge, it is best not to continue to use +// other unless you understand what you are doing. +func (m StatementStatsMap) Merge(other StatementStatsMap) { + if m == nil || other == nil { + return + } + for newDigest, newItem := range other { + item, ok := m[newDigest] + if !ok { + m[newDigest] = newItem + continue + } + item.Merge(newItem) + } +} + +// StatementStatsItem represents a set of mergeable statistics. +// StatementStatsItem is used in a larger data structure to represent +// the stats of a certain SQLPlanDigest under a certain timestamp. +// If there are more indicators that need to be added in the future, +// please add it in StatementStatsItem and implement its aggregation +// in the Merge method. +type StatementStatsItem struct { + // ExecCount represents the number of SQL executions of TiDB. + ExecCount uint64 + + // KvStatsItem contains all indicators of kv layer. + KvStatsItem KvStatementStatsItem +} + +// NewStatementStatsItem creates an empty StatementStatsItem. +func NewStatementStatsItem() *StatementStatsItem { + return &StatementStatsItem{ + KvStatsItem: NewKvStatementStatsItem(), + } +} + +// Merge merges other into StatementStatsItem. +// +// After executing Merge, some pointers in other may be referenced +// by i. So after calling Merge, it is best not to continue to use +// other unless you understand what you are doing. +// +// If you add additional indicators, you need to add their merge code here. +func (i *StatementStatsItem) Merge(other *StatementStatsItem) { + if i == nil || other == nil { + return + } + i.ExecCount += other.ExecCount + i.KvStatsItem.Merge(other.KvStatsItem) +} + +// KvStatementStatsItem is part of StatementStatsItem, it only contains +// indicators of kv layer. +type KvStatementStatsItem struct { + // KvExecCount represents the number of SQL executions of TiKV. + KvExecCount map[string]uint64 +} + +// NewKvStatementStatsItem creates an empty KvStatementStatsItem. +func NewKvStatementStatsItem() KvStatementStatsItem { + return KvStatementStatsItem{ + KvExecCount: map[string]uint64{}, + } +} + +// Merge merges other into KvStatementStatsItem. +// +// After executing Merge, some pointers in other may be referenced +// by i. So after calling Merge, it is best not to continue to use +// other unless you understand what you are doing. +// +// If you add additional indicators, you need to add their merge code here. +func (i *KvStatementStatsItem) Merge(other KvStatementStatsItem) { + if i.KvExecCount == nil { + i.KvExecCount = other.KvExecCount + } else { + for target, count := range other.KvExecCount { + i.KvExecCount[target] += count + } + } +} diff --git a/util/topsql/stmtstats/stmtstats_test.go b/util/topsql/stmtstats/stmtstats_test.go new file mode 100644 index 0000000000000..b78208d918d76 --- /dev/null +++ b/util/topsql/stmtstats/stmtstats_test.go @@ -0,0 +1,187 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package stmtstats + +import ( + "bytes" + "encoding/json" + "fmt" + "testing" + + "github.com/stretchr/testify/assert" +) + +// String is only used for debugging. +func (d SQLPlanDigest) String() string { + bs := bytes.NewBufferString("") + if len(d.SQLDigest) >= 5 { + bs.Write([]byte(d.SQLDigest)[:5]) + } + if len(d.PlanDigest) >= 5 { + bs.WriteRune('-') + bs.Write([]byte(d.PlanDigest)[:5]) + } + return bs.String() +} + +// String is only used for debugging. +func (m StatementStatsMap) String() string { + if len(m) == 0 { + return "StatementStatsMap {}" + } + bs := bytes.NewBufferString("") + bs.WriteString("StatementStatsMap {\n") + for k, v := range m { + bs.WriteString(fmt.Sprintf(" %s => %s\n", k, v)) + } + bs.WriteString("}") + return bs.String() +} + +// String is only used for debugging. +func (i *StatementStatsItem) String() string { + if i == nil { + return "" + } + b, _ := json.Marshal(i) + return string(b) +} + +func TestKvStatementStatsItem_Merge(t *testing.T) { + item1 := KvStatementStatsItem{ + KvExecCount: map[string]uint64{ + "127.0.0.1:10001": 1, + "127.0.0.1:10002": 2, + }, + } + item2 := KvStatementStatsItem{ + KvExecCount: map[string]uint64{ + "127.0.0.1:10002": 2, + "127.0.0.1:10003": 3, + }, + } + assert.Len(t, item1.KvExecCount, 2) + assert.Len(t, item2.KvExecCount, 2) + item1.Merge(item2) + assert.Len(t, item1.KvExecCount, 3) + assert.Len(t, item2.KvExecCount, 2) + assert.Equal(t, uint64(1), item1.KvExecCount["127.0.0.1:10001"]) + assert.Equal(t, uint64(3), item1.KvExecCount["127.0.0.1:10003"]) + assert.Equal(t, uint64(3), item1.KvExecCount["127.0.0.1:10003"]) +} + +func TestStatementsStatsItem_Merge(t *testing.T) { + item1 := &StatementStatsItem{ + ExecCount: 1, + KvStatsItem: NewKvStatementStatsItem(), + } + item2 := &StatementStatsItem{ + ExecCount: 2, + KvStatsItem: NewKvStatementStatsItem(), + } + item1.Merge(item2) + assert.Equal(t, uint64(3), item1.ExecCount) +} + +func TestStatementStatsMap_Merge(t *testing.T) { + m1 := StatementStatsMap{ + SQLPlanDigest{SQLDigest: "SQL-1"}: &StatementStatsItem{ + ExecCount: 1, + KvStatsItem: KvStatementStatsItem{ + KvExecCount: map[string]uint64{ + "KV-1": 1, + "KV-2": 2, + }, + }, + }, + SQLPlanDigest{SQLDigest: "SQL-2"}: &StatementStatsItem{ + ExecCount: 1, + KvStatsItem: KvStatementStatsItem{ + KvExecCount: map[string]uint64{ + "KV-1": 1, + "KV-2": 2, + }, + }, + }, + } + m2 := StatementStatsMap{ + SQLPlanDigest{SQLDigest: "SQL-2"}: &StatementStatsItem{ + ExecCount: 1, + KvStatsItem: KvStatementStatsItem{ + KvExecCount: map[string]uint64{ + "KV-1": 1, + "KV-2": 2, + }, + }, + }, + SQLPlanDigest{SQLDigest: "SQL-3"}: &StatementStatsItem{ + ExecCount: 1, + KvStatsItem: KvStatementStatsItem{ + KvExecCount: map[string]uint64{ + "KV-1": 1, + "KV-2": 2, + }, + }, + }, + } + assert.Len(t, m1, 2) + assert.Len(t, m2, 2) + m1.Merge(m2) + assert.Len(t, m1, 3) + assert.Len(t, m2, 2) + assert.Equal(t, uint64(1), m1[SQLPlanDigest{SQLDigest: "SQL-1"}].ExecCount) + assert.Equal(t, uint64(2), m1[SQLPlanDigest{SQLDigest: "SQL-2"}].ExecCount) + assert.Equal(t, uint64(1), m1[SQLPlanDigest{SQLDigest: "SQL-3"}].ExecCount) + assert.Equal(t, uint64(1), m1[SQLPlanDigest{SQLDigest: "SQL-1"}].KvStatsItem.KvExecCount["KV-1"]) + assert.Equal(t, uint64(2), m1[SQLPlanDigest{SQLDigest: "SQL-1"}].KvStatsItem.KvExecCount["KV-2"]) + assert.Equal(t, uint64(2), m1[SQLPlanDigest{SQLDigest: "SQL-2"}].KvStatsItem.KvExecCount["KV-1"]) + assert.Equal(t, uint64(4), m1[SQLPlanDigest{SQLDigest: "SQL-2"}].KvStatsItem.KvExecCount["KV-2"]) + assert.Equal(t, uint64(1), m1[SQLPlanDigest{SQLDigest: "SQL-3"}].KvStatsItem.KvExecCount["KV-1"]) + assert.Equal(t, uint64(2), m1[SQLPlanDigest{SQLDigest: "SQL-3"}].KvStatsItem.KvExecCount["KV-2"]) + m1.Merge(nil) + assert.Len(t, m1, 3) +} + +func TestCreateStatementStats(t *testing.T) { + stats := CreateStatementStats() + assert.NotNil(t, stats) + _, ok := globalAggregator.statsSet.Load(stats) + assert.True(t, ok) + assert.False(t, stats.Finished()) + stats.SetFinished() + assert.True(t, stats.Finished()) +} + +func TestExecCounter_AddExecCount_Take(t *testing.T) { + stats := CreateStatementStats() + m := stats.Take() + assert.Len(t, m, 0) + for n := 0; n < 1; n++ { + stats.OnExecutionBegin([]byte("SQL-1"), []byte("")) + } + for n := 0; n < 2; n++ { + stats.OnExecutionBegin([]byte("SQL-2"), []byte("")) + } + for n := 0; n < 3; n++ { + stats.OnExecutionBegin([]byte("SQL-3"), []byte("")) + } + m = stats.Take() + assert.Len(t, m, 3) + assert.Equal(t, uint64(1), m[SQLPlanDigest{SQLDigest: "SQL-1"}].ExecCount) + assert.Equal(t, uint64(2), m[SQLPlanDigest{SQLDigest: "SQL-2"}].ExecCount) + assert.Equal(t, uint64(3), m[SQLPlanDigest{SQLDigest: "SQL-3"}].ExecCount) + m = stats.Take() + assert.Len(t, m, 0) +} diff --git a/util/topsql/stmtstats/stmtstatstest/main_test.go b/util/topsql/stmtstats/stmtstatstest/main_test.go new file mode 100644 index 0000000000000..ecf1220642ecf --- /dev/null +++ b/util/topsql/stmtstats/stmtstatstest/main_test.go @@ -0,0 +1,31 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package stmtstatstest + +import ( + "testing" + + "github.com/pingcap/tidb/util/testbridge" + "go.uber.org/goleak" +) + +func TestMain(m *testing.M) { + testbridge.WorkaroundGoCheckFlags() + opts := []goleak.Option{ + goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), + } + goleak.VerifyTestMain(m, opts...) +} diff --git a/util/topsql/stmtstats/stmtstatstest/stmtstats_test.go b/util/topsql/stmtstats/stmtstatstest/stmtstats_test.go new file mode 100644 index 0000000000000..d37de52178e0e --- /dev/null +++ b/util/topsql/stmtstats/stmtstatstest/stmtstats_test.go @@ -0,0 +1,150 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package stmtstatstest + +import ( + "fmt" + "sync" + "testing" + "time" + + "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/parser" + "github.com/pingcap/tidb/session" + "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/store/mockstore" + "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/util/topsql/stmtstats" + "github.com/stretchr/testify/assert" + "github.com/tikv/client-go/v2/testutils" +) + +func TestExecCount(t *testing.T) { + // Prepare stmt stats. + stmtstats.SetupAggregator() + defer stmtstats.CloseAggregator() + + // Register stmt stats collector. + var mu sync.Mutex + total := stmtstats.StatementStatsMap{} + stmtstats.RegisterCollector(newMockCollector(func(rs []stmtstats.StatementStatsRecord) { + mu.Lock() + defer mu.Unlock() + for _, r := range rs { + total.Merge(r.Data) + } + })) + + // Create mock store. + store, err := mockstore.NewMockStore(mockstore.WithClusterInspector(func(c testutils.Cluster) { + mockstore.BootstrapWithSingleStore(c) + })) + assert.NoError(t, err) + defer func() { + assert.NoError(t, store.Close()) + }() + + // Prepare mock store. + session.SetSchemaLease(0) + session.DisableStats4Test() + d, err := session.BootstrapSession(store) + assert.NoError(t, err) + defer d.Close() + d.SetStatsUpdating(true) + config.UpdateGlobal(func(conf *config.Config) { + conf.OOMAction = config.OOMActionLog + }) + + // Create table for testing. + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t(a int);") + + // Enable TopSQL + variable.TopSQLVariable.Enable.Store(true) + config.UpdateGlobal(func(conf *config.Config) { + conf.TopSQL.ReceiverAddress = "mock-agent" + }) + + // Execute CRUD. + const ExecCountPerSQL = 100 + _, insertSQLDigest := parser.NormalizeDigest("insert into t values (0);") + for n := 0; n < ExecCountPerSQL; n++ { + tk.MustExec(fmt.Sprintf("insert into t values (%d);", n)) + } + _, updateSQLDigest := parser.NormalizeDigest("update t set a = 0 where a = 0;") + for n := 0; n < ExecCountPerSQL; n++ { + tk.MustExec(fmt.Sprintf("update t set a = %d where a = %d;", n, n)) + } + _, selectSQLDigest := parser.NormalizeDigest("select a from t where a = 0;") + for n := 0; n < ExecCountPerSQL; n++ { + tk.MustQuery(fmt.Sprintf("select a from t where a = %d;", n)) + } + _, deleteSQLDigest := parser.NormalizeDigest("delete from t where a = 0;") + for n := 1; n <= ExecCountPerSQL; n++ { + tk.MustExec(fmt.Sprintf("delete from t where a = %d;", n)) + } + + // Wait for collect. + time.Sleep(2 * time.Second) + + // Assertion. + func() { + mu.Lock() + defer mu.Unlock() + + assert.NotEmpty(t, total) + sqlDigests := map[stmtstats.BinaryDigest]struct{}{ + stmtstats.BinaryDigest(insertSQLDigest.Bytes()): {}, + stmtstats.BinaryDigest(updateSQLDigest.Bytes()): {}, + stmtstats.BinaryDigest(selectSQLDigest.Bytes()): {}, + stmtstats.BinaryDigest(deleteSQLDigest.Bytes()): {}, + } + found := 0 + for digest, item := range total { + if _, ok := sqlDigests[digest.SQLDigest]; ok { + found++ + assert.Equal(t, uint64(ExecCountPerSQL), item.ExecCount) + var kvSum uint64 + for _, kvCount := range item.KvStatsItem.KvExecCount { + kvSum += kvCount + } + assert.Equal(t, uint64(ExecCountPerSQL), kvSum) + } + } + assert.Equal(t, 4, found) // insert, update, select, delete + }() + + // Drop table. + tk.MustExec("use test") + r := tk.MustQuery("show tables") + for _, tb := range r.Rows() { + tableName := tb[0] + tk.MustExec(fmt.Sprintf("drop table %v", tableName)) + } +} + +type mockCollector struct { + f func(records []stmtstats.StatementStatsRecord) +} + +func newMockCollector(f func(records []stmtstats.StatementStatsRecord)) stmtstats.Collector { + return &mockCollector{f: f} +} + +func (c *mockCollector) CollectStmtStatsRecords(records []stmtstats.StatementStatsRecord) { + c.f(records) +} diff --git a/util/topsql/topsql.go b/util/topsql/topsql.go index 8f2aac1566642..12a9da430e7c0 100644 --- a/util/topsql/topsql.go +++ b/util/topsql/topsql.go @@ -25,6 +25,7 @@ import ( "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/plancodec" "github.com/pingcap/tidb/util/topsql/reporter" + "github.com/pingcap/tidb/util/topsql/stmtstats" "github.com/pingcap/tidb/util/topsql/tracecpu" "go.uber.org/zap" ) @@ -50,6 +51,7 @@ func SetupTopSQL() { tracecpu.GlobalSQLCPUProfiler.SetCollector(remoteReporter) tracecpu.GlobalSQLCPUProfiler.Run() + stmtstats.SetupAggregator() } // Close uses to close and release the top sql resource. @@ -60,6 +62,7 @@ func Close() { if globalTopSQLReport != nil { globalTopSQLReport.Close() } + stmtstats.CloseAggregator() } // AttachSQLInfo attach the sql information info top sql. From a8a858bac27c618fa3c4f28daf79393a7b67b8ee Mon Sep 17 00:00:00 2001 From: Zhenchi Date: Tue, 21 Dec 2021 16:17:47 +0800 Subject: [PATCH 05/44] topsql: add pubsub datasink (#30860) --- server/rpc_server.go | 2 + sessionctx/variable/tidb_vars.go | 2 +- util/topsql/reporter/mock/pubsub.go | 67 +++++++ util/topsql/reporter/pubsub.go | 267 ++++++++++++++++++++++++++ util/topsql/reporter/single_target.go | 49 +++-- util/topsql/topsql.go | 12 +- util/topsql/topsql_test.go | 160 +++++++++++++++ 7 files changed, 537 insertions(+), 22 deletions(-) create mode 100644 util/topsql/reporter/mock/pubsub.go create mode 100644 util/topsql/reporter/pubsub.go diff --git a/server/rpc_server.go b/server/rpc_server.go index 674047781a6bd..3b23539c0bac1 100644 --- a/server/rpc_server.go +++ b/server/rpc_server.go @@ -33,6 +33,7 @@ import ( "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/memory" + "github.com/pingcap/tidb/util/topsql" "go.uber.org/zap" "google.golang.org/grpc" "google.golang.org/grpc/keepalive" @@ -64,6 +65,7 @@ func NewRPCServer(config *config.Config, dom *domain.Domain, sm util.SessionMana } diagnosticspb.RegisterDiagnosticsServer(s, rpcSrv) tikvpb.RegisterTikvServer(s, rpcSrv) + topsql.RegisterPubSubServer(s) return s } diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index ee01348a76441..ccec9e1b5a8fc 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -832,5 +832,5 @@ type TopSQL struct { // TopSQLEnabled uses to check whether enabled the top SQL feature. func TopSQLEnabled() bool { - return TopSQLVariable.Enable.Load() && config.GetGlobalConfig().TopSQL.ReceiverAddress != "" + return TopSQLVariable.Enable.Load() } diff --git a/util/topsql/reporter/mock/pubsub.go b/util/topsql/reporter/mock/pubsub.go new file mode 100644 index 0000000000000..493d95c17f827 --- /dev/null +++ b/util/topsql/reporter/mock/pubsub.go @@ -0,0 +1,67 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package mock + +import ( + "fmt" + "net" + + "github.com/pingcap/tidb/util/logutil" + "go.uber.org/zap" + "google.golang.org/grpc" +) + +type mockPubSubServer struct { + addr string + listen net.Listener + grpcServer *grpc.Server +} + +// NewMockPubSubServer creates a mock publisher server. +func NewMockPubSubServer() (*mockPubSubServer, error) { + addr := "127.0.0.1:0" + lis, err := net.Listen("tcp", addr) + if err != nil { + return nil, err + } + server := grpc.NewServer() + + return &mockPubSubServer{ + addr: fmt.Sprintf("127.0.0.1:%d", lis.Addr().(*net.TCPAddr).Port), + listen: lis, + grpcServer: server, + }, nil +} + +func (svr *mockPubSubServer) Serve() { + err := svr.grpcServer.Serve(svr.listen) + if err != nil { + logutil.BgLogger().Warn("[top-sql] mock pubsub server serve failed", zap.Error(err)) + } +} + +func (svr *mockPubSubServer) Server() *grpc.Server { + return svr.grpcServer +} + +func (svr *mockPubSubServer) Address() string { + return svr.addr +} + +func (svr *mockPubSubServer) Stop() { + if svr.grpcServer != nil { + svr.grpcServer.Stop() + } +} diff --git a/util/topsql/reporter/pubsub.go b/util/topsql/reporter/pubsub.go new file mode 100644 index 0000000000000..7d01c077e058f --- /dev/null +++ b/util/topsql/reporter/pubsub.go @@ -0,0 +1,267 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package reporter + +import ( + "context" + "errors" + "time" + + "github.com/pingcap/tidb/util" + "github.com/pingcap/tidb/util/logutil" + "github.com/pingcap/tipb/go-tipb" + "go.uber.org/zap" +) + +// TopSQLPubSubService implements tipb.TopSQLPubSubServer. +// +// If a client subscribes to TopSQL records, the TopSQLPubSubService is responsible +// for registering an associated DataSink to the reporter. Then the DataSink sends +// data to the client periodically. +type TopSQLPubSubService struct { + dataSinkRegisterer DataSinkRegisterer +} + +// NewTopSQLPubSubService creates a new TopSQLPubSubService. +func NewTopSQLPubSubService(dataSinkRegisterer DataSinkRegisterer) *TopSQLPubSubService { + return &TopSQLPubSubService{dataSinkRegisterer: dataSinkRegisterer} +} + +var _ tipb.TopSQLPubSubServer = &TopSQLPubSubService{} + +// Subscribe registers dataSinks to the reporter and redirects data received from reporter +// to subscribers associated with those dataSinks. +func (ps *TopSQLPubSubService) Subscribe(_ *tipb.TopSQLSubRequest, stream tipb.TopSQLPubSub_SubscribeServer) error { + ds := newPubSubDataSink(stream, ps.dataSinkRegisterer) + if err := ps.dataSinkRegisterer.Register(ds); err != nil { + return err + } + return ds.run() +} + +type pubSubDataSink struct { + ctx context.Context + cancel context.CancelFunc + + stream tipb.TopSQLPubSub_SubscribeServer + sendTaskCh chan sendTask + + // for deregister + registerer DataSinkRegisterer +} + +func newPubSubDataSink(stream tipb.TopSQLPubSub_SubscribeServer, registerer DataSinkRegisterer) *pubSubDataSink { + ctx, cancel := context.WithCancel(stream.Context()) + + return &pubSubDataSink{ + ctx: ctx, + cancel: cancel, + + stream: stream, + sendTaskCh: make(chan sendTask, 1), + + registerer: registerer, + } +} + +var _ DataSink = &pubSubDataSink{} + +func (ds *pubSubDataSink) TrySend(data *ReportData, deadline time.Time) error { + select { + case ds.sendTaskCh <- sendTask{data: data, deadline: deadline}: + return nil + case <-ds.ctx.Done(): + return ds.ctx.Err() + default: + ignoreReportChannelFullCounter.Inc() + return errors.New("the channel of pubsub dataSink is full") + } +} + +func (ds *pubSubDataSink) OnReporterClosing() { + ds.cancel() +} + +func (ds *pubSubDataSink) run() error { + defer func() { + ds.registerer.Deregister(ds) + ds.cancel() + }() + + for { + select { + case task := <-ds.sendTaskCh: + ctx, cancel := context.WithDeadline(ds.ctx, task.deadline) + var err error + + start := time.Now() + go util.WithRecovery(func() { + defer cancel() + err = ds.doSend(ctx, task.data) + + if err != nil { + reportAllDurationFailedHistogram.Observe(time.Since(start).Seconds()) + } else { + reportAllDurationSuccHistogram.Observe(time.Since(start).Seconds()) + } + }, nil) + + // When the deadline is exceeded, the closure inside `go util.WithRecovery` above may not notice that + // immediately because it can be blocked by `stream.Send`. + // In order to clean up resources as quickly as possible, we let that closure run in an individual goroutine, + // and wait for timeout here. + <-ctx.Done() + + if errors.Is(ctx.Err(), context.DeadlineExceeded) { + logutil.BgLogger().Warn( + "[top-sql] pubsub datasink failed to send data to subscriber due to deadline exceeded", + zap.Time("deadline", task.deadline), + ) + return ctx.Err() + } + + if err != nil { + logutil.BgLogger().Warn( + "[top-sql] pubsub datasink failed to send data to subscriber", + zap.Error(err), + ) + return err + } + case <-ds.ctx.Done(): + return ds.ctx.Err() + } + } +} + +func (ds *pubSubDataSink) doSend(ctx context.Context, data *ReportData) error { + if err := ds.sendCPUTime(ctx, data.CPUTimeRecords); err != nil { + return err + } + if err := ds.sendSQLMeta(ctx, data.SQLMetas); err != nil { + return err + } + return ds.sendPlanMeta(ctx, data.PlanMetas) +} + +func (ds *pubSubDataSink) sendCPUTime(ctx context.Context, records []tipb.CPUTimeRecord) (err error) { + if len(records) == 0 { + return + } + + start := time.Now() + sentCount := 0 + defer func() { + topSQLReportRecordCounterHistogram.Observe(float64(sentCount)) + if err != nil { + reportRecordDurationFailedHistogram.Observe(time.Since(start).Seconds()) + } else { + reportRecordDurationSuccHistogram.Observe(time.Since(start).Seconds()) + } + }() + + cpuRecord := &tipb.TopSQLSubResponse_Record{} + r := &tipb.TopSQLSubResponse{RespOneof: cpuRecord} + + for i := range records { + cpuRecord.Record = &records[i] + if err = ds.stream.Send(r); err != nil { + return + } + sentCount += 1 + + select { + case <-ctx.Done(): + err = ctx.Err() + return + default: + } + } + + return +} + +func (ds *pubSubDataSink) sendSQLMeta(ctx context.Context, sqlMetas []tipb.SQLMeta) (err error) { + if len(sqlMetas) == 0 { + return + } + + start := time.Now() + sentCount := 0 + defer func() { + topSQLReportSQLCountHistogram.Observe(float64(sentCount)) + if err != nil { + reportSQLDurationFailedHistogram.Observe(time.Since(start).Seconds()) + } else { + reportSQLDurationSuccHistogram.Observe(time.Since(start).Seconds()) + } + }() + + sqlMeta := &tipb.TopSQLSubResponse_SqlMeta{} + r := &tipb.TopSQLSubResponse{RespOneof: sqlMeta} + + for i := range sqlMetas { + sqlMeta.SqlMeta = &sqlMetas[i] + if err = ds.stream.Send(r); err != nil { + return + } + sentCount += 1 + + select { + case <-ctx.Done(): + err = ctx.Err() + return + default: + } + } + + return +} + +func (ds *pubSubDataSink) sendPlanMeta(ctx context.Context, planMetas []tipb.PlanMeta) (err error) { + if len(planMetas) == 0 { + return + } + + start := time.Now() + sentCount := 0 + defer func() { + topSQLReportPlanCountHistogram.Observe(float64(sentCount)) + if err != nil { + reportPlanDurationFailedHistogram.Observe(time.Since(start).Seconds()) + } else { + reportPlanDurationSuccHistogram.Observe(time.Since(start).Seconds()) + } + }() + + planMeta := &tipb.TopSQLSubResponse_PlanMeta{} + r := &tipb.TopSQLSubResponse{RespOneof: planMeta} + + for i := range planMetas { + planMeta.PlanMeta = &planMetas[i] + if err = ds.stream.Send(r); err != nil { + return + } + sentCount += 1 + + select { + case <-ctx.Done(): + err = ctx.Err() + return + default: + } + } + + return +} diff --git a/util/topsql/reporter/single_target.go b/util/topsql/reporter/single_target.go index 3ea61d75f633a..3744702ba26d6 100644 --- a/util/topsql/reporter/single_target.go +++ b/util/topsql/reporter/single_target.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tipb/go-tipb" + "go.uber.org/atomic" "go.uber.org/zap" "google.golang.org/grpc" "google.golang.org/grpc/backoff" @@ -38,7 +39,7 @@ type SingleTargetDataSink struct { conn *grpc.ClientConn sendTaskCh chan sendTask - registered bool + registered *atomic.Bool registerer DataSinkRegisterer } @@ -53,7 +54,7 @@ func NewSingleTargetDataSink(registerer DataSinkRegisterer) *SingleTargetDataSin conn: nil, sendTaskCh: make(chan sendTask, 1), - registered: false, + registered: atomic.NewBool(false), registerer: registerer, } @@ -64,6 +65,7 @@ func NewSingleTargetDataSink(registerer DataSinkRegisterer) *SingleTargetDataSin logutil.BgLogger().Warn("failed to register single target datasink", zap.Error(err)) return nil } + dataSink.registered.Store(true) } go dataSink.recoverRun() @@ -111,25 +113,27 @@ func (ds *SingleTargetDataSink) run() (rerun bool) { targetRPCAddr = config.GetGlobalConfig().TopSQL.ReceiverAddress } - if err := ds.tryRegister(targetRPCAddr); err != nil { - logutil.BgLogger().Warn("failed to register the single target datasink", zap.Error(err)) + if err := ds.trySwitchRegistration(targetRPCAddr); err != nil { return false } } } -func (ds *SingleTargetDataSink) tryRegister(addr string) error { - if addr == "" && ds.registered { +func (ds *SingleTargetDataSink) trySwitchRegistration(addr string) error { + // deregister if `addr` is empty and registered before + if addr == "" && ds.registered.Load() { ds.registerer.Deregister(ds) - ds.registered = false + ds.registered.Store(false) return nil } - if addr != "" && !ds.registered { + // register if `add` is not empty and not registered before + if addr != "" && !ds.registered.Load() { if err := ds.registerer.Register(ds); err != nil { + logutil.BgLogger().Warn("failed to register the single target datasink", zap.Error(err)) return err } - ds.registered = true + ds.registered.Store(true) } return nil } @@ -160,8 +164,9 @@ func (ds *SingleTargetDataSink) OnReporterClosing() { func (ds *SingleTargetDataSink) Close() { ds.cancel() - if ds.registered { + if ds.registered.Load() { ds.registerer.Deregister(ds) + ds.registered.Store(false) } } @@ -172,12 +177,14 @@ func (ds *SingleTargetDataSink) doSend(addr string, task sendTask) { var err error start := time.Now() - if err != nil { - logutil.BgLogger().Warn("[top-sql] single target data sink failed to send data to receiver", zap.Error(err)) - reportAllDurationFailedHistogram.Observe(time.Since(start).Seconds()) - } else { - reportAllDurationSuccHistogram.Observe(time.Since(start).Seconds()) - } + defer func() { + if err != nil { + logutil.BgLogger().Warn("[top-sql] single target data sink failed to send data to receiver", zap.Error(err)) + reportAllDurationFailedHistogram.Observe(time.Since(start).Seconds()) + } else { + reportAllDurationSuccHistogram.Observe(time.Since(start).Seconds()) + } + }() ctx, cancel := context.WithDeadline(context.Background(), task.deadline) defer cancel() @@ -223,8 +230,9 @@ func (ds *SingleTargetDataSink) sendBatchCPUTimeRecord(ctx context.Context, reco topSQLReportRecordCounterHistogram.Observe(float64(sentCount)) if err != nil { reportRecordDurationFailedHistogram.Observe(time.Since(start).Seconds()) + } else { + reportRecordDurationSuccHistogram.Observe(time.Since(start).Seconds()) } - reportRecordDurationSuccHistogram.Observe(time.Since(start).Seconds()) }() client := tipb.NewTopSQLAgentClient(ds.conn) @@ -254,11 +262,11 @@ func (ds *SingleTargetDataSink) sendBatchSQLMeta(ctx context.Context, sqlMetas [ sentCount := 0 defer func() { topSQLReportSQLCountHistogram.Observe(float64(sentCount)) - if err != nil { reportSQLDurationFailedHistogram.Observe(time.Since(start).Seconds()) + } else { + reportSQLDurationSuccHistogram.Observe(time.Since(start).Seconds()) } - reportSQLDurationSuccHistogram.Observe(time.Since(start).Seconds()) }() client := tipb.NewTopSQLAgentClient(ds.conn) @@ -291,8 +299,9 @@ func (ds *SingleTargetDataSink) sendBatchPlanMeta(ctx context.Context, planMetas topSQLReportPlanCountHistogram.Observe(float64(sentCount)) if err != nil { reportPlanDurationFailedHistogram.Observe(time.Since(start).Seconds()) + } else { + reportPlanDurationSuccHistogram.Observe(time.Since(start).Seconds()) } - reportPlanDurationSuccHistogram.Observe(time.Since(start).Seconds()) }() client := tipb.NewTopSQLAgentClient(ds.conn) diff --git a/util/topsql/topsql.go b/util/topsql/topsql.go index 12a9da430e7c0..ccc1f61eef0a5 100644 --- a/util/topsql/topsql.go +++ b/util/topsql/topsql.go @@ -27,7 +27,9 @@ import ( "github.com/pingcap/tidb/util/topsql/reporter" "github.com/pingcap/tidb/util/topsql/stmtstats" "github.com/pingcap/tidb/util/topsql/tracecpu" + "github.com/pingcap/tipb/go-tipb" "go.uber.org/zap" + "google.golang.org/grpc" ) const ( @@ -38,7 +40,7 @@ const ( ) var ( - globalTopSQLReport reporter.TopSQLReporter + globalTopSQLReport *reporter.RemoteTopSQLReporter singleTargetDataSink *reporter.SingleTargetDataSink ) @@ -54,6 +56,14 @@ func SetupTopSQL() { stmtstats.SetupAggregator() } +// RegisterPubSubServer registers TopSQLPubSubService to the given gRPC server. +func RegisterPubSubServer(s *grpc.Server) { + if globalTopSQLReport != nil { + service := reporter.NewTopSQLPubSubService(globalTopSQLReport) + tipb.RegisterTopSQLPubSubServer(s, service) + } +} + // Close uses to close and release the top sql resource. func Close() { if singleTargetDataSink != nil { diff --git a/util/topsql/topsql_test.go b/util/topsql/topsql_test.go index d4aabc746ce0e..462c6ffb70aa2 100644 --- a/util/topsql/topsql_test.go +++ b/util/topsql/topsql_test.go @@ -29,7 +29,10 @@ import ( mockServer "github.com/pingcap/tidb/util/topsql/reporter/mock" "github.com/pingcap/tidb/util/topsql/tracecpu" "github.com/pingcap/tidb/util/topsql/tracecpu/mock" + "github.com/pingcap/tipb/go-tipb" "github.com/stretchr/testify/require" + "google.golang.org/grpc" + "google.golang.org/grpc/keepalive" ) type collectorWrapper struct { @@ -213,6 +216,163 @@ func TestMaxSQLAndPlanTest(t *testing.T) { require.Empty(t, cPlan) } +func TestTopSQLPubSub(t *testing.T) { + variable.TopSQLVariable.MaxStatementCount.Store(200) + variable.TopSQLVariable.ReportIntervalSeconds.Store(1) + + report := reporter.NewRemoteTopSQLReporter(mockPlanBinaryDecoderFunc) + defer report.Close() + tracecpu.GlobalSQLCPUProfiler.SetCollector(&collectorWrapper{report}) + + server, err := mockServer.NewMockPubSubServer() + require.NoError(t, err) + pubsubService := reporter.NewTopSQLPubSubService(report) + tipb.RegisterTopSQLPubSubServer(server.Server(), pubsubService) + go server.Serve() + defer server.Stop() + + conn, err := grpc.Dial( + server.Address(), + grpc.WithBlock(), + grpc.WithInsecure(), + grpc.WithKeepaliveParams(keepalive.ClientParameters{ + Time: 10 * time.Second, + Timeout: 3 * time.Second, + }), + ) + require.NoError(t, err) + defer conn.Close() + + ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) + defer cancel() + client := tipb.NewTopSQLPubSubClient(conn) + stream, err := client.Subscribe(ctx, &tipb.TopSQLSubRequest{}) + require.NoError(t, err) + + reqs := []struct { + sql string + plan string + }{ + {"select * from t where a=?", "point-get"}, + {"select * from t where a>?", "table-scan"}, + {"insert into t values (?)", ""}, + } + + digest2sql := make(map[string]string) + sql2plan := make(map[string]string) + for _, req := range reqs { + sql2plan[req.sql] = req.plan + sqlDigest := mock.GenSQLDigest(req.sql) + digest2sql[string(sqlDigest.Bytes())] = req.sql + + go func(sql, plan string) { + for { + select { + case <-ctx.Done(): + return + default: + mockExecuteSQL(sql, plan) + } + } + }(req.sql, req.plan) + } + + sqlMetas := make(map[string]*tipb.SQLMeta) + planMetas := make(map[string]string) + records := make(map[string]*tipb.CPUTimeRecord) + + for { + r, err := stream.Recv() + if err != nil { + break + } + + if r.GetRecord() != nil { + rec := r.GetRecord() + if _, ok := records[string(rec.SqlDigest)]; !ok { + records[string(rec.SqlDigest)] = rec + } else { + cpu := records[string(rec.SqlDigest)] + if rec.PlanDigest != nil { + cpu.PlanDigest = rec.PlanDigest + } + cpu.RecordListTimestampSec = append(cpu.RecordListTimestampSec, rec.RecordListTimestampSec...) + cpu.RecordListCpuTimeMs = append(cpu.RecordListCpuTimeMs, rec.RecordListCpuTimeMs...) + } + } else if r.GetSqlMeta() != nil { + sql := r.GetSqlMeta() + if _, ok := sqlMetas[string(sql.SqlDigest)]; !ok { + sqlMetas[string(sql.SqlDigest)] = sql + } + } else if r.GetPlanMeta() != nil { + plan := r.GetPlanMeta() + if _, ok := planMetas[string(plan.PlanDigest)]; !ok { + planMetas[string(plan.PlanDigest)] = plan.NormalizedPlan + } + } + } + + checkSQLPlanMap := map[string]struct{}{} + for i := range records { + record := records[i] + require.Greater(t, len(record.RecordListCpuTimeMs), 0) + require.Greater(t, record.RecordListCpuTimeMs[0], uint32(0)) + sqlMeta, exist := sqlMetas[string(record.SqlDigest)] + require.True(t, exist) + expectedNormalizedSQL, exist := digest2sql[string(record.SqlDigest)] + require.True(t, exist) + require.Equal(t, expectedNormalizedSQL, sqlMeta.NormalizedSql) + + expectedNormalizedPlan := sql2plan[expectedNormalizedSQL] + if expectedNormalizedPlan == "" || len(record.PlanDigest) == 0 { + require.Equal(t, len(record.PlanDigest), 0) + continue + } + normalizedPlan, exist := planMetas[string(record.PlanDigest)] + require.True(t, exist) + require.Equal(t, expectedNormalizedPlan, normalizedPlan) + checkSQLPlanMap[expectedNormalizedSQL] = struct{}{} + } + require.Equal(t, len(checkSQLPlanMap), 2) +} + +func TestPubSubWhenReporterIsStopped(t *testing.T) { + report := reporter.NewRemoteTopSQLReporter(mockPlanBinaryDecoderFunc) + + server, err := mockServer.NewMockPubSubServer() + require.NoError(t, err) + + pubsubService := reporter.NewTopSQLPubSubService(report) + tipb.RegisterTopSQLPubSubServer(server.Server(), pubsubService) + go server.Serve() + defer server.Stop() + + // stop reporter first + report.Close() + + // try to subscribe + conn, err := grpc.Dial( + server.Address(), + grpc.WithBlock(), + grpc.WithInsecure(), + grpc.WithKeepaliveParams(keepalive.ClientParameters{ + Time: 10 * time.Second, + Timeout: 3 * time.Second, + }), + ) + require.NoError(t, err) + defer conn.Close() + + ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) + defer cancel() + client := tipb.NewTopSQLPubSubClient(conn) + stream, err := client.Subscribe(ctx, &tipb.TopSQLSubRequest{}) + require.NoError(t, err) + + _, err = stream.Recv() + require.Error(t, err, "reporter is closed") +} + func setTopSQLEnable(enabled bool) { variable.TopSQLVariable.Enable.Store(enabled) } From 2b7ce8e61ffb565c7a1fde86f7fd767210793526 Mon Sep 17 00:00:00 2001 From: cfzjywxk Date: Tue, 21 Dec 2021 16:57:46 +0800 Subject: [PATCH 06/44] executor: fix the incorrect untouch used in optimistic transactions (#30447) close pingcap/tidb#30410 --- go.mod | 2 +- go.sum | 4 ++-- session/session.go | 8 +++++--- store/driver/txn/txn_driver.go | 15 +++++++++++++-- table/tables/index.go | 18 ++++++++++++++++-- table/tables/tables_test.go | 26 ++++++++++++++++++++++++++ 6 files changed, 63 insertions(+), 10 deletions(-) diff --git a/go.mod b/go.mod index 6f2a22ed5dd5d..f11eab1cf0456 100644 --- a/go.mod +++ b/go.mod @@ -65,7 +65,7 @@ require ( github.com/spf13/pflag v1.0.5 github.com/stretchr/testify v1.7.0 github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2 - github.com/tikv/client-go/v2 v2.0.0-rc.0.20211218050306-6165dbaa95d0 + github.com/tikv/client-go/v2 v2.0.0-rc.0.20211221041211-e9de5625c45c github.com/tikv/pd v1.1.0-beta.0.20211118054146-02848d2660ee github.com/twmb/murmur3 v1.1.3 github.com/uber/jaeger-client-go v2.22.1+incompatible diff --git a/go.sum b/go.sum index 36c8604668b6b..2a4c23d9d70bc 100644 --- a/go.sum +++ b/go.sum @@ -712,8 +712,8 @@ github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2/go.mod h1:2PfK github.com/tidwall/gjson v1.3.5/go.mod h1:P256ACg0Mn+j1RXIDXoss50DeIABTYK1PULOJHhxOls= github.com/tidwall/match v1.0.1/go.mod h1:LujAq0jyVjBy028G1WhWfIzbpQfMO8bBZ6Tyb0+pL9E= github.com/tidwall/pretty v1.0.0/go.mod h1:XNkn88O1ChpSDQmQeStsy+sBenx6DDtFZJxhVysOjyk= -github.com/tikv/client-go/v2 v2.0.0-rc.0.20211218050306-6165dbaa95d0 h1:38Jst/O36MKXAt7aD1Ipnx4nKwclG66ifkcmi4f0NZ4= -github.com/tikv/client-go/v2 v2.0.0-rc.0.20211218050306-6165dbaa95d0/go.mod h1:wRuh+W35daKTiYBld0oBlT6PSkzEVr+pB/vChzJZk+8= +github.com/tikv/client-go/v2 v2.0.0-rc.0.20211221041211-e9de5625c45c h1:1P6iN1csRSZNHXuaylArmG3/bA5MpYVzc9ZkdHK/L2Y= +github.com/tikv/client-go/v2 v2.0.0-rc.0.20211221041211-e9de5625c45c/go.mod h1:wRuh+W35daKTiYBld0oBlT6PSkzEVr+pB/vChzJZk+8= github.com/tikv/pd v1.1.0-beta.0.20211029083450-e65f0c55b6ae/go.mod h1:varH0IE0jJ9E9WN2Ei/N6pajMlPkcXdDEf7f5mmsUVQ= github.com/tikv/pd v1.1.0-beta.0.20211118054146-02848d2660ee h1:rAAdvQ8Hh36syHr92g0VmZEpkH+40RGQBpFL2121xMs= github.com/tikv/pd v1.1.0-beta.0.20211118054146-02848d2660ee/go.mod h1:lRbwxBAhnTQR5vqbTzeI/Bj62bD2OvYYuFezo2vrmeI= diff --git a/session/session.go b/session/session.go index b7171a79c9525..561120ff4b9cc 100644 --- a/session/session.go +++ b/session/session.go @@ -44,6 +44,7 @@ import ( "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/parser/terror" + "github.com/pingcap/tidb/store/driver/txn" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/table/temptable" "github.com/pingcap/tidb/util/topsql" @@ -753,14 +754,15 @@ func (s *session) commitTxnWithTemporaryData(ctx context.Context, txn kv.Transac type temporaryTableKVFilter map[int64]tableutil.TempTable -func (m temporaryTableKVFilter) IsUnnecessaryKeyValue(key, value []byte, flags tikvstore.KeyFlags) bool { +func (m temporaryTableKVFilter) IsUnnecessaryKeyValue(key, value []byte, flags tikvstore.KeyFlags) (bool, error) { tid := tablecodec.DecodeTableID(key) if _, ok := m[tid]; ok { - return true + return true, nil } // This is the default filter for all tables. - return tablecodec.IsUntouchedIndexKValue(key, value) + defaultFilter := txn.TiDBKVFilter{} + return defaultFilter.IsUnnecessaryKeyValue(key, value, flags) } // errIsNoisy is used to filter DUPLCATE KEY errors. diff --git a/store/driver/txn/txn_driver.go b/store/driver/txn/txn_driver.go index bb9e38a4f3c03..730dba0c3c7fb 100644 --- a/store/driver/txn/txn_driver.go +++ b/store/driver/txn/txn_driver.go @@ -28,12 +28,14 @@ import ( derr "github.com/pingcap/tidb/store/driver/error" "github.com/pingcap/tidb/store/driver/options" "github.com/pingcap/tidb/tablecodec" + "github.com/pingcap/tidb/util/logutil" tikverr "github.com/tikv/client-go/v2/error" tikvstore "github.com/tikv/client-go/v2/kv" "github.com/tikv/client-go/v2/tikv" "github.com/tikv/client-go/v2/tikvrpc" "github.com/tikv/client-go/v2/tikvrpc/interceptor" "github.com/tikv/client-go/v2/txnkv/txnsnapshot" + "go.uber.org/zap" ) type tikvTxn struct { @@ -292,6 +294,15 @@ func (txn *tikvTxn) extractKeyExistsErr(key kv.Key) error { type TiDBKVFilter struct{} // IsUnnecessaryKeyValue defines which kinds of KV pairs from TiDB needn't be committed. -func (f TiDBKVFilter) IsUnnecessaryKeyValue(key, value []byte, flags tikvstore.KeyFlags) bool { - return tablecodec.IsUntouchedIndexKValue(key, value) +func (f TiDBKVFilter) IsUnnecessaryKeyValue(key, value []byte, flags tikvstore.KeyFlags) (bool, error) { + isUntouchedValue := tablecodec.IsUntouchedIndexKValue(key, value) + if isUntouchedValue && flags.HasPresumeKeyNotExists() { + logutil.BgLogger().Error("unexpected path the untouched key value with PresumeKeyNotExists flag", + zap.Stringer("key", kv.Key(key)), zap.Stringer("value", kv.Key(value)), + zap.Uint16("flags", uint16(flags)), zap.Stack("stack")) + return false, errors.Errorf( + "unexpected path the untouched key=%s value=%s contains PresumeKeyNotExists flag keyFlags=%v", + kv.Key(key).String(), kv.Key(value).String(), flags) + } + return isUntouchedValue, nil } diff --git a/table/tables/index.go b/table/tables/index.go index 08d3ecef1f820..8350925fe15b4 100644 --- a/table/tables/index.go +++ b/table/tables/index.go @@ -169,8 +169,22 @@ func (c *index) Create(sctx sessionctx.Context, txn kv.Transaction, indexedValue // should not overwrite the key with un-commit flag. // So if the key exists, just do nothing and return. v, err := txn.GetMemBuffer().Get(ctx, key) - if err == nil && len(v) != 0 { - return nil, nil + if err == nil { + if len(v) != 0 { + return nil, nil + } + // The key is marked as deleted in the memory buffer, as the existence check is done lazily + // for optimistic transactions by default. The "untouched" key could still exist in the store, + // it's needed to commit this key to do the existence check so unset the untouched flag. + if !txn.IsPessimistic() { + keyFlags, err := txn.GetMemBuffer().GetFlags(key) + if err != nil { + return nil, err + } + if keyFlags.HasPresumeKeyNotExists() { + opt.Untouched = false + } + } } } diff --git a/table/tables/tables_test.go b/table/tables/tables_test.go index b093e96c6be38..dcf3507d92c49 100644 --- a/table/tables/tables_test.go +++ b/table/tables/tables_test.go @@ -736,3 +736,29 @@ func TestViewColumns(t *testing.T) { "Warning|1356|View 'test.va' references invalid table(s) or column(s) or function(s) or definer/invoker of view lack rights to use them")) } } + +func TestConstraintCheckForOptimisticUntouched(t *testing.T) { + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists test_optimistic_untouched_flag;") + tk.MustExec(`create table test_optimistic_untouched_flag(c0 int, c1 varchar(20), c2 varchar(20), unique key uk(c0));`) + tk.MustExec(`insert into test_optimistic_untouched_flag(c0, c1, c2) values (1, null, 'green');`) + + // Insert a row with duplicated entry on the unique key, the commit should fail. + tk.MustExec("begin optimistic;") + tk.MustExec(`insert into test_optimistic_untouched_flag(c0, c1, c2) values (1, 'red', 'white');`) + tk.MustExec(`delete from test_optimistic_untouched_flag where c1 is null;`) + tk.MustExec("update test_optimistic_untouched_flag set c2 = 'green' where c2 between 'purple' and 'white';") + err := tk.ExecToErr("commit") + require.Error(t, err) + + tk.MustExec("begin optimistic;") + tk.MustExec(`insert into test_optimistic_untouched_flag(c0, c1, c2) values (1, 'red', 'white');`) + tk.MustExec("update test_optimistic_untouched_flag set c2 = 'green' where c2 between 'purple' and 'white';") + err = tk.ExecToErr("commit") + require.Error(t, err) +} From 55a38cb1627b2f5476e5b6fe0459d7b9a5706fea Mon Sep 17 00:00:00 2001 From: Hangjie Mo Date: Tue, 21 Dec 2021 17:27:47 +0800 Subject: [PATCH 07/44] expression, cmd: let crc32() support gbk (#30900) close pingcap/tidb#30898 --- .../r/new_character_set_builtin.result | 11 +++++ .../t/new_character_set_builtin.test | 9 ++++ expression/builtin_convert_charset.go | 2 + expression/builtin_math_test.go | 43 +++++++++++-------- 4 files changed, 48 insertions(+), 17 deletions(-) diff --git a/cmd/explaintest/r/new_character_set_builtin.result b/cmd/explaintest/r/new_character_set_builtin.result index f587a5ac1370e..74673a02b0832 100644 --- a/cmd/explaintest/r/new_character_set_builtin.result +++ b/cmd/explaintest/r/new_character_set_builtin.result @@ -523,3 +523,14 @@ select hex(aes_encrypt(a, '123')), hex(aes_encrypt(b, '123')), hex(aes_encrypt(c hex(aes_encrypt(a, '123')) hex(aes_encrypt(b, '123')) hex(aes_encrypt(c, '123')) C54279F381B0710E145E94106F03C94C 7A747EC6F1906276D036B1F3CE27BAAB A0E5E01289017B8A3691CCFBDE81A59ED4A9D5BF50A298D41287E395CDDCAD56 set @@tidb_enable_vectorized_expression = false; +drop table if exists t; +create table t (a char(20) charset utf8mb4, b char(20) charset gbk, c binary(20)); +insert into t values ('一二三', '一二三', '一二三'); +select crc32(a), crc32(b), crc32(c) from t; +crc32(a) crc32(b) crc32(c) +1785250883 3461331449 4092198678 +set @@tidb_enable_vectorized_expression = true; +select crc32(a), crc32(b), crc32(c) from t; +crc32(a) crc32(b) crc32(c) +1785250883 3461331449 4092198678 +set @@tidb_enable_vectorized_expression = false; diff --git a/cmd/explaintest/t/new_character_set_builtin.test b/cmd/explaintest/t/new_character_set_builtin.test index bb0a6321e8a53..ae9ab76f9093e 100644 --- a/cmd/explaintest/t/new_character_set_builtin.test +++ b/cmd/explaintest/t/new_character_set_builtin.test @@ -239,3 +239,12 @@ select hex(aes_encrypt(a, '123', '1234567890123456')), hex(aes_encrypt(b, '123', set @@block_encryption_mode='aes-128-ecb'; select hex(aes_encrypt(a, '123')), hex(aes_encrypt(b, '123')), hex(aes_encrypt(c, '123')) from t; set @@tidb_enable_vectorized_expression = false; + +-- test for builtin crc32() +drop table if exists t; +create table t (a char(20) charset utf8mb4, b char(20) charset gbk, c binary(20)); +insert into t values ('一二三', '一二三', '一二三'); +select crc32(a), crc32(b), crc32(c) from t; +set @@tidb_enable_vectorized_expression = true; +select crc32(a), crc32(b), crc32(c) from t; +set @@tidb_enable_vectorized_expression = false; diff --git a/expression/builtin_convert_charset.go b/expression/builtin_convert_charset.go index 5c0ea98f752a5..bc7ad4dfafbbe 100644 --- a/expression/builtin_convert_charset.go +++ b/expression/builtin_convert_charset.go @@ -287,6 +287,8 @@ var convertActionMap = map[funcProp][]string{ ast.Like, ast.Strcmp, /* regex */ ast.Regexp, + /* math */ + ast.CRC32, }, } diff --git a/expression/builtin_math_test.go b/expression/builtin_math_test.go index 898693d14b394..d1ae3ca6ed30c 100644 --- a/expression/builtin_math_test.go +++ b/expression/builtin_math_test.go @@ -23,6 +23,7 @@ import ( "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/charset" "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/testkit/trequire" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" @@ -531,27 +532,35 @@ func TestTruncate(t *testing.T) { func TestCRC32(t *testing.T) { ctx := createContext(t) tbl := []struct { - Arg []interface{} - Ret interface{} + input []interface{} + chs string + result int64 + isNull bool }{ - {[]interface{}{nil}, nil}, - {[]interface{}{""}, 0}, - {[]interface{}{-1}, 808273962}, - {[]interface{}{"-1"}, 808273962}, - {[]interface{}{"mysql"}, 2501908538}, - {[]interface{}{"MySQL"}, 3259397556}, - {[]interface{}{"hello"}, 907060870}, + {[]interface{}{nil}, "utf8", 0, true}, + {[]interface{}{""}, "utf8", 0, false}, + {[]interface{}{-1}, "utf8", 808273962, false}, + {[]interface{}{"-1"}, "utf8", 808273962, false}, + {[]interface{}{"mysql"}, "utf8", 2501908538, false}, + {[]interface{}{"MySQL"}, "utf8", 3259397556, false}, + {[]interface{}{"hello"}, "utf8", 907060870, false}, + {[]interface{}{"一二三"}, "utf8", 1785250883, false}, + {[]interface{}{"一"}, "utf8", 2416838398, false}, + {[]interface{}{"一二三"}, "gbk", 3461331449, false}, + {[]interface{}{"一"}, "gbk", 2925846374, false}, } - - Dtbl := tblToDtbl(tbl) - - for _, tt := range Dtbl { - fc := funcs[ast.CRC32] - f, err := fc.getFunction(ctx, datumsToConstants(tt["Arg"])) + for _, c := range tbl { + err := ctx.GetSessionVars().SetSystemVar(variable.CharacterSetConnection, c.chs) require.NoError(t, err) - v, err := evalBuiltinFunc(f, chunk.Row{}) + f, err := newFunctionForTest(ctx, ast.CRC32, primitiveValsToConstants(ctx, c.input)...) require.NoError(t, err) - trequire.DatumEqual(t, tt["Ret"][0], v) + d, err := f.Eval(chunk.Row{}) + require.NoError(t, err) + if c.isNull { + require.True(t, d.IsNull()) + } else { + require.Equal(t, c.result, d.GetInt64()) + } } } From feee7c258ac7799499f561d281434b243c4fc92e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20van=20Eeden?= Date: Tue, 21 Dec 2021 10:53:48 +0100 Subject: [PATCH 08/44] server: Add uptime status var and statistics (#29790) close pingcap/tidb#8842 --- domain/infosync/info.go | 2 +- domain/infosync/info_test.go | 4 +-- server/conn.go | 15 +++++++-- server/stat.go | 14 +++++++++ server/stat_test.go | 61 ++++++++++++++++++++++++++++++++++++ 5 files changed, 91 insertions(+), 5 deletions(-) create mode 100644 server/stat_test.go diff --git a/domain/infosync/info.go b/domain/infosync/info.go index fc58783ff3108..8aec3c35275e8 100644 --- a/domain/infosync/info.go +++ b/domain/infosync/info.go @@ -856,7 +856,7 @@ func getServerInfo(id string, serverIDGetter func() uint64) *ServerInfo { failpoint.Inject("mockServerInfo", func(val failpoint.Value) { if val.(bool) { - info.StartTimestamp = 1282967700000 + info.StartTimestamp = 1282967700 info.Labels = map[string]string{ "foo": "bar", } diff --git a/domain/infosync/info_test.go b/domain/infosync/info_test.go index 001a106632230..bb0ded60e7e25 100644 --- a/domain/infosync/info_test.go +++ b/domain/infosync/info_test.go @@ -72,7 +72,7 @@ func TestTopology(t *testing.T) { topology, err := info.getTopologyFromEtcd(ctx) require.NoError(t, err) - require.Equal(t, int64(1282967700000), topology.StartTimestamp) + require.Equal(t, int64(1282967700), topology.StartTimestamp) v, ok := topology.Labels["foo"] require.True(t, ok) @@ -97,7 +97,7 @@ func TestTopology(t *testing.T) { dir := path.Dir(s) require.Equal(t, dir, topology.DeployPath) - require.Equal(t, int64(1282967700000), topology.StartTimestamp) + require.Equal(t, int64(1282967700), topology.StartTimestamp) require.Equal(t, info.getTopologyInfo(), *topology) // check ttl key diff --git a/server/conn.go b/server/conn.go index e9325a6dd97d6..ede5a12bf276d 100644 --- a/server/conn.go +++ b/server/conn.go @@ -59,6 +59,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/domain/infosync" "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/infoschema" @@ -1374,11 +1375,21 @@ func (cc *clientConn) dispatch(ctx context.Context, data []byte) error { } func (cc *clientConn) writeStats(ctx context.Context) error { - msg := []byte("Uptime: 0 Threads: 0 Questions: 0 Slow queries: 0 Opens: 0 Flush tables: 0 Open tables: 0 Queries per second avg: 0.000") + var err error + var uptime int64 = 0 + info := serverInfo{} + info.ServerInfo, err = infosync.GetServerInfo() + if err != nil { + logutil.BgLogger().Error("Failed to get ServerInfo for uptime status", zap.Error(err)) + } else { + uptime = int64(time.Since(time.Unix(info.ServerInfo.StartTimestamp, 0)).Seconds()) + } + msg := []byte(fmt.Sprintf("Uptime: %d Threads: 0 Questions: 0 Slow queries: 0 Opens: 0 Flush tables: 0 Open tables: 0 Queries per second avg: 0.000", + uptime)) data := cc.alloc.AllocWithLen(4, len(msg)) data = append(data, msg...) - err := cc.writePacket(data) + err = cc.writePacket(data) if err != nil { return err } diff --git a/server/stat.go b/server/stat.go index 9725a7ec5e480..382a68e701ce0 100644 --- a/server/stat.go +++ b/server/stat.go @@ -16,7 +16,9 @@ package server import ( "crypto/x509" + "time" + "github.com/pingcap/tidb/domain/infosync" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/util/logutil" "go.uber.org/zap" @@ -25,11 +27,13 @@ import ( var ( serverNotAfter = "Ssl_server_not_after" serverNotBefore = "Ssl_server_not_before" + upTime = "Uptime" ) var defaultStatus = map[string]*variable.StatusVal{ serverNotAfter: {Scope: variable.ScopeGlobal | variable.ScopeSession, Value: ""}, serverNotBefore: {Scope: variable.ScopeGlobal | variable.ScopeSession, Value: ""}, + upTime: {Scope: variable.ScopeGlobal, Value: 0}, } // GetScope gets the status variables scope. @@ -57,5 +61,15 @@ func (s *Server) Stats(vars *variable.SessionVars) (map[string]interface{}, erro } } } + + var err error + info := serverInfo{} + info.ServerInfo, err = infosync.GetServerInfo() + if err != nil { + logutil.BgLogger().Error("Failed to get ServerInfo for uptime status", zap.Error(err)) + } else { + m[upTime] = int64(time.Since(time.Unix(info.ServerInfo.StartTimestamp, 0)).Seconds()) + } + return m, nil } diff --git a/server/stat_test.go b/server/stat_test.go new file mode 100644 index 0000000000000..88fc2e0081a43 --- /dev/null +++ b/server/stat_test.go @@ -0,0 +1,61 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package server + +import ( + "context" + "testing" + "time" + + "github.com/pingcap/failpoint" + "github.com/pingcap/tidb/domain/infosync" + "github.com/pingcap/tidb/session" + "github.com/pingcap/tidb/store/mockstore" + "github.com/stretchr/testify/require" +) + +func TestUptime(t *testing.T) { + var err error + + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/domain/infosync/mockServerInfo", "return(true)")) + defer func() { + err := failpoint.Disable("github.com/pingcap/tidb/domain/infosync/mockServerInfo") + require.NoError(t, err) + }() + + store, err := mockstore.NewMockStore() + require.NoError(t, err) + + dom, err := session.BootstrapSession(store) + defer func() { + dom.Close() + err := store.Close() + require.NoError(t, err) + }() + require.NoError(t, err) + + _, err = infosync.GlobalInfoSyncerInit(context.Background(), dom.DDL().GetID(), dom.ServerID, dom.GetEtcdClient(), true) + require.NoError(t, err) + + tidbdrv := NewTiDBDriver(store) + cfg := newTestConfig() + cfg.Socket = "" + server, err := NewServer(cfg, tidbdrv) + require.NoError(t, err) + + stats, err := server.Stats(nil) + require.NoError(t, err) + require.GreaterOrEqual(t, stats[upTime].(int64), int64(time.Since(time.Unix(1282967700, 0)).Seconds())) +} From e12342b49446913ce392ef9fb3172f0d008e45a1 Mon Sep 17 00:00:00 2001 From: fengou1 <85682690+fengou1@users.noreply.github.com> Date: Tue, 21 Dec 2021 18:09:46 +0800 Subject: [PATCH 09/44] br: error log optimization (#29640) close pingcap/tidb#27015 --- br/pkg/backup/client.go | 8 +++++++- br/pkg/summary/collector.go | 11 ++++++++++- 2 files changed, 17 insertions(+), 2 deletions(-) diff --git a/br/pkg/backup/client.go b/br/pkg/backup/client.go index 12a4344a432fe..7a4b1e0e8eb66 100644 --- a/br/pkg/backup/client.go +++ b/br/pkg/backup/client.go @@ -470,7 +470,13 @@ func (bc *Client) BackupRanges( elctx := logutil.ContextWithField(ectx, logutil.RedactAny("range-sn", id)) err := bc.BackupRange(elctx, sk, ek, req, metaWriter, progressCallBack) if err != nil { - return errors.Trace(err) + // The error due to context cancel, stack trace is meaningless, the stack shall be suspended (also clear) + if errors.Cause(err) == context.Canceled { + return errors.SuspendStack(err) + } else { + return errors.Trace(err) + } + } return nil }) diff --git a/br/pkg/summary/collector.go b/br/pkg/summary/collector.go index 5493f82f77967..6c82bf54fba25 100644 --- a/br/pkg/summary/collector.go +++ b/br/pkg/summary/collector.go @@ -3,11 +3,13 @@ package summary import ( + "context" "strings" "sync" "time" "github.com/docker/go-units" + berror "github.com/pingcap/errors" "github.com/pingcap/log" "go.uber.org/zap" ) @@ -188,9 +190,16 @@ func (tc *logCollector) Summary(name string) { } if len(tc.failureReasons) != 0 || !tc.successStatus { + var canceledUnits int for unitName, reason := range tc.failureReasons { - logFields = append(logFields, zap.String("unit-name", unitName), zap.Error(reason)) + if berror.Cause(reason) != context.Canceled { + logFields = append(logFields, zap.String("unit-name", unitName), zap.Error(reason)) + } else { + canceledUnits++ + } } + // only print total number of cancel unit + log.Info("units canceled", zap.Int("cancel-unit", canceledUnits)) tc.log(name+" failed summary", logFields...) return } From 416617eb9cf633bfc964370ea19cc4fc949fa71b Mon Sep 17 00:00:00 2001 From: wjHuang Date: Tue, 21 Dec 2021 20:47:46 +0800 Subject: [PATCH 10/44] planner: fix wrong collation when rewrite in condition (#30492) close pingcap/tidb#30486 --- expression/integration_serial_test.go | 10 +++++ planner/core/expression_rewriter.go | 60 +++++++++++++++++++++++++++ 2 files changed, 70 insertions(+) diff --git a/expression/integration_serial_test.go b/expression/integration_serial_test.go index 0665f2b2082ba..3077e2f1b33a7 100644 --- a/expression/integration_serial_test.go +++ b/expression/integration_serial_test.go @@ -175,6 +175,16 @@ func TestCollationBasic(t *testing.T) { tk.MustQuery("select * from t1 where col1 >= 0xc484 and col1 <= 0xc3b3;").Check(testkit.Rows("Ȇ")) tk.MustQuery("select collation(IF('a' < 'B' collate utf8mb4_general_ci, 'smaller', 'greater' collate utf8mb4_unicode_ci));").Check(testkit.Rows("utf8mb4_unicode_ci")) + + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a char(10))") + tk.MustExec("insert into t values ('a')") + tk.MustQuery("select * from t where a in ('b' collate utf8mb4_general_ci, 'A', 3)").Check(testkit.Rows("a")) + // These test cases may not the same as MySQL, but it's more reasonable. + tk.MustQuery("select ('a', 'a') in (('A' collate utf8mb4_general_ci, 'A' collate utf8mb4_general_ci));").Check(testkit.Rows("1")) + tk.MustQuery("select ('a', 'a') in (('A' collate utf8mb4_general_ci, 'A' collate utf8mb4_bin));").Check(testkit.Rows("0")) + tk.MustQuery("select ('a', 'a') in (('A' collate utf8mb4_general_ci, 'A' collate utf8mb4_general_ci), ('b', 'b'));").Check(testkit.Rows("1")) + tk.MustQuery("select ('a', 'a') in (('A' collate utf8mb4_general_ci, 'A' collate utf8mb4_bin), ('b', 'b'));").Check(testkit.Rows("0")) } func TestWeightString(t *testing.T) { diff --git a/planner/core/expression_rewriter.go b/planner/core/expression_rewriter.go index 7dcadc13f4005..08e0262613cb9 100644 --- a/planner/core/expression_rewriter.go +++ b/planner/core/expression_rewriter.go @@ -1492,6 +1492,12 @@ func (er *expressionRewriter) inToExpression(lLen int, not bool, tp *types.Field if allSameType && l == 1 && lLen > 1 { function = er.notToExpression(not, ast.In, tp, er.ctxStack[stkLen-lLen-1:]...) } else { + // If we rewrite IN to EQ, we need to decide what's the collation EQ uses. + coll := er.deriveCollationForIn(l, lLen, stkLen, args) + if er.err != nil { + return + } + er.castCollationForIn(l, lLen, stkLen, coll) eqFunctions := make([]expression.Expression, 0, lLen) for i := stkLen - lLen; i < stkLen; i++ { expr, err := er.constructBinaryOpFunction(args[0], er.ctxStack[i], ast.EQ) @@ -1515,6 +1521,60 @@ func (er *expressionRewriter) inToExpression(lLen int, not bool, tp *types.Field er.ctxStackAppend(function, types.EmptyName) } +// deriveCollationForIn derives collation for in expression. +func (er *expressionRewriter) deriveCollationForIn(colLen int, elemCnt int, stkLen int, args []expression.Expression) []*expression.ExprCollation { + coll := make([]*expression.ExprCollation, 0, colLen) + if colLen == 1 { + // a in (x, y, z) => coll[0] + coll2, err := expression.CheckAndDeriveCollationFromExprs(er.sctx, "IN", types.ETInt, args...) + er.err = err + if er.err != nil { + return nil + } + coll = append(coll, coll2) + } else { + // (a, b, c) in ((x1, x2, x3), (y1, y2, y3), (z1, z2, z3)) => coll[0], coll[1], coll[2] + for i := 0; i < colLen; i++ { + args := make([]expression.Expression, 0, elemCnt) + for j := stkLen - elemCnt - 1; j < stkLen; j++ { + rowFunc, _ := er.ctxStack[j].(*expression.ScalarFunction) + args = append(args, rowFunc.GetArgs()[i]) + } + coll2, err := expression.CheckAndDeriveCollationFromExprs(er.sctx, "IN", types.ETInt, args...) + er.err = err + if er.err != nil { + return nil + } + coll = append(coll, coll2) + } + } + return coll +} + +// castCollationForIn casts collation info for arguments in the `in clause` to make sure the used collation is correct after we +// rewrite it to equal expression. +func (er *expressionRewriter) castCollationForIn(colLen int, elemCnt int, stkLen int, coll []*expression.ExprCollation) { + for i := stkLen - elemCnt; i < stkLen; i++ { + if colLen == 1 && er.ctxStack[i].GetType().EvalType() == types.ETString { + tp := er.ctxStack[i].GetType().Clone() + tp.Charset, tp.Collate = coll[0].Charset, coll[0].Collation + er.ctxStack[i] = expression.BuildCastFunction(er.sctx, er.ctxStack[i], tp) + er.ctxStack[i].SetCoercibility(expression.CoercibilityExplicit) + } else { + rowFunc, _ := er.ctxStack[i].(*expression.ScalarFunction) + for j := 0; j < colLen; j++ { + if er.ctxStack[i].GetType().EvalType() != types.ETString { + continue + } + tp := rowFunc.GetArgs()[j].GetType().Clone() + tp.Charset, tp.Collate = coll[j].Charset, coll[j].Collation + rowFunc.GetArgs()[j] = expression.BuildCastFunction(er.sctx, rowFunc.GetArgs()[j], tp) + rowFunc.GetArgs()[j].SetCoercibility(expression.CoercibilityExplicit) + } + } + } +} + func (er *expressionRewriter) caseToExpression(v *ast.CaseExpr) { stkLen := len(er.ctxStack) argsLen := 2 * len(v.WhenClauses) From 63d23f8aaa3fb5f748325a6debf023d2470b225a Mon Sep 17 00:00:00 2001 From: db <39407623+IcePigZDB@users.noreply.github.com> Date: Tue, 21 Dec 2021 22:49:46 +0800 Subject: [PATCH 11/44] planner: add extractor for tikv_region_peers (#30656) --- planner/core/logical_plan_builder.go | 2 + planner/core/memtable_predicate_extractor.go | 53 +++++++ .../core/memtable_predicate_extractor_test.go | 132 ++++++++++++++++++ 3 files changed, 187 insertions(+) diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index 5df63d975950a..bd7ae44d36cdf 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -4348,6 +4348,8 @@ func (b *PlanBuilder) buildMemTable(_ context.Context, dbName model.CIStr, table p.Extractor = &TiFlashSystemTableExtractor{} case infoschema.TableStatementsSummary, infoschema.TableStatementsSummaryHistory: p.Extractor = &StatementsSummaryExtractor{} + case infoschema.TableTiKVRegionPeers: + p.Extractor = &TikvRegionPeersExtractor{} } } return p, nil diff --git a/planner/core/memtable_predicate_extractor.go b/planner/core/memtable_predicate_extractor.go index 923d025ac7d13..57b53a75cc080 100644 --- a/planner/core/memtable_predicate_extractor.go +++ b/planner/core/memtable_predicate_extractor.go @@ -1415,3 +1415,56 @@ func (e *StatementsSummaryExtractor) explainInfo(p *PhysicalMemTable) string { } return fmt.Sprintf("digests: [%s]", extractStringFromStringSet(e.Digests)) } + +// TikvRegionPeersExtractor is used to extract some predicates of cluster table. +type TikvRegionPeersExtractor struct { + extractHelper + + // SkipRequest means the where clause always false, we don't need to request any component + SkipRequest bool + + // RegionIDs/StoreIDs represents all region/store ids we should filter in PD to reduce network IO. + // e.g: + // 1. SELECT * FROM tikv_region_peers WHERE region_id=1 + // 2. SELECT * FROM tikv_region_peers WHERE table_id in (11, 22) + RegionIDs []uint64 + StoreIDs []uint64 +} + +// Extract implements the MemTablePredicateExtractor Extract interface +func (e *TikvRegionPeersExtractor) Extract(_ sessionctx.Context, + schema *expression.Schema, + names []*types.FieldName, + predicates []expression.Expression, +) []expression.Expression { + // Extract the `region_id/store_id` columns. + remained, regionIDSkipRequest, regionIDs := e.extractCol(schema, names, predicates, "region_id", false) + remained, storeIDSkipRequest, storeIDs := e.extractCol(schema, names, remained, "store_id", false) + e.RegionIDs, e.StoreIDs = e.parseUint64(regionIDs), e.parseUint64(storeIDs) + + e.SkipRequest = regionIDSkipRequest || storeIDSkipRequest + if e.SkipRequest { + return nil + } + + return remained +} + +func (e *TikvRegionPeersExtractor) explainInfo(p *PhysicalMemTable) string { + if e.SkipRequest { + return "skip_request:true" + } + r := new(bytes.Buffer) + if len(e.RegionIDs) > 0 { + r.WriteString(fmt.Sprintf("region_ids:[%s], ", extractStringFromUint64Slice(e.RegionIDs))) + } + if len(e.StoreIDs) > 0 { + r.WriteString(fmt.Sprintf("store_ids:[%s], ", extractStringFromUint64Slice(e.StoreIDs))) + } + // remove the last ", " in the message info + s := r.String() + if len(s) > 2 { + return s[:len(s)-2] + } + return s +} diff --git a/planner/core/memtable_predicate_extractor_test.go b/planner/core/memtable_predicate_extractor_test.go index 311b9ae4a8838..d932176b9f2eb 100644 --- a/planner/core/memtable_predicate_extractor_test.go +++ b/planner/core/memtable_predicate_extractor_test.go @@ -1420,3 +1420,135 @@ func (s *extractorSuite) TestTiDBHotRegionsHistoryTableExtractor(c *C) { } } } + +func (s *extractorSuite) TestTikvRegionPeersExtractor(c *C) { + se, err := session.CreateSession4Test(s.store) + c.Assert(err, IsNil) + + var cases = []struct { + sql string + regionIDs, storeIDs []uint64 + skipRequest bool + }{ + // Test `region_id`, `store_id` columns. + { + sql: "select * from information_schema.tikv_region_peers where region_id=100", + regionIDs: []uint64{100}, + }, + { + sql: "select * from information_schema.tikv_region_peers where 100=region_id", + regionIDs: []uint64{100}, + }, + { + sql: "select * from information_schema.tikv_region_peers where 100=region_id or region_id=101", + regionIDs: []uint64{100, 101}, + }, + { + sql: "select * from information_schema.tikv_region_peers where 100=region_id or region_id=101 or region_id=102 or 103 = region_id", + regionIDs: []uint64{100, 101, 102, 103}, + }, + { + sql: "select * from information_schema.tikv_region_peers where (region_id=100 or region_id=101) and (store_id=200 or store_id=201)", + regionIDs: []uint64{100, 101}, + storeIDs: []uint64{200, 201}, + }, + { + sql: "select * from information_schema.tikv_region_peers where region_id in (100, 101)", + regionIDs: []uint64{100, 101}, + }, + { + sql: "select * from information_schema.tikv_region_peers where region_id in (100, 101) and store_id=200", + regionIDs: []uint64{100, 101}, + storeIDs: []uint64{200}, + }, + { + sql: "select * from information_schema.tikv_region_peers where region_id in (100, 101) and store_id in (200, 201)", + regionIDs: []uint64{100, 101}, + storeIDs: []uint64{200, 201}, + }, + { + sql: "select * from information_schema.tikv_region_peers where region_id=100 and store_id in (200, 201)", + regionIDs: []uint64{100}, + storeIDs: []uint64{200, 201}, + }, + { + sql: "select * from information_schema.tikv_region_peers where region_id=100 and store_id=200", + regionIDs: []uint64{100}, + storeIDs: []uint64{200}, + }, + { + sql: "select * from information_schema.tikv_region_peers where region_id=100 and region_id=101", + skipRequest: true, + }, + { + sql: "select * from information_schema.tikv_region_peers where region_id=100 and region_id in (100,101)", + regionIDs: []uint64{100}, + }, + { + sql: "select * from information_schema.tikv_region_peers where region_id=100 and region_id in (100,101) and store_id=200 and store_id in (200,201)", + regionIDs: []uint64{100}, + storeIDs: []uint64{200}, + }, + { + sql: "select * from information_schema.tikv_region_peers where region_id=100 and region_id in (101,102)", + skipRequest: true, + }, + { + sql: "select * from information_schema.tikv_region_peers where region_id=100 and region_id in (101,102) and store_id=200 and store_id in (200,201)", + skipRequest: true, + }, + { + sql: "select * from information_schema.tikv_region_peers where region_id=100 and region_id in (100,101) and store_id=200 and store_id in (201,202)", + skipRequest: true, + }, + { + sql: `select * from information_schema.tikv_region_peers + where region_id=100 and region_id in (100,101) + and store_id=200 and store_id in (201,202)`, + skipRequest: true, + }, + { + sql: "select * from information_schema.tikv_region_peers where region_id in (100,101) and region_id in (101,102)", + regionIDs: []uint64{101}, + }, + { + sql: `select * from information_schema.tikv_region_peers + where region_id in (100,101) + and region_id in (101,102) + and store_id in (200,201) + and store_id in (201,202)`, + regionIDs: []uint64{101}, + storeIDs: []uint64{201}, + }, + { + sql: `select * from information_schema.tikv_region_peers + where region_id in (100,101) + and region_id in (100,102) + and region_id in (102,103) + and region_id in (103,104)`, + skipRequest: true, + }, + // Test columns that is not extracted by TikvRegionPeersExtractor + { + sql: `select * from information_schema.tikv_region_peers + where peer_id=100 + and is_learner=0 + and is_leader=1 + and status='NORMAL' + and down_seconds=1000`, + }, + } + parser := parser.New() + for _, ca := range cases { + logicalMemTable := s.getLogicalMemTable(c, se, parser, ca.sql) + c.Assert(logicalMemTable.Extractor, NotNil) + + tikvRegionPeersExtractor := logicalMemTable.Extractor.(*plannercore.TikvRegionPeersExtractor) + if len(ca.regionIDs) > 0 { + c.Assert(tikvRegionPeersExtractor.RegionIDs, DeepEquals, ca.regionIDs, Commentf("SQL: %v", ca.sql)) + } + if len(ca.storeIDs) > 0 { + c.Assert(tikvRegionPeersExtractor.StoreIDs, DeepEquals, ca.storeIDs, Commentf("SQL: %v", ca.sql)) + } + } +} From 393415782452903b417a539d7fc8cdcecc6ab1bf Mon Sep 17 00:00:00 2001 From: glorv Date: Wed, 22 Dec 2021 10:17:47 +0800 Subject: [PATCH 12/44] lightning: add back table empty check and add a switch config (#30887) close pingcap/tidb#27919 --- br/pkg/lightning/config/config.go | 1 + br/pkg/lightning/restore/check_info.go | 91 +++++++++++- br/pkg/lightning/restore/check_info_test.go | 139 ++++++++++++++++++ br/pkg/lightning/restore/meta_manager.go | 58 +++++++- br/pkg/lightning/restore/restore.go | 20 ++- .../lightning_distributed_import/config.toml | 1 + .../config1.toml | 1 + .../config2.toml | 1 + br/tests/lightning_incremental/config.toml | 2 + br/tests/lightning_local_backend/run.sh | 17 ++- .../data/rowid.pre_rebase-schema.sql | 2 +- br/tests/lightning_tidb_rowid/run.sh | 9 +- 12 files changed, 325 insertions(+), 17 deletions(-) diff --git a/br/pkg/lightning/config/config.go b/br/pkg/lightning/config/config.go index d080d1bad16cf..cac695801a64a 100644 --- a/br/pkg/lightning/config/config.go +++ b/br/pkg/lightning/config/config.go @@ -527,6 +527,7 @@ type TikvImporter struct { DiskQuota ByteSize `toml:"disk-quota" json:"disk-quota"` RangeConcurrency int `toml:"range-concurrency" json:"range-concurrency"` DuplicateResolution DuplicateResolutionAlgorithm `toml:"duplicate-resolution" json:"duplicate-resolution"` + IncrementalImport bool `toml:"incremental-import" json:"incremental-import"` EngineMemCacheSize ByteSize `toml:"engine-mem-cache-size" json:"engine-mem-cache-size"` LocalWriterMemCacheSize ByteSize `toml:"local-writer-mem-cache-size" json:"local-writer-mem-cache-size"` diff --git a/br/pkg/lightning/restore/check_info.go b/br/pkg/lightning/restore/check_info.go index f97afc33b7cd0..4da674e1cd40a 100644 --- a/br/pkg/lightning/restore/check_info.go +++ b/br/pkg/lightning/restore/check_info.go @@ -17,6 +17,7 @@ package restore import ( "bytes" "context" + "database/sql" "fmt" "io" "path/filepath" @@ -24,10 +25,15 @@ import ( "sort" "strconv" "strings" + "sync" "github.com/docker/go-units" "github.com/pingcap/errors" "github.com/pingcap/failpoint" + "go.uber.org/zap" + "golang.org/x/sync/errgroup" + "modernc.org/mathutil" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/tidb/br/pkg/lightning/backend" "github.com/pingcap/tidb/br/pkg/lightning/backend/kv" @@ -38,6 +44,7 @@ import ( "github.com/pingcap/tidb/br/pkg/lightning/mydump" "github.com/pingcap/tidb/br/pkg/lightning/verification" "github.com/pingcap/tidb/br/pkg/storage" + "github.com/pingcap/tidb/br/pkg/utils" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/table" @@ -45,9 +52,6 @@ import ( "github.com/pingcap/tidb/types" "github.com/tikv/pd/server/api" pdconfig "github.com/tikv/pd/server/config" - - "go.uber.org/zap" - "modernc.org/mathutil" ) const ( @@ -1053,3 +1057,84 @@ outloop: log.L().Info("Sample source data", zap.String("table", tableMeta.Name), zap.Float64("IndexRatio", tableMeta.IndexRatio), zap.Bool("IsSourceOrder", tableMeta.IsRowOrdered)) return nil } + +func (rc *Controller) checkTableEmpty(ctx context.Context) error { + if rc.cfg.TikvImporter.Backend == config.BackendTiDB || rc.cfg.TikvImporter.IncrementalImport { + return nil + } + db, _ := rc.tidbGlue.GetDB() + + tableCount := 0 + for _, db := range rc.dbMetas { + tableCount += len(db.Tables) + } + + var lock sync.Mutex + tableNames := make([]string, 0) + concurrency := utils.MinInt(tableCount, rc.cfg.App.RegionConcurrency) + ch := make(chan string, concurrency) + eg, gCtx := errgroup.WithContext(ctx) + for i := 0; i < concurrency; i++ { + eg.Go(func() error { + for tblName := range ch { + // skip tables that have checkpoint + if rc.cfg.Checkpoint.Enable { + _, err := rc.checkpointsDB.Get(gCtx, tblName) + switch { + case err == nil: + continue + case errors.IsNotFound(err): + default: + return errors.Trace(err) + } + } + + hasData, err1 := tableContainsData(gCtx, db, tblName) + if err1 != nil { + return err1 + } + if hasData { + lock.Lock() + tableNames = append(tableNames, tblName) + lock.Unlock() + } + } + return nil + }) + } + for _, db := range rc.dbMetas { + for _, tbl := range db.Tables { + ch <- common.UniqueTable(tbl.DB, tbl.Name) + } + } + close(ch) + if err := eg.Wait(); err != nil { + if common.IsContextCanceledError(err) { + return nil + } + return errors.Trace(err) + } + + if len(tableNames) > 0 { + // sort the failed names + sort.Strings(tableNames) + msg := fmt.Sprintf("table(s) [%s] are not empty", strings.Join(tableNames, ", ")) + rc.checkTemplate.Collect(Critical, false, msg) + } + return nil +} + +func tableContainsData(ctx context.Context, db utils.QueryExecutor, tableName string) (bool, error) { + query := "select 1 from " + tableName + " limit 1" + var dump int + err := db.QueryRowContext(ctx, query).Scan(&dump) + + switch { + case err == sql.ErrNoRows: + return false, nil + case err != nil: + return false, errors.Trace(err) + default: + return true, nil + } +} diff --git a/br/pkg/lightning/restore/check_info_test.go b/br/pkg/lightning/restore/check_info_test.go index ccc4aa74c0c28..c679298f6a612 100644 --- a/br/pkg/lightning/restore/check_info_test.go +++ b/br/pkg/lightning/restore/check_info_test.go @@ -16,15 +16,18 @@ package restore import ( "context" + "database/sql" "fmt" "os" "path/filepath" + "github.com/DATA-DOG/go-sqlmock" . "github.com/pingcap/check" "github.com/pingcap/failpoint" "github.com/pingcap/tidb/br/pkg/lightning/checkpoints" "github.com/pingcap/tidb/br/pkg/lightning/config" + "github.com/pingcap/tidb/br/pkg/lightning/glue" "github.com/pingcap/tidb/br/pkg/lightning/mydump" "github.com/pingcap/tidb/br/pkg/lightning/worker" "github.com/pingcap/tidb/br/pkg/storage" @@ -404,6 +407,142 @@ func (s *checkInfoSuite) TestCheckCSVHeader(c *C) { } } +func (s *checkInfoSuite) TestCheckTableEmpty(c *C) { + dir := c.MkDir() + cfg := config.NewConfig() + cfg.Checkpoint.Enable = false + dbMetas := []*mydump.MDDatabaseMeta{ + { + Name: "test1", + Tables: []*mydump.MDTableMeta{ + { + DB: "test1", + Name: "tbl1", + }, + { + DB: "test1", + Name: "tbl2", + }, + }, + }, + { + Name: "test2", + Tables: []*mydump.MDTableMeta{ + { + DB: "test2", + Name: "tbl1", + }, + }, + }, + } + + rc := &Controller{ + cfg: cfg, + dbMetas: dbMetas, + checkpointsDB: checkpoints.NewNullCheckpointsDB(), + } + + ctx := context.Background() + + // test tidb will do nothing + rc.cfg.TikvImporter.Backend = config.BackendTiDB + err := rc.checkTableEmpty(ctx) + c.Assert(err, IsNil) + + // test incremental mode + rc.cfg.TikvImporter.Backend = config.BackendLocal + rc.cfg.TikvImporter.IncrementalImport = true + err = rc.checkTableEmpty(ctx) + c.Assert(err, IsNil) + + rc.cfg.TikvImporter.IncrementalImport = false + db, mock, err := sqlmock.New() + c.Assert(err, IsNil) + mock.MatchExpectationsInOrder(false) + rc.tidbGlue = glue.NewExternalTiDBGlue(db, mysql.ModeNone) + mock.ExpectQuery("select 1 from `test1`.`tbl1` limit 1"). + WillReturnRows(sqlmock.NewRows([]string{""}).RowError(0, sql.ErrNoRows)) + mock.ExpectQuery("select 1 from `test1`.`tbl2` limit 1"). + WillReturnRows(sqlmock.NewRows([]string{""}).RowError(0, sql.ErrNoRows)) + mock.ExpectQuery("select 1 from `test2`.`tbl1` limit 1"). + WillReturnRows(sqlmock.NewRows([]string{""}).RowError(0, sql.ErrNoRows)) + // not error, need not to init check template + err = rc.checkTableEmpty(ctx) + c.Assert(err, IsNil) + c.Assert(mock.ExpectationsWereMet(), IsNil) + + // single table contains data + db, mock, err = sqlmock.New() + c.Assert(err, IsNil) + rc.tidbGlue = glue.NewExternalTiDBGlue(db, mysql.ModeNone) + mock.MatchExpectationsInOrder(false) + mock.ExpectQuery("select 1 from `test1`.`tbl1` limit 1"). + WillReturnRows(sqlmock.NewRows([]string{""}).RowError(0, sql.ErrNoRows)) + mock.ExpectQuery("select 1 from `test1`.`tbl2` limit 1"). + WillReturnRows(sqlmock.NewRows([]string{""}).RowError(0, sql.ErrNoRows)) + mock.ExpectQuery("select 1 from `test2`.`tbl1` limit 1"). + WillReturnRows(sqlmock.NewRows([]string{""}).AddRow(1)) + rc.checkTemplate = NewSimpleTemplate() + err = rc.checkTableEmpty(ctx) + c.Assert(err, IsNil) + c.Assert(mock.ExpectationsWereMet(), IsNil) + tmpl := rc.checkTemplate.(*SimpleTemplate) + c.Assert(len(tmpl.criticalMsgs), Equals, 1) + c.Assert(tmpl.criticalMsgs[0], Matches, "table\\(s\\) \\[`test2`.`tbl1`\\] are not empty") + + // multi tables contains data + db, mock, err = sqlmock.New() + c.Assert(err, IsNil) + rc.tidbGlue = glue.NewExternalTiDBGlue(db, mysql.ModeNone) + mock.MatchExpectationsInOrder(false) + mock.ExpectQuery("select 1 from `test1`.`tbl1` limit 1"). + WillReturnRows(sqlmock.NewRows([]string{""}).AddRow(1)) + mock.ExpectQuery("select 1 from `test1`.`tbl2` limit 1"). + WillReturnRows(sqlmock.NewRows([]string{""}).RowError(0, sql.ErrNoRows)) + mock.ExpectQuery("select 1 from `test2`.`tbl1` limit 1"). + WillReturnRows(sqlmock.NewRows([]string{""}).AddRow(1)) + rc.checkTemplate = NewSimpleTemplate() + err = rc.checkTableEmpty(ctx) + c.Assert(err, IsNil) + c.Assert(mock.ExpectationsWereMet(), IsNil) + tmpl = rc.checkTemplate.(*SimpleTemplate) + c.Assert(len(tmpl.criticalMsgs), Equals, 1) + c.Assert(tmpl.criticalMsgs[0], Matches, "table\\(s\\) \\[`test1`.`tbl1`, `test2`.`tbl1`\\] are not empty") + + // init checkpoint with only two of the three tables + dbInfos := map[string]*checkpoints.TidbDBInfo{ + "test1": { + Name: "test1", + Tables: map[string]*checkpoints.TidbTableInfo{ + "tbl1": { + Name: "tbl1", + }, + }, + }, + "test2": { + Name: "test2", + Tables: map[string]*checkpoints.TidbTableInfo{ + "tbl1": { + Name: "tbl1", + }, + }, + }, + } + rc.cfg.Checkpoint.Enable = true + rc.checkpointsDB = checkpoints.NewFileCheckpointsDB(filepath.Join(dir, "cp.pb")) + err = rc.checkpointsDB.Initialize(ctx, cfg, dbInfos) + c.Check(err, IsNil) + db, mock, err = sqlmock.New() + c.Assert(err, IsNil) + rc.tidbGlue = glue.NewExternalTiDBGlue(db, mysql.ModeNone) + // only need to check the one that is not in checkpoint + mock.ExpectQuery("select 1 from `test1`.`tbl2` limit 1"). + WillReturnRows(sqlmock.NewRows([]string{""}).RowError(0, sql.ErrNoRows)) + err = rc.checkTableEmpty(ctx) + c.Assert(err, IsNil) + c.Assert(mock.ExpectationsWereMet(), IsNil) +} + func (s *checkInfoSuite) TestLocalResource(c *C) { dir := c.MkDir() mockStore, err := storage.NewLocalStorage(dir) diff --git a/br/pkg/lightning/restore/meta_manager.go b/br/pkg/lightning/restore/meta_manager.go index 544b91c0b5f90..49358a9aee102 100644 --- a/br/pkg/lightning/restore/meta_manager.go +++ b/br/pkg/lightning/restore/meta_manager.go @@ -1027,9 +1027,65 @@ func (m noopTableMetaMgr) UpdateTableBaseChecksum(ctx context.Context, checksum } func (m noopTableMetaMgr) CheckAndUpdateLocalChecksum(ctx context.Context, checksum *verify.KVChecksum, hasLocalDupes bool) (bool, bool, *verify.KVChecksum, error) { - return false, false, nil, nil + return true, true, &verify.KVChecksum{}, nil } func (m noopTableMetaMgr) FinishTable(ctx context.Context) error { return nil } + +type singleMgrBuilder struct{} + +func (b singleMgrBuilder) Init(context.Context) error { + return nil +} + +func (b singleMgrBuilder) TaskMetaMgr(pd *pdutil.PdController) taskMetaMgr { + return &singleTaskMetaMgr{ + pd: pd, + } +} + +func (b singleMgrBuilder) TableMetaMgr(tr *TableRestore) tableMetaMgr { + return noopTableMetaMgr{} +} + +type singleTaskMetaMgr struct { + pd *pdutil.PdController +} + +func (m *singleTaskMetaMgr) InitTask(ctx context.Context, source int64) error { + return nil +} + +func (m *singleTaskMetaMgr) CheckTasksExclusively(ctx context.Context, action func(tasks []taskMeta) ([]taskMeta, error)) error { + _, err := action(nil) + return err +} + +func (m *singleTaskMetaMgr) CheckAndPausePdSchedulers(ctx context.Context) (pdutil.UndoFunc, error) { + return m.pd.RemoveSchedulers(ctx) +} + +func (m *singleTaskMetaMgr) CheckTaskExist(ctx context.Context) (bool, error) { + return true, nil +} + +func (m *singleTaskMetaMgr) CheckAndFinishRestore(context.Context, bool) (shouldSwitchBack bool, shouldCleanupMeta bool, err error) { + return true, true, nil +} + +func (m *singleTaskMetaMgr) Cleanup(ctx context.Context) error { + return nil +} + +func (m *singleTaskMetaMgr) CleanupTask(ctx context.Context) error { + return nil +} + +func (m *singleTaskMetaMgr) CleanupAllMetas(ctx context.Context) error { + return nil +} + +func (m *singleTaskMetaMgr) Close() { +} diff --git a/br/pkg/lightning/restore/restore.go b/br/pkg/lightning/restore/restore.go index 82a8465eb8181..79f132b1cf5f6 100644 --- a/br/pkg/lightning/restore/restore.go +++ b/br/pkg/lightning/restore/restore.go @@ -379,14 +379,17 @@ func NewRestoreControllerWithPauser( } var metaBuilder metaMgrBuilder - switch cfg.TikvImporter.Backend { - case config.BackendLocal, config.BackendImporter: + isSSTImport := cfg.TikvImporter.Backend == config.BackendLocal || cfg.TikvImporter.Backend == config.BackendImporter + switch { + case isSSTImport && cfg.TikvImporter.IncrementalImport: metaBuilder = &dbMetaMgrBuilder{ db: db, taskID: cfg.TaskID, schema: cfg.App.MetaSchemaName, needChecksum: cfg.PostRestore.Checksum != config.OpLevelOff, } + case isSSTImport: + metaBuilder = singleMgrBuilder{} default: metaBuilder = noopMetaMgrBuilder{} } @@ -1967,11 +1970,6 @@ func (rc *Controller) DataCheck(ctx context.Context) error { } } } - err = rc.checkCSVHeader(ctx, rc.dbMetas) - if err != nil { - return err - } - if len(checkPointCriticalMsgs) != 0 { rc.checkTemplate.Collect(Critical, false, strings.Join(checkPointCriticalMsgs, "\n")) } else { @@ -1982,6 +1980,14 @@ func (rc *Controller) DataCheck(ctx context.Context) error { } else { rc.checkTemplate.Collect(Critical, true, "table schemas are valid") } + + if err := rc.checkTableEmpty(ctx); err != nil { + return errors.Trace(err) + } + if err = rc.checkCSVHeader(ctx, rc.dbMetas); err != nil { + return err + } + return nil } diff --git a/br/tests/lightning_distributed_import/config.toml b/br/tests/lightning_distributed_import/config.toml index 200af8e45dfdc..947b16037dd5d 100644 --- a/br/tests/lightning_distributed_import/config.toml +++ b/br/tests/lightning_distributed_import/config.toml @@ -1,6 +1,7 @@ [tikv-importer] backend = 'local' duplicate-resolution = 'none' +incremental-import = true [post-restore] checksum = "required" diff --git a/br/tests/lightning_duplicate_detection/config1.toml b/br/tests/lightning_duplicate_detection/config1.toml index 0b2b6df2a70e8..6497e9e30949b 100644 --- a/br/tests/lightning_duplicate_detection/config1.toml +++ b/br/tests/lightning_duplicate_detection/config1.toml @@ -6,6 +6,7 @@ table-concurrency = 10 [tikv-importer] backend = "local" duplicate-resolution = 'record' +incremental-import = true [checkpoint] enable = true diff --git a/br/tests/lightning_duplicate_detection/config2.toml b/br/tests/lightning_duplicate_detection/config2.toml index e978ffb9cd8b5..760f50168508a 100644 --- a/br/tests/lightning_duplicate_detection/config2.toml +++ b/br/tests/lightning_duplicate_detection/config2.toml @@ -6,6 +6,7 @@ table-concurrency = 10 [tikv-importer] backend = "local" duplicate-resolution = 'record' +incremental-import = true [checkpoint] enable = true diff --git a/br/tests/lightning_incremental/config.toml b/br/tests/lightning_incremental/config.toml index e69de29bb2d1d..761e60b91b804 100644 --- a/br/tests/lightning_incremental/config.toml +++ b/br/tests/lightning_incremental/config.toml @@ -0,0 +1,2 @@ +[tikv-importer] +incremental-import = true diff --git a/br/tests/lightning_local_backend/run.sh b/br/tests/lightning_local_backend/run.sh index 6d0e7e9864145..5843210fea738 100755 --- a/br/tests/lightning_local_backend/run.sh +++ b/br/tests/lightning_local_backend/run.sh @@ -20,12 +20,23 @@ check_cluster_version 4 0 0 'local backend' || exit 0 ENGINE_COUNT=6 -# First, verify that inject with not leader error is fine. -rm -f "$TEST_DIR/lightning-local.log" +# Test check table contains data rm -f "/tmp/tidb_lightning_checkpoint_local_backend_test.pb" +rm -rf $TEST_DIR/lightning.log run_sql 'DROP DATABASE IF EXISTS cpeng;' -export GO_FAILPOINTS='github.com/pingcap/tidb/br/pkg/lightning/backend/local/FailIngestMeta=1*return("notleader")' +run_sql 'CREATE DATABASE cpeng;' +run_sql 'CREATE TABLE cpeng.a (c int);' +run_sql 'CREATE TABLE cpeng.b (c int);' +run_sql "INSERT INTO cpeng.a values (1), (2);" +run_sql "INSERT INTO cpeng.b values (3);" +! run_lightning --backend local --enable-checkpoint=0 +grep -Fq 'table(s) [`cpeng`.`a`, `cpeng`.`b`] are not empty' $TEST_DIR/lightning.log + +# First, verify that inject with not leader error is fine. +export GO_FAILPOINTS='github.com/pingcap/tidb/br/pkg/lightning/backend/local/FailIngestMeta=1*return("notleader")' +rm -f "$TEST_DIR/lightning-local.log" +run_sql 'DROP DATABASE IF EXISTS cpeng;' run_lightning --backend local --enable-checkpoint=1 --log-file "$TEST_DIR/lightning-local.log" --config "tests/$TEST_NAME/config.toml" # Check that everything is correctly imported diff --git a/br/tests/lightning_tidb_rowid/data/rowid.pre_rebase-schema.sql b/br/tests/lightning_tidb_rowid/data/rowid.pre_rebase-schema.sql index 887540be58110..1738b64457de6 100644 --- a/br/tests/lightning_tidb_rowid/data/rowid.pre_rebase-schema.sql +++ b/br/tests/lightning_tidb_rowid/data/rowid.pre_rebase-schema.sql @@ -1 +1 @@ -create table pre_rebase (pk varchar(6) primary key) auto_increment=70000; +create table pre_rebase (pk varchar(6) primary key /*T![clustered_index] NONCLUSTERED */) auto_increment=70000; diff --git a/br/tests/lightning_tidb_rowid/run.sh b/br/tests/lightning_tidb_rowid/run.sh index e877f420cf43f..ae762c514d93c 100755 --- a/br/tests/lightning_tidb_rowid/run.sh +++ b/br/tests/lightning_tidb_rowid/run.sh @@ -58,8 +58,13 @@ for BACKEND in local importer tidb; do run_sql 'SELECT count(*), min(_tidb_rowid), max(_tidb_rowid) FROM rowid.pre_rebase' check_contains 'count(*): 1' - check_contains 'min(_tidb_rowid): 70000' - check_contains 'max(_tidb_rowid): 70000' + if [ "$BACKEND" == 'tidb' ]; then + check_contains 'min(_tidb_rowid): 70000' + check_contains 'max(_tidb_rowid): 70000' + else + check_contains 'min(_tidb_rowid): 1' + check_contains 'max(_tidb_rowid): 1' + fi run_sql 'INSERT INTO rowid.pre_rebase VALUES ("?")' run_sql 'SELECT _tidb_rowid > 70000 FROM rowid.pre_rebase WHERE pk = "?"' check_contains '_tidb_rowid > 70000: 1' From 3bd732f9c4731c7617bf84216a2f3fcfddf845ae Mon Sep 17 00:00:00 2001 From: fengou1 <85682690+fengou1@users.noreply.github.com> Date: Wed, 22 Dec 2021 11:19:47 +0800 Subject: [PATCH 13/44] br: improve backoff unit test (#30892) --- br/pkg/utils/backoff_test.go | 40 ++++++++++++++++++++++++++++++++++++ 1 file changed, 40 insertions(+) diff --git a/br/pkg/utils/backoff_test.go b/br/pkg/utils/backoff_test.go index 9ee312f24feab..b0c0f640ab677 100644 --- a/br/pkg/utils/backoff_test.go +++ b/br/pkg/utils/backoff_test.go @@ -123,3 +123,43 @@ func TestPdBackoffWithRetryableError(t *testing.T) { gRPCError, }, multierr.Errors(err)) } + +func TestNewImportSSTBackofferWithSucess(t *testing.T) { + t.Parallel() + + var counter int + backoffer := utils.NewImportSSTBackoffer() + err := utils.WithRetry(context.Background(), func() error { + defer func() { counter++ }() + if counter == 15 { + return nil + } else { + return berrors.ErrKVDownloadFailed + } + }, backoffer) + require.Equal(t, 16, counter) + require.Nil(t, err) +} + +func TestNewDownloadSSTBackofferWithCancel(t *testing.T) { + t.Parallel() + + var counter int + backoffer := utils.NewDownloadSSTBackoffer() + err := utils.WithRetry(context.Background(), func() error { + defer func() { counter++ }() + if counter == 3 { + return context.Canceled + } else { + return berrors.ErrKVIngestFailed + } + + }, backoffer) + require.Equal(t, 4, counter) + require.Equal(t, []error{ + berrors.ErrKVIngestFailed, + berrors.ErrKVIngestFailed, + berrors.ErrKVIngestFailed, + context.Canceled, + }, multierr.Errors(err)) +} From 529ce88c1cc65cbdcf6bd81c48c3a2d0efc424ba Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=8E=8B=E8=B6=85?= Date: Wed, 22 Dec 2021 13:53:43 +0800 Subject: [PATCH 14/44] *: add TxnManager to manage txn in session (#30574) * *: add TxnManager to manage txn in session * modify * add tests * move failpoint content to a single file --- ddl/db_change_test.go | 4 +- executor/adapter.go | 26 + executor/compiler.go | 13 +- executor/executor.go | 3 +- executor/prepared.go | 7 + executor/seqtest/prepared_test.go | 3 +- planner/core/cache.go | 20 + planner/core/preprocess.go | 52 ++ planner/optimize.go | 21 +- session/session.go | 73 ++- session/session_test.go | 3 +- session/txnmanager.go | 62 +++ sessionctx/context.go | 1 + sessionctx/variable/session.go | 3 + sessiontxn/failpoint.go | 74 +++ sessiontxn/interface.go | 59 +++ sessiontxn/txn_context_serial_test.go | 706 ++++++++++++++++++++++++++ 17 files changed, 1090 insertions(+), 40 deletions(-) create mode 100644 session/txnmanager.go create mode 100644 sessiontxn/failpoint.go create mode 100644 sessiontxn/interface.go create mode 100644 sessiontxn/txn_context_serial_test.go diff --git a/ddl/db_change_test.go b/ddl/db_change_test.go index 3ae2259ddaf07..1873cb02d9fe0 100644 --- a/ddl/db_change_test.go +++ b/ddl/db_change_test.go @@ -411,7 +411,9 @@ func (t *testExecInfo) compileSQL(idx int) (err error) { compiler := executor.Compiler{Ctx: c.session} se := c.session ctx := context.TODO() - se.PrepareTxnCtx(ctx) + if err = se.PrepareTxnCtx(ctx); err != nil { + return err + } sctx := se.(sessionctx.Context) if err = executor.ResetContextOfStmt(sctx, c.rawStmt); err != nil { return errors.Trace(err) diff --git a/executor/adapter.go b/executor/adapter.go index 39e660099ed3f..be83c42d99940 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -46,6 +46,7 @@ import ( "github.com/pingcap/tidb/plugin" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/sessiontxn" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/execdetails" @@ -263,6 +264,12 @@ func (a *ExecStmt) PointGet(ctx context.Context, is infoschema.InfoSchema) (*rec a.PsStmt.Executor = newExecutor } pointExecutor := a.PsStmt.Executor.(*PointGetExecutor) + + failpoint.Inject("assertTxnManagerInShortPointGetPlan", func() { + sessiontxn.RecordAssert(a.Ctx, "assertTxnManagerInShortPointGetPlan", true) + sessiontxn.AssertTxnManagerInfoSchema(a.Ctx, is) + }) + if err = pointExecutor.Open(ctx); err != nil { terror.Call(pointExecutor.Close) return nil, err @@ -298,6 +305,16 @@ func (a *ExecStmt) RebuildPlan(ctx context.Context) (int64, error) { if err := plannercore.Preprocess(a.Ctx, a.StmtNode, plannercore.InTxnRetry, plannercore.WithPreprocessorReturn(ret)); err != nil { return 0, err } + + failpoint.Inject("assertTxnManagerInRebuildPlan", func() { + if is, ok := a.Ctx.Value(sessiontxn.AssertTxnInfoSchemaAfterRetryKey).(infoschema.InfoSchema); ok { + a.Ctx.SetValue(sessiontxn.AssertTxnInfoSchemaKey, is) + a.Ctx.SetValue(sessiontxn.AssertTxnInfoSchemaAfterRetryKey, nil) + } + sessiontxn.RecordAssert(a.Ctx, "assertTxnManagerInRebuildPlan", true) + sessiontxn.AssertTxnManagerInfoSchema(a.Ctx, ret.InfoSchema) + }) + a.InfoSchema = ret.InfoSchema a.SnapshotTS = ret.LastSnapshotTS a.IsStaleness = ret.IsStaleness @@ -755,6 +772,10 @@ func (a *ExecStmt) handlePessimisticLockError(ctx context.Context, err error) (E a.Ctx.GetSessionVars().StmtCtx.ResetForRetry() a.Ctx.GetSessionVars().RetryInfo.ResetOffset() + failpoint.Inject("assertTxnManagerAfterPessimisticLockErrorRetry", func() { + sessiontxn.RecordAssert(a.Ctx, "assertTxnManagerAfterPessimisticLockErrorRetry", true) + }) + if err = e.Open(ctx); err != nil { return nil, err } @@ -809,6 +830,11 @@ func (a *ExecStmt) buildExecutor() (Executor, error) { return nil, errors.Trace(b.err) } + failpoint.Inject("assertTxnManagerAfterBuildExecutor", func() { + sessiontxn.RecordAssert(a.Ctx, "assertTxnManagerAfterBuildExecutor", true) + sessiontxn.AssertTxnManagerInfoSchema(b.ctx, b.is) + }) + // ExecuteExec is not a real Executor, we only use it to build another Executor from a prepared statement. if executorExec, ok := e.(*ExecuteExec); ok { err := executorExec.Build(b) diff --git a/executor/compiler.go b/executor/compiler.go index 74a878b4d3293..baf49979572c2 100644 --- a/executor/compiler.go +++ b/executor/compiler.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/tidb/planner" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/sessiontxn" ) var ( @@ -57,11 +58,21 @@ func (c *Compiler) Compile(ctx context.Context, stmtNode ast.StmtNode) (*ExecStm ret := &plannercore.PreprocessorReturn{} pe := &plannercore.PreprocessExecuteISUpdate{ExecuteInfoSchemaUpdate: planner.GetExecuteForUpdateReadIS, Node: stmtNode} - err := plannercore.Preprocess(c.Ctx, stmtNode, plannercore.WithPreprocessorReturn(ret), plannercore.WithExecuteInfoSchemaUpdate(pe)) + err := plannercore.Preprocess(c.Ctx, + stmtNode, + plannercore.WithPreprocessorReturn(ret), + plannercore.WithExecuteInfoSchemaUpdate(pe), + plannercore.InitTxnContextProvider, + ) if err != nil { return nil, err } + failpoint.Inject("assertTxnManagerInCompile", func() { + sessiontxn.RecordAssert(c.Ctx, "assertTxnManagerInCompile", true) + sessiontxn.AssertTxnManagerInfoSchema(c.Ctx, ret.InfoSchema) + }) + finalPlan, names, err := planner.Optimize(ctx, c.Ctx, stmtNode, ret.InfoSchema) if err != nil { return nil, err diff --git a/executor/executor.go b/executor/executor.go index 4338529cea8ac..af46ffc2b999f 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -45,7 +45,6 @@ import ( "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/parser/terror" - "github.com/pingcap/tidb/planner" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/privilege" "github.com/pingcap/tidb/sessionctx" @@ -1718,7 +1717,7 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) { sc.MemTracker.SetActionOnExceed(action) } if execStmt, ok := s.(*ast.ExecuteStmt); ok { - prepareStmt, err := planner.GetPreparedStmt(execStmt, vars) + prepareStmt, err := plannercore.GetPreparedStmt(execStmt, vars) if err != nil { return err } diff --git a/executor/prepared.go b/executor/prepared.go index 82a030e76b6c1..3b703c75a9cf5 100644 --- a/executor/prepared.go +++ b/executor/prepared.go @@ -21,6 +21,7 @@ import ( "time" "github.com/pingcap/errors" + "github.com/pingcap/failpoint" "github.com/pingcap/log" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/infoschema" @@ -31,6 +32,7 @@ import ( plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/sessiontxn" "github.com/pingcap/tidb/types" driver "github.com/pingcap/tidb/types/parser_driver" "github.com/pingcap/tidb/util" @@ -340,6 +342,11 @@ func CompileExecutePreparedStmt(ctx context.Context, sctx sessionctx.Context, return nil, false, false, err } + failpoint.Inject("assertTxnManagerInCompile", func() { + sessiontxn.RecordAssert(sctx, "assertTxnManagerInCompile", true) + sessiontxn.AssertTxnManagerInfoSchema(sctx, is) + }) + stmt := &ExecStmt{ GoCtx: ctx, InfoSchema: is, diff --git a/executor/seqtest/prepared_test.go b/executor/seqtest/prepared_test.go index 42fbf72164e54..a5aafc326d0c5 100644 --- a/executor/seqtest/prepared_test.go +++ b/executor/seqtest/prepared_test.go @@ -163,7 +163,8 @@ func TestPrepared(t *testing.T) { require.Equal(t, query, stmt.OriginText()) // Check that rebuild plan works. - tk.Session().PrepareTxnCtx(ctx) + err = tk.Session().PrepareTxnCtx(ctx) + require.NoError(t, err) _, err = stmt.RebuildPlan(ctx) require.NoError(t, err) rs, err = stmt.Exec(ctx) diff --git a/planner/core/cache.go b/planner/core/cache.go index a386c4a5a3649..4113f3e911e88 100644 --- a/planner/core/cache.go +++ b/planner/core/cache.go @@ -19,6 +19,7 @@ import ( "sync/atomic" "time" + "github.com/pingcap/errors" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/ast" @@ -213,3 +214,22 @@ type CachedPrepareStmt struct { ForUpdateRead bool SnapshotTSEvaluator func(sessionctx.Context) (uint64, error) } + +// GetPreparedStmt extract the prepared statement from the execute statement. +func GetPreparedStmt(stmt *ast.ExecuteStmt, vars *variable.SessionVars) (*CachedPrepareStmt, error) { + var ok bool + execID := stmt.ExecID + if stmt.Name != "" { + if execID, ok = vars.PreparedStmtNameToID[stmt.Name]; !ok { + return nil, ErrStmtNotFound + } + } + if preparedPointer, ok := vars.PreparedStmts[execID]; ok { + preparedObj, ok := preparedPointer.(*CachedPrepareStmt) + if !ok { + return nil, errors.Errorf("invalid CachedPrepareStmt type") + } + return preparedObj, nil + } + return nil, ErrStmtNotFound +} diff --git a/planner/core/preprocess.go b/planner/core/preprocess.go index 101e2a0c53479..6e2f9aaed4f53 100644 --- a/planner/core/preprocess.go +++ b/planner/core/preprocess.go @@ -37,6 +37,7 @@ import ( "github.com/pingcap/tidb/privilege" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/sessiontxn" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/temptable" "github.com/pingcap/tidb/types" @@ -59,6 +60,11 @@ func InTxnRetry(p *preprocessor) { p.flag |= inTxnRetry } +// InitTxnContextProvider is a PreprocessOpt that indicates preprocess should init transaction's context +func InitTxnContextProvider(p *preprocessor) { + p.flag |= initTxnContextProvider +} + // WithPreprocessorReturn returns a PreprocessOpt to initialize the PreprocessorReturn. func WithPreprocessorReturn(ret *PreprocessorReturn) PreprocessOpt { return func(p *preprocessor) { @@ -117,6 +123,9 @@ func Preprocess(ctx sessionctx.Context, node ast.Node, preprocessOpt ...Preproce node.Accept(&v) // InfoSchema must be non-nil after preprocessing v.ensureInfoSchema() + + v.initTxnContextProviderIfNecessary(node) + return errors.Trace(v.err) } @@ -136,6 +145,8 @@ const ( // inSequenceFunction is set when visiting a sequence function. // This flag indicates the tableName in these function should be checked as sequence object. inSequenceFunction + // initTxnContextProvider is set when we should init txn context in preprocess + initTxnContextProvider ) // Make linter happy. @@ -193,6 +204,9 @@ func (p *preprocessor) Enter(in ast.Node) (out ast.Node, skipChildren bool) { // handle the insert table name imminently // insert into t with t ..., the insert can not see t here. We should hand it before the CTE statement p.handleTableName(node.Table.TableRefs.Left.(*ast.TableSource).Source.(*ast.TableName)) + case *ast.ExecuteStmt: + p.stmtTp = TypeExecute + p.resolveExecuteStmt(node) case *ast.CreateTableStmt: p.stmtTp = TypeCreate p.flag |= inCreateOrDropTable @@ -361,6 +375,8 @@ const ( TypeRepair // TypeShow for ShowStmt TypeShow + // TypeExecute for ExecuteStmt + TypeExecute ) func bindableStmtType(node ast.StmtNode) byte { @@ -1489,6 +1505,32 @@ func (p *preprocessor) resolveShowStmt(node *ast.ShowStmt) { } } +func (p *preprocessor) resolveExecuteStmt(node *ast.ExecuteStmt) { + prepared, err := GetPreparedStmt(node, p.ctx.GetSessionVars()) + if err != nil { + p.err = err + return + } + + if prepared.SnapshotTSEvaluator != nil { + snapshotTS, err := prepared.SnapshotTSEvaluator(p.ctx) + if err != nil { + p.err = err + return + } + + is, err := domain.GetDomain(p.ctx).GetSnapshotInfoSchema(snapshotTS) + if err != nil { + p.err = err + return + } + + p.LastSnapshotTS = snapshotTS + p.initedLastSnapshotTS = true + p.InfoSchema = temptable.AttachLocalTemporaryTableInfoSchema(p.ctx, is) + } +} + func (p *preprocessor) resolveCreateTableStmt(node *ast.CreateTableStmt) { for _, val := range node.Constraints { if val.Refer != nil && val.Refer.Table.Schema.String() == "" { @@ -1689,3 +1731,13 @@ func (p *preprocessor) ensureInfoSchema() infoschema.InfoSchema { p.InfoSchema = p.ctx.GetInfoSchema().(infoschema.InfoSchema) return p.InfoSchema } + +func (p *preprocessor) initTxnContextProviderIfNecessary(node ast.Node) { + if p.err != nil || p.flag&initTxnContextProvider == 0 { + return + } + + p.err = sessiontxn.GetTxnManager(p.ctx).SetContextProvider(&sessiontxn.SimpleTxnContextProvider{ + InfoSchema: p.ensureInfoSchema(), + }) +} diff --git a/planner/optimize.go b/planner/optimize.go index b16fc09a238f0..e5d82a95d6e49 100644 --- a/planner/optimize.go +++ b/planner/optimize.go @@ -48,29 +48,10 @@ import ( "go.uber.org/zap" ) -// GetPreparedStmt extract the prepared statement from the execute statement. -func GetPreparedStmt(stmt *ast.ExecuteStmt, vars *variable.SessionVars) (*plannercore.CachedPrepareStmt, error) { - var ok bool - execID := stmt.ExecID - if stmt.Name != "" { - if execID, ok = vars.PreparedStmtNameToID[stmt.Name]; !ok { - return nil, plannercore.ErrStmtNotFound - } - } - if preparedPointer, ok := vars.PreparedStmts[execID]; ok { - preparedObj, ok := preparedPointer.(*plannercore.CachedPrepareStmt) - if !ok { - return nil, errors.Errorf("invalid CachedPrepareStmt type") - } - return preparedObj, nil - } - return nil, plannercore.ErrStmtNotFound -} - // IsReadOnly check whether the ast.Node is a read only statement. func IsReadOnly(node ast.Node, vars *variable.SessionVars) bool { if execStmt, isExecStmt := node.(*ast.ExecuteStmt); isExecStmt { - prepareStmt, err := GetPreparedStmt(execStmt, vars) + prepareStmt, err := plannercore.GetPreparedStmt(execStmt, vars) if err != nil { logutil.BgLogger().Warn("GetPreparedStmt failed", zap.Error(err)) return false diff --git a/session/session.go b/session/session.go index 561120ff4b9cc..cd1ab05da5498 100644 --- a/session/session.go +++ b/session/session.go @@ -44,6 +44,7 @@ import ( "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/parser/terror" + "github.com/pingcap/tidb/sessiontxn" "github.com/pingcap/tidb/store/driver/txn" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/table/temptable" @@ -153,7 +154,7 @@ type Session interface { // Return the information of the txn current running TxnInfo() *txninfo.TxnInfo // PrepareTxnCtx is exported for test. - PrepareTxnCtx(context.Context) + PrepareTxnCtx(context.Context) error // FieldList returns fields list of a table. FieldList(tableName string) (fields []*ast.ResultField, err error) SetPort(port string) @@ -997,7 +998,9 @@ func (s *session) retry(ctx context.Context, maxCnt uint) (err error) { orgStartTS := sessVars.TxnCtx.StartTS label := s.GetSQLLabel() for { - s.PrepareTxnCtx(ctx) + if err = s.PrepareTxnCtx(ctx); err != nil { + return err + } s.sessionVars.RetryInfo.ResetOffset() for i, sr := range nh.history { st := sr.st @@ -1634,7 +1637,10 @@ func (s *session) ExecuteStmt(ctx context.Context, stmtNode ast.StmtNode) (sqlex ctx = opentracing.ContextWithSpan(ctx, span1) } - s.PrepareTxnCtx(ctx) + if err := s.PrepareTxnCtx(ctx); err != nil { + return nil, err + } + if err := s.loadCommonGlobalVariablesIfNeeded(); err != nil { return nil, err } @@ -1772,6 +1778,13 @@ func (s *session) hasQuerySpecial() bool { // runStmt executes the sqlexec.Statement and commit or rollback the current transaction. func runStmt(ctx context.Context, se *session, s sqlexec.Statement) (rs sqlexec.RecordSet, err error) { + failpoint.Inject("assertTxnManagerInRunStmt", func() { + sessiontxn.RecordAssert(se, "assertTxnManagerInRunStmt", true) + if stmt, ok := s.(*executor.ExecStmt); ok { + sessiontxn.AssertTxnManagerInfoSchema(se, stmt.InfoSchema) + } + }) + if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { span1 := span.Tracer().StartSpan("session.runStmt", opentracing.ChildOf(span.Context())) span1.LogKV("sql", s.OriginText()) @@ -1912,7 +1925,9 @@ func (s *session) PrepareStmt(sql string) (stmtID uint32, paramCount int, fields inTxn := s.GetSessionVars().InTxn() // NewPrepareExec may need startTS to build the executor, for example prepare statement has subquery in int. // So we have to call PrepareTxnCtx here. - s.PrepareTxnCtx(ctx) + if err = s.PrepareTxnCtx(ctx); err != nil { + return + } s.PrepareTSFuture(ctx) prepareExec := executor.NewPrepareExec(s, sql) err = prepareExec.Next(ctx, nil) @@ -1929,6 +1944,12 @@ func (s *session) PrepareStmt(sql string) (stmtID uint32, paramCount int, fields func (s *session) preparedStmtExec(ctx context.Context, is infoschema.InfoSchema, snapshotTS uint64, stmtID uint32, prepareStmt *plannercore.CachedPrepareStmt, args []types.Datum) (sqlexec.RecordSet, error) { + + failpoint.Inject("assertTxnManagerInPreparedStmtExec", func() { + sessiontxn.RecordAssert(s, "assertTxnManagerInPreparedStmtExec", true) + sessiontxn.AssertTxnManagerInfoSchema(s, is) + }) + st, tiFlashPushDown, tiFlashExchangePushDown, err := executor.CompileExecutePreparedStmt(ctx, s, stmtID, is, snapshotTS, args) if err != nil { return nil, err @@ -1951,6 +1972,12 @@ func (s *session) preparedStmtExec(ctx context.Context, func (s *session) cachedPlanExec(ctx context.Context, is infoschema.InfoSchema, snapshotTS uint64, stmtID uint32, prepareStmt *plannercore.CachedPrepareStmt, args []types.Datum) (sqlexec.RecordSet, error) { + + failpoint.Inject("assertTxnManagerInCachedPlanExec", func() { + sessiontxn.RecordAssert(s, "assertTxnManagerInCachedPlanExec", true) + sessiontxn.AssertTxnManagerInfoSchema(s, is) + }) + prepared := prepareStmt.PreparedAst // compile ExecStmt execAst := &ast.ExecuteStmt{ExecID: stmtID} @@ -2066,8 +2093,11 @@ func (s *session) IsCachedExecOk(ctx context.Context, preparedStmt *plannercore. // ExecutePreparedStmt executes a prepared statement. func (s *session) ExecutePreparedStmt(ctx context.Context, stmtID uint32, args []types.Datum) (sqlexec.RecordSet, error) { - s.PrepareTxnCtx(ctx) var err error + if err = s.PrepareTxnCtx(ctx); err != nil { + return nil, err + } + s.sessionVars.StartTime = time.Now() preparedPointer, ok := s.sessionVars.PreparedStmts[stmtID] if !ok { @@ -2079,13 +2109,7 @@ func (s *session) ExecutePreparedStmt(ctx context.Context, stmtID uint32, args [ if !ok { return nil, errors.Errorf("invalid CachedPrepareStmt type") } - executor.CountStmtNode(preparedStmt.PreparedAst.Stmt, s.sessionVars.InRestrictedSQL) - ok, err = s.IsCachedExecOk(ctx, preparedStmt) - if err != nil { - return nil, err - } - s.txn.onStmtStart(preparedStmt.SQLDigest.String()) - defer s.txn.onStmtEnd() + var is infoschema.InfoSchema var snapshotTS uint64 if preparedStmt.ForUpdateRead { @@ -2102,6 +2126,22 @@ func (s *session) ExecutePreparedStmt(ctx context.Context, stmtID uint32, args [ } else { is = s.GetInfoSchema().(infoschema.InfoSchema) } + + txnCtxProvider := &sessiontxn.SimpleTxnContextProvider{ + InfoSchema: is, + } + if err = sessiontxn.GetTxnManager(s).SetContextProvider(txnCtxProvider); err != nil { + return nil, err + } + + executor.CountStmtNode(preparedStmt.PreparedAst.Stmt, s.sessionVars.InRestrictedSQL) + ok, err = s.IsCachedExecOk(ctx, preparedStmt) + if err != nil { + return nil, err + } + s.txn.onStmtStart(preparedStmt.SQLDigest.String()) + defer s.txn.onStmtEnd() + if ok { return s.cachedPlanExec(ctx, is, snapshotTS, stmtID, preparedStmt, args) } @@ -2843,10 +2883,10 @@ func (s *session) loadCommonGlobalVariablesIfNeeded() error { // PrepareTxnCtx starts a goroutine to begin a transaction if needed, and creates a new transaction context. // It is called before we execute a sql query. -func (s *session) PrepareTxnCtx(ctx context.Context) { +func (s *session) PrepareTxnCtx(ctx context.Context) error { s.currentCtx = ctx if s.txn.validOrPending() { - return + return nil } is := s.GetInfoSchema() @@ -2861,6 +2901,11 @@ func (s *session) PrepareTxnCtx(ctx context.Context) { s.sessionVars.TxnCtx.IsPessimistic = true } } + + txnCtxProvider := &sessiontxn.SimpleTxnContextProvider{ + InfoSchema: is.(infoschema.InfoSchema), + } + return sessiontxn.GetTxnManager(s).SetContextProvider(txnCtxProvider) } // PrepareTSFuture uses to try to get ts future. diff --git a/session/session_test.go b/session/session_test.go index 4602758eeaa28..e70660f172111 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -3058,7 +3058,8 @@ func (s *testSchemaSuite) TestDisableTxnAutoRetry(c *C) { // session 1 starts a transaction early. // execute a select statement to clear retry history. tk1.MustExec("select 1") - tk1.Se.PrepareTxnCtx(context.Background()) + err = tk1.Se.PrepareTxnCtx(context.Background()) + c.Assert(err, IsNil) // session 2 update the value. tk2.MustExec("update no_retry set id = 4") // AutoCommit update will retry, so it would not fail. diff --git a/session/txnmanager.go b/session/txnmanager.go new file mode 100644 index 0000000000000..0e47a8cf7406a --- /dev/null +++ b/session/txnmanager.go @@ -0,0 +1,62 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package session + +import ( + "github.com/pingcap/tidb/infoschema" + "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/sessiontxn" +) + +func init() { + sessiontxn.GetTxnManager = getTxnManager +} + +func getTxnManager(sctx sessionctx.Context) sessiontxn.TxnManager { + if manager, ok := sctx.GetSessionVars().TxnManager.(sessiontxn.TxnManager); ok { + return manager + } + + manager := newTxnManager(sctx) + sctx.GetSessionVars().TxnManager = manager + return manager +} + +// txnManager implements sessiontxn.TxnManager +type txnManager struct { + sctx sessionctx.Context + + ctxProvider sessiontxn.TxnContextProvider +} + +func newTxnManager(sctx sessionctx.Context) *txnManager { + return &txnManager{sctx: sctx} +} + +func (m *txnManager) GetTxnInfoSchema() infoschema.InfoSchema { + if m.ctxProvider == nil { + return nil + } + return m.ctxProvider.GetTxnInfoSchema() +} + +func (m *txnManager) GetContextProvider() sessiontxn.TxnContextProvider { + return m.ctxProvider +} + +func (m *txnManager) SetContextProvider(provider sessiontxn.TxnContextProvider) error { + m.ctxProvider = provider + return nil +} diff --git a/sessionctx/context.go b/sessionctx/context.go index 2f9a50aa211f6..bc2237e5b9987 100644 --- a/sessionctx/context.go +++ b/sessionctx/context.go @@ -70,6 +70,7 @@ type Context interface { // ClearValue clears the value associated with this context for key. ClearValue(key fmt.Stringer) + // Deprecated: Use TxnManager.GetTxnInfoSchema to get the current schema in session GetInfoSchema() InfoschemaMetaVersion GetSessionVars() *variable.SessionVars diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index d03cbdce86fde..3b0c8f33402e9 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -474,6 +474,9 @@ type SessionVars struct { // TxnCtx Should be reset on transaction finished. TxnCtx *TransactionContext + // TxnManager is used to manage txn context in session + TxnManager interface{} + // KVVars is the variables for KV storage. KVVars *tikvstore.Variables diff --git a/sessiontxn/failpoint.go b/sessiontxn/failpoint.go new file mode 100644 index 0000000000000..1d0a832de1083 --- /dev/null +++ b/sessiontxn/failpoint.go @@ -0,0 +1,74 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package sessiontxn + +import ( + "fmt" + + "github.com/pingcap/tidb/infoschema" + "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/util/stringutil" +) + +// AssertRecordsKey is used to save failPoint invoke records +// Only for test +var AssertRecordsKey stringutil.StringerStr = "assertTxnManagerRecords" + +// AssertTxnInfoSchemaKey is used to set the expected infoschema that should be check in failPoint +// Only for test +var AssertTxnInfoSchemaKey stringutil.StringerStr = "assertTxnInfoSchemaKey" + +// AssertTxnInfoSchemaAfterRetryKey is used to set the expected infoschema that should be check in failPoint after retry +// Only for test +var AssertTxnInfoSchemaAfterRetryKey stringutil.StringerStr = "assertTxnInfoSchemaAfterRetryKey" + +// RecordAssert is used only for test +func RecordAssert(sctx sessionctx.Context, name string, value interface{}) { + records, ok := sctx.Value(AssertRecordsKey).(map[string]interface{}) + if !ok { + records = make(map[string]interface{}) + sctx.SetValue(AssertRecordsKey, records) + } + records[name] = value +} + +// AssertTxnManagerInfoSchema is used only for test +func AssertTxnManagerInfoSchema(sctx sessionctx.Context, is interface{}) { + assertVersion := func(expected interface{}) { + if expected == nil { + return + } + + expectVer := expected.(infoschema.InfoSchema).SchemaMetaVersion() + gotVer := GetTxnManager(sctx).GetTxnInfoSchema().SchemaMetaVersion() + if gotVer != expectVer { + panic(fmt.Sprintf("Txn schema version not match, expect:%d, got:%d", expectVer, gotVer)) + } + } + + if localTables := sctx.GetSessionVars().LocalTemporaryTables; localTables != nil { + got, ok := GetTxnManager(sctx).GetTxnInfoSchema().(*infoschema.TemporaryTableAttachedInfoSchema) + if !ok { + panic("Expected to be a TemporaryTableAttachedInfoSchema") + } + + if got.LocalTemporaryTables != localTables { + panic("Local tables should be the same with the one in session") + } + } + + assertVersion(is) + assertVersion(sctx.Value(AssertTxnInfoSchemaKey)) +} diff --git a/sessiontxn/interface.go b/sessiontxn/interface.go new file mode 100644 index 0000000000000..fc7357ad10d55 --- /dev/null +++ b/sessiontxn/interface.go @@ -0,0 +1,59 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package sessiontxn + +import ( + "github.com/pingcap/tidb/infoschema" + "github.com/pingcap/tidb/sessionctx" +) + +// TxnContextProvider provides txn context +type TxnContextProvider interface { + // Initialize the provider with session context + Initialize(sctx sessionctx.Context) error + // GetTxnInfoSchema returns the information schema used by txn + GetTxnInfoSchema() infoschema.InfoSchema +} + +// SimpleTxnContextProvider implements TxnContextProvider +// It is only used in refactor stage +// TODO: remove it after refactor finished +type SimpleTxnContextProvider struct { + InfoSchema infoschema.InfoSchema +} + +// Initialize the provider with session context +func (p *SimpleTxnContextProvider) Initialize(_ sessionctx.Context) error { + return nil +} + +// GetTxnInfoSchema returns the information schema used by txn +func (p *SimpleTxnContextProvider) GetTxnInfoSchema() infoschema.InfoSchema { + return p.InfoSchema +} + +// TxnManager is an interface providing txn context management in session +type TxnManager interface { + // GetTxnInfoSchema returns the information schema used by txn + GetTxnInfoSchema() infoschema.InfoSchema + + // GetContextProvider returns the current TxnContextProvider + GetContextProvider() TxnContextProvider + // SetContextProvider sets the context provider + SetContextProvider(provider TxnContextProvider) error +} + +// GetTxnManager returns the TxnManager object from session context +var GetTxnManager func(sctx sessionctx.Context) TxnManager diff --git a/sessiontxn/txn_context_serial_test.go b/sessiontxn/txn_context_serial_test.go new file mode 100644 index 0000000000000..099af86a0e689 --- /dev/null +++ b/sessiontxn/txn_context_serial_test.go @@ -0,0 +1,706 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package sessiontxn_test + +import ( + "context" + "fmt" + "testing" + "time" + + "github.com/pingcap/failpoint" + "github.com/pingcap/tidb/domain" + "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/sessiontxn" + "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/util/testbridge" + "github.com/stretchr/testify/require" + "go.uber.org/goleak" +) + +func TestMain(m *testing.M) { + testbridge.WorkaroundGoCheckFlags() + opts := []goleak.Option{ + goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), + } + goleak.VerifyTestMain(m, opts...) +} + +func setupTxnContextTest(t *testing.T) (kv.Storage, *domain.Domain, func()) { + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/assertTxnManagerInCompile", "return")) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/assertTxnManagerInRebuildPlan", "return")) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/assertTxnManagerAfterBuildExecutor", "return")) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/assertTxnManagerAfterPessimisticLockErrorRetry", "return")) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/assertTxnManagerInShortPointGetPlan", "return")) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/session/assertTxnManagerInRunStmt", "return")) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/session/assertTxnManagerInPreparedStmtExec", "return")) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/session/assertTxnManagerInCachedPlanExec", "return")) + + store, do, clean := testkit.CreateMockStoreAndDomain(t) + + tk := testkit.NewTestKit(t, store) + tk.Session().SetValue(sessiontxn.AssertRecordsKey, nil) + tk.Session().SetValue(sessiontxn.AssertTxnInfoSchemaKey, nil) + + tk.MustExec("use test") + tk.MustExec("drop table if exists t1,t2") + + tk.MustExec("create table t1 (id int primary key, v int)") + tk.MustExec("insert into t1 values(1, 10)") + + tk.MustExec("create table t2 (id int)") + + tk.MustExec("create temporary table tmp (id int)") + tk.MustExec("insert into tmp values(10)") + + return store, do, func() { + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/assertTxnManagerInCompile")) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/assertTxnManagerInRebuildPlan")) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/assertTxnManagerAfterBuildExecutor")) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/assertTxnManagerAfterPessimisticLockErrorRetry")) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/assertTxnManagerInShortPointGetPlan")) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/session/assertTxnManagerInRunStmt")) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/session/assertTxnManagerInPreparedStmtExec")) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/session/assertTxnManagerInCachedPlanExec")) + + tk.Session().SetValue(sessiontxn.AssertRecordsKey, nil) + tk.Session().SetValue(sessiontxn.AssertTxnInfoSchemaKey, nil) + tk.Session().SetValue(sessiontxn.AssertTxnInfoSchemaAfterRetryKey, nil) + clean() + } +} + +func checkAssertRecordExits(t *testing.T, se sessionctx.Context, name string) { + records, ok := se.Value(sessiontxn.AssertRecordsKey).(map[string]interface{}) + require.True(t, ok, fmt.Sprintf("'%s' not in record, maybe failpoint not enabled?", name)) + _, ok = records[name] + require.True(t, ok, fmt.Sprintf("'%s' not in record", name)) +} + +func doWithCheckPath(t *testing.T, se sessionctx.Context, names []string, do func()) { + se.SetValue(sessiontxn.AssertRecordsKey, nil) + do() + for _, name := range names { + checkAssertRecordExits(t, se, name) + } +} + +var normalPathRecords = []string{ + "assertTxnManagerInCompile", + "assertTxnManagerInRunStmt", + "assertTxnManagerAfterBuildExecutor", +} + +func TestTxnContextForSimpleCases(t *testing.T) { + store, do, deferFunc := setupTxnContextTest(t) + defer deferFunc() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + se := tk.Session() + + tk2 := testkit.NewTestKit(t, store) + tk2.MustExec("use test") + + is1 := do.InfoSchema() + se.SetValue(sessiontxn.AssertTxnInfoSchemaKey, is1) + // test for write + doWithCheckPath(t, se, normalPathRecords, func() { + tk.MustExec("insert into t2 (id) values(3)") + }) + // test for select + doWithCheckPath(t, se, normalPathRecords, func() { + tk.MustQuery("select * from t1 where id=1").Check(testkit.Rows("1 10")) + }) + // test for select for update + doWithCheckPath(t, se, normalPathRecords, func() { + tk.MustQuery("select * from t1 where id=1 for update").Check(testkit.Rows("1 10")) + }) + + tk2.MustExec("alter table t2 add column(c1 int)") + is2 := do.InfoSchema() + require.True(t, is2.SchemaMetaVersion() > is1.SchemaMetaVersion()) + + se.SetValue(sessiontxn.AssertTxnInfoSchemaKey, is2) + doWithCheckPath(t, se, normalPathRecords, func() { + tk.MustQuery("select * from t1 where id=1").Check(testkit.Rows("1 10")) + }) +} + +func TestTxnContextInExplicitTxn(t *testing.T) { + store, do, deferFunc := setupTxnContextTest(t) + defer deferFunc() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + se := tk.Session() + + tk2 := testkit.NewTestKit(t, store) + tk2.MustExec("use test") + + is1 := do.InfoSchema() + se.SetValue(sessiontxn.AssertTxnInfoSchemaKey, is1) + + tk.MustExec("begin") + // test for write + doWithCheckPath(t, se, normalPathRecords, func() { + tk.MustExec("insert into t2 (id) values(2)") + }) + // test for select + doWithCheckPath(t, se, normalPathRecords, func() { + tk.MustQuery("select * from t1 where id=1").Check(testkit.Rows("1 10")) + }) + // test for select for update + doWithCheckPath(t, se, normalPathRecords, func() { + tk.MustQuery("select * from t1 where id=1 for update").Check(testkit.Rows("1 10")) + }) + + // info schema changed when txn not finish, the info schema in old txn should not change + tk2.MustExec("alter table t2 add column(c1 int)") + is2 := do.InfoSchema() + require.True(t, is2.SchemaMetaVersion() > is1.SchemaMetaVersion()) + + // test for write + doWithCheckPath(t, se, normalPathRecords, func() { + tk.MustExec("insert into t2 (id) values(2)") + }) + // test for select + doWithCheckPath(t, se, normalPathRecords, func() { + tk.MustQuery("select * from t1 where id=1").Check(testkit.Rows("1 10")) + }) + // test for select for update + doWithCheckPath(t, se, normalPathRecords, func() { + tk.MustQuery("select * from t1 where id=1 for update").Check(testkit.Rows("1 10")) + }) + + doWithCheckPath(t, se, normalPathRecords, func() { + tk.MustExec("commit") + }) + + // the info schema in new txn should use the newest one + se.SetValue(sessiontxn.AssertTxnInfoSchemaKey, is2) + tk.MustExec("begin") + // test for write + doWithCheckPath(t, se, normalPathRecords, func() { + tk.MustExec("insert into t2 (id) values(2)") + }) + // test for select + doWithCheckPath(t, se, normalPathRecords, func() { + tk.MustQuery("select * from t1 where id=1").Check(testkit.Rows("1 10")) + }) + // test for select for update + doWithCheckPath(t, se, normalPathRecords, func() { + tk.MustQuery("select * from t1 where id=1 for update").Check(testkit.Rows("1 10")) + }) +} + +func TestTxnContextBeginInUnfinishedTxn(t *testing.T) { + store, do, deferFunc := setupTxnContextTest(t) + defer deferFunc() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + se := tk.Session() + + tk2 := testkit.NewTestKit(t, store) + tk2.MustExec("use test") + + is1 := do.InfoSchema() + se.SetValue(sessiontxn.AssertTxnInfoSchemaKey, is1) + tk.MustExec("begin") + + doWithCheckPath(t, se, normalPathRecords, func() { + tk.MustQuery("select * from t1 where id=1").Check(testkit.Rows("1 10")) + }) + + tk2.MustExec("alter table t2 add column(c1 int)") + is2 := do.InfoSchema() + require.True(t, is2.SchemaMetaVersion() > is1.SchemaMetaVersion()) + + doWithCheckPath(t, se, normalPathRecords, func() { + tk.MustQuery("select * from t1 where id=1").Check(testkit.Rows("1 10")) + }) + + tk.MustExec("begin") + se.SetValue(sessiontxn.AssertTxnInfoSchemaKey, is2) + doWithCheckPath(t, se, normalPathRecords, func() { + tk.MustQuery("select * from t1 where id=1").Check(testkit.Rows("1 10")) + }) + tk.MustExec("rollback") +} + +func TestTxnContextWithAutocommitFalse(t *testing.T) { + store, do, deferFunc := setupTxnContextTest(t) + defer deferFunc() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + se := tk.Session() + + tk2 := testkit.NewTestKit(t, store) + tk2.MustExec("use test") + + is1 := do.InfoSchema() + se.SetValue(sessiontxn.AssertTxnInfoSchemaKey, is1) + tk.MustExec("begin") + + tk.MustExec("set autocommit=0") + se.SetValue(sessiontxn.AssertTxnInfoSchemaKey, do.InfoSchema()) + // test for write + doWithCheckPath(t, se, normalPathRecords, func() { + tk.MustExec("insert into t2 (id) values(2)") + }) + + // schema change should not affect because it is in txn + tk2.MustExec("alter table t2 add column(c1 int)") + + // test for select + doWithCheckPath(t, se, normalPathRecords, func() { + tk.MustQuery("select * from t1 where id=1").Check(testkit.Rows("1 10")) + }) + // test for select for update + doWithCheckPath(t, se, normalPathRecords, func() { + tk.MustQuery("select * from t1 where id=1 for update").Check(testkit.Rows("1 10")) + }) + tk.MustExec("rollback") +} + +func TestTxnContextInRC(t *testing.T) { + store, do, deferFunc := setupTxnContextTest(t) + defer deferFunc() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + se := tk.Session() + + tk2 := testkit.NewTestKit(t, store) + tk2.MustExec("use test") + + is1 := do.InfoSchema() + tk.MustExec("set tx_isolation = 'READ-COMMITTED'") + se.SetValue(sessiontxn.AssertTxnInfoSchemaKey, is1) + doWithCheckPath(t, se, normalPathRecords, func() { + tk.MustQuery("select * from t1 where id=1").Check(testkit.Rows("1 10")) + }) + + tk.MustExec("begin pessimistic") + + // schema change should not affect even in rc isolation + se.SetValue(sessiontxn.AssertTxnInfoSchemaKey, nil) + tk2.MustExec("alter table t2 add column(c1 int)") + + se.SetValue(sessiontxn.AssertTxnInfoSchemaKey, is1) + // test for write + doWithCheckPath(t, se, normalPathRecords, func() { + tk.MustExec("insert into t2 (id) values(2)") + }) + + // test for select + doWithCheckPath(t, se, normalPathRecords, func() { + tk.MustQuery("select * from t1 where id=1").Check(testkit.Rows("1 10")) + }) + + tk2.MustExec("update t1 set v=11 where id=1") + + // test for select + doWithCheckPath(t, se, normalPathRecords, func() { + tk.MustQuery("select * from t1 where id=1").Check(testkit.Rows("1 11")) + }) + + // test for select for update + doWithCheckPath(t, se, normalPathRecords, func() { + tk.MustQuery("select * from t1 where id=1 for update").Check(testkit.Rows("1 11")) + }) + + tk.MustExec("rollback") +} + +func TestTxnContextInPessimisticKeyConflict(t *testing.T) { + store, do, deferFunc := setupTxnContextTest(t) + defer deferFunc() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + se := tk.Session() + is1 := do.InfoSchema() + + tk.MustExec("begin pessimistic") + + // trigger retry + tk2 := testkit.NewTestKit(t, store) + tk2.MustExec("use test") + tk2.MustExec("update t1 set v=11 where id=1") + tk2.MustExec("alter table t2 add column(c1 int)") + + se.SetValue(sessiontxn.AssertTxnInfoSchemaKey, is1) + path := append([]string{"assertTxnManagerAfterPessimisticLockErrorRetry"}, normalPathRecords...) + doWithCheckPath(t, se, path, func() { + tk.MustExec("update t1 set v=12 where id=1") + }) + + tk.MustExec("rollback") +} + +func TestTxnContextInOptimisticRetry(t *testing.T) { + store, do, deferFunc := setupTxnContextTest(t) + defer deferFunc() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set @@tidb_disable_txn_auto_retry=0") + se := tk.Session() + is1 := do.InfoSchema() + + tk.MustExec("begin optimistic") + + // trigger retry + tk2 := testkit.NewTestKit(t, store) + tk2.MustExec("use test") + tk2.MustExec("update t1 set v=11 where id=1") + tk2.MustExec("alter table t2 add column(c1 int)") + + tk.MustExec("update t1 set v=12 where id=1") + + // check retry context + path := append([]string{"assertTxnManagerInRebuildPlan"}, normalPathRecords...) + se.SetValue(sessiontxn.AssertTxnInfoSchemaKey, is1) + se.SetValue(sessiontxn.AssertTxnInfoSchemaAfterRetryKey, do.InfoSchema()) + doWithCheckPath(t, se, path, func() { + tk.MustExec("commit") + }) + + tk.MustQuery("select * from t1 where id=1").Check(testkit.Rows("1 12")) +} + +func TestTxnContextForHistoricalRead(t *testing.T) { + store, do, deferFunc := setupTxnContextTest(t) + defer deferFunc() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + se := tk.Session() + + safePoint := "20160102-15:04:05 -0700" + tk.MustExec(fmt.Sprintf(`INSERT INTO mysql.tidb VALUES ('tikv_gc_safe_point', '%s', '') ON DUPLICATE KEY UPDATE variable_value = '%s', comment=''`, safePoint, safePoint)) + + is1 := do.InfoSchema() + tk.MustExec("set @a=now(6)") + // change schema + tk.MustExec("alter table t2 add column(c1 int)") + tk.MustExec("update t1 set v=11 where id=1") + + tk.MustExec("set @@tidb_snapshot=@a") + se.SetValue(sessiontxn.AssertTxnInfoSchemaKey, is1) + doWithCheckPath(t, se, normalPathRecords, func() { + tk.MustQuery("select * from t1 where id=1").Check(testkit.Rows("1 10")) + }) + + doWithCheckPath(t, se, normalPathRecords, func() { + tk.MustQuery("select * from t1 where id=1 for update").Check(testkit.Rows("1 10")) + }) + + se.SetValue(sessiontxn.AssertTxnInfoSchemaKey, nil) + tk.MustExec("set @@tidb_snapshot=''") + tk.MustExec("begin") + + se.SetValue(sessiontxn.AssertTxnInfoSchemaKey, do.InfoSchema()) + doWithCheckPath(t, se, normalPathRecords, func() { + tk.MustQuery("select * from t1 where id=1").Check(testkit.Rows("1 11")) + }) + + doWithCheckPath(t, se, normalPathRecords, func() { + tk.MustQuery("select * from t1 where id=1 for update").Check(testkit.Rows("1 11")) + }) + + se.SetValue(sessiontxn.AssertTxnInfoSchemaKey, nil) + tk.MustExec("set @@tidb_snapshot=@a") + se.SetValue(sessiontxn.AssertTxnInfoSchemaKey, is1) + + doWithCheckPath(t, se, normalPathRecords, func() { + tk.MustQuery("select * from t1 where id=1").Check(testkit.Rows("1 10")) + }) + + doWithCheckPath(t, se, normalPathRecords, func() { + tk.MustQuery("select * from t1 where id=1 for update").Check(testkit.Rows("1 11")) + }) + + tk.MustExec("rollback") +} + +func TestTxnContextForStaleRead(t *testing.T) { + store, do, deferFunc := setupTxnContextTest(t) + defer deferFunc() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + se := tk.Session() + + safePoint := "20160102-15:04:05 -0700" + tk.MustExec(fmt.Sprintf(`INSERT INTO mysql.tidb VALUES ('tikv_gc_safe_point', '%s', '') ON DUPLICATE KEY UPDATE variable_value = '%s', comment=''`, safePoint, safePoint)) + + is1 := do.InfoSchema() + tk.MustExec("set @a=now(6)") + time.Sleep(time.Millisecond * 1200) + + // change schema + tk.MustExec("alter table t2 add column(c1 int)") + tk.MustExec("update t1 set v=11 where id=1") + + // @@tidb_read_staleness + tk.MustExec("set @@tidb_read_staleness=-1") + se.SetValue(sessiontxn.AssertTxnInfoSchemaKey, is1) + doWithCheckPath(t, se, normalPathRecords, func() { + tk.MustQuery("select * from t1 as of timestamp @a").Check(testkit.Rows("1 10")) + }) + se.SetValue(sessiontxn.AssertTxnInfoSchemaKey, nil) + tk.MustExec("set @@tidb_read_staleness=''") + + // select ... as of ... + se.SetValue(sessiontxn.AssertTxnInfoSchemaKey, is1) + doWithCheckPath(t, se, normalPathRecords, func() { + tk.MustQuery("select * from t1 as of timestamp @a").Check(testkit.Rows("1 10")) + }) + + // @@tx_read_ts + se.SetValue(sessiontxn.AssertTxnInfoSchemaKey, nil) + tk.MustExec("set @@tx_read_ts=@a") + se.SetValue(sessiontxn.AssertTxnInfoSchemaKey, is1) + doWithCheckPath(t, se, normalPathRecords, func() { + tk.MustQuery("select * from t1 where id=1").Check(testkit.Rows("1 10")) + }) + se.SetValue(sessiontxn.AssertTxnInfoSchemaKey, do.InfoSchema()) + doWithCheckPath(t, se, normalPathRecords, func() { + tk.MustQuery("select * from t1 where id=1").Check(testkit.Rows("1 11")) + }) + + // txn begin with @tx_read_ts + se.SetValue(sessiontxn.AssertTxnInfoSchemaKey, nil) + tk.MustExec("set @@tx_read_ts=@a") + tk.MustExec("begin") + se.SetValue(sessiontxn.AssertTxnInfoSchemaKey, is1) + doWithCheckPath(t, se, normalPathRecords, func() { + tk.MustQuery("select * from t1 where id=1").Check(testkit.Rows("1 10")) + }) + tk.MustExec("rollback") + se.SetValue(sessiontxn.AssertTxnInfoSchemaKey, do.InfoSchema()) + doWithCheckPath(t, se, normalPathRecords, func() { + tk.MustQuery("select * from t1 where id=1").Check(testkit.Rows("1 11")) + }) + + // txn begin ... as of ... + se.SetValue(sessiontxn.AssertTxnInfoSchemaKey, nil) + tk.MustExec("start transaction read only as of timestamp @a") + se.SetValue(sessiontxn.AssertTxnInfoSchemaKey, is1) + doWithCheckPath(t, se, normalPathRecords, func() { + tk.MustQuery("select * from t1 where id=1").Check(testkit.Rows("1 10")) + }) + tk.MustExec("rollback") + se.SetValue(sessiontxn.AssertTxnInfoSchemaKey, do.InfoSchema()) + doWithCheckPath(t, se, normalPathRecords, func() { + tk.MustQuery("select * from t1 where id=1").Check(testkit.Rows("1 11")) + }) +} + +func TestTxnContextForPrepareExecute(t *testing.T) { + store, do, deferFunc := setupTxnContextTest(t) + defer deferFunc() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + se := tk.Session() + + stmtID, _, _, err := se.PrepareStmt("select * from t1 where id=1") + require.NoError(t, err) + + is1 := do.InfoSchema() + se.SetValue(sessiontxn.AssertTxnInfoSchemaKey, is1) + + // Test prepare/execute in SQL + doWithCheckPath(t, se, normalPathRecords, func() { + tk.MustExec("prepare s from 'select * from t1 where id=1'") + }) + doWithCheckPath(t, se, normalPathRecords, func() { + tk.MustQuery("execute s").Check(testkit.Rows("1 10")) + }) + + // Test ExecutePreparedStmt + path := append([]string{"assertTxnManagerInPreparedStmtExec"}, normalPathRecords...) + doWithCheckPath(t, se, path, func() { + rs, err := se.ExecutePreparedStmt(context.TODO(), stmtID, nil) + require.NoError(t, err) + tk.ResultSetToResult(rs, fmt.Sprintf("%v", rs)).Check(testkit.Rows("1 10")) + }) + + // Test PlanCache + path = []string{"assertTxnManagerInCachedPlanExec", "assertTxnManagerInShortPointGetPlan"} + doWithCheckPath(t, se, path, func() { + rs, err := se.ExecutePreparedStmt(context.TODO(), stmtID, nil) + require.NoError(t, err) + tk.ResultSetToResult(rs, fmt.Sprintf("%v", rs)).Check(testkit.Rows("1 10")) + }) + + // In txn + se.SetValue(sessiontxn.AssertTxnInfoSchemaKey, nil) + tk.MustExec("begin") + + //change schema + tk2 := testkit.NewTestKit(t, store) + tk2.MustExec("use test") + tk2.MustExec("alter table t2 add column(c1 int)") + tk2.MustExec("update t1 set v=11 where id=1") + + se.SetValue(sessiontxn.AssertTxnInfoSchemaKey, is1) + doWithCheckPath(t, se, normalPathRecords, func() { + tk.MustExec("prepare s from 'select * from t1 where id=1'") + }) + doWithCheckPath(t, se, normalPathRecords, func() { + tk.MustQuery("execute s").Check(testkit.Rows("1 10")) + }) + path = append([]string{"assertTxnManagerInPreparedStmtExec"}, normalPathRecords...) + doWithCheckPath(t, se, path, func() { + rs, err := se.ExecutePreparedStmt(context.TODO(), stmtID, nil) + require.NoError(t, err) + tk.ResultSetToResult(rs, fmt.Sprintf("%v", rs)).Check(testkit.Rows("1 10")) + }) + + tk.MustExec("rollback") +} + +func TestTxnContextForStaleReadInPrepare(t *testing.T) { + store, do, deferFunc := setupTxnContextTest(t) + defer deferFunc() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + se := tk.Session() + + is1 := do.InfoSchema() + tk.MustExec("set @a=now(6)") + tk.MustExec("prepare s1 from 'select * from t1 where id=1'") + tk.MustExec("prepare s2 from 'select * from t1 as of timestamp @a where id=1 '") + + stmtID1, _, _, err := se.PrepareStmt("select * from t1 where id=1") + require.NoError(t, err) + + stmtID2, _, _, err := se.PrepareStmt("select * from t1 as of timestamp @a where id=1 ") + require.NoError(t, err) + + //change schema + tk.MustExec("use test") + tk.MustExec("alter table t2 add column(c1 int)") + tk.MustExec("update t1 set v=11 where id=1") + + tk.MustExec("set @@tx_read_ts=@a") + stmtID3, _, _, err := se.PrepareStmt("select * from t1 where id=1 ") + require.NoError(t, err) + tk.MustExec("set @@tx_read_ts=''") + + tk.MustExec("set @@tx_read_ts=@a") + tk.MustExec("prepare s3 from 'select * from t1 where id=1 '") + tk.MustExec("set @@tx_read_ts=''") + + // tx_read_ts + tk.MustExec("set @@tx_read_ts=@a") + se.SetValue(sessiontxn.AssertTxnInfoSchemaKey, is1) + path := append([]string{"assertTxnManagerInPreparedStmtExec"}, normalPathRecords...) + doWithCheckPath(t, se, path, func() { + rs, err := se.ExecutePreparedStmt(context.TODO(), stmtID1, nil) + require.NoError(t, err) + tk.ResultSetToResult(rs, fmt.Sprintf("%v", rs)).Check(testkit.Rows("1 10")) + }) + se.SetValue(sessiontxn.AssertTxnInfoSchemaKey, nil) + tk.MustExec("set @@tx_read_ts=''") + + tk.MustExec("set @@tx_read_ts=@a") + se.SetValue(sessiontxn.AssertTxnInfoSchemaKey, is1) + doWithCheckPath(t, se, normalPathRecords, func() { + tk.MustExec("execute s1") + }) + se.SetValue(sessiontxn.AssertTxnInfoSchemaKey, nil) + tk.MustExec("set @@tx_read_ts=''") + + // select ... as of ... + se.SetValue(sessiontxn.AssertTxnInfoSchemaKey, is1) + doWithCheckPath(t, se, path, func() { + rs, err := se.ExecutePreparedStmt(context.TODO(), stmtID2, nil) + require.NoError(t, err) + tk.ResultSetToResult(rs, fmt.Sprintf("%v", rs)).Check(testkit.Rows("1 10")) + }) + doWithCheckPath(t, se, normalPathRecords, func() { + tk.MustExec("execute s2") + }) + + // plan cache for stmtID2 + doWithCheckPath(t, se, []string{"assertTxnManagerInCachedPlanExec", "assertTxnManagerInShortPointGetPlan"}, func() { + rs, err := se.ExecutePreparedStmt(context.TODO(), stmtID2, nil) + require.NoError(t, err) + tk.ResultSetToResult(rs, fmt.Sprintf("%v", rs)).Check(testkit.Rows("1 10")) + }) + + // tx_read_ts in prepare + se.SetValue(sessiontxn.AssertTxnInfoSchemaKey, is1) + doWithCheckPath(t, se, path, func() { + rs, err := se.ExecutePreparedStmt(context.TODO(), stmtID3, nil) + require.NoError(t, err) + tk.ResultSetToResult(rs, fmt.Sprintf("%v", rs)).Check(testkit.Rows("1 10")) + }) + doWithCheckPath(t, se, normalPathRecords, func() { + tk.MustExec("execute s3") + }) + + // plan cache for stmtID3 + doWithCheckPath(t, se, []string{"assertTxnManagerInCachedPlanExec", "assertTxnManagerInShortPointGetPlan"}, func() { + rs, err := se.ExecutePreparedStmt(context.TODO(), stmtID3, nil) + require.NoError(t, err) + tk.ResultSetToResult(rs, fmt.Sprintf("%v", rs)).Check(testkit.Rows("1 10")) + }) +} + +func TestTxnContextPreparedStmtWithForUpdate(t *testing.T) { + store, do, deferFunc := setupTxnContextTest(t) + defer deferFunc() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + se := tk.Session() + + is1 := do.InfoSchema() + + stmtID1, _, _, err := se.PrepareStmt("select * from t1 where id=1 for update") + require.NoError(t, err) + tk.MustExec("prepare s from 'select * from t1 where id=1 for update'") + tk.MustExec("begin pessimistic") + + //change schema + tk2 := testkit.NewTestKit(t, store) + tk2.MustExec("use test") + tk2.MustExec("alter table t1 add column(c int default 100)") + tk2.MustExec("update t1 set v=11 where id=1") + + se.SetValue(sessiontxn.AssertTxnInfoSchemaKey, is1) + doWithCheckPath(t, se, normalPathRecords, func() { + tk.MustQuery("select * from t1 where id=1 for update").Check(testkit.Rows("1 11")) + }) + + se.SetValue(sessiontxn.AssertTxnInfoSchemaKey, do.InfoSchema()) + path := append([]string{"assertTxnManagerInPreparedStmtExec"}, normalPathRecords...) + doWithCheckPath(t, se, path, func() { + rs, err := se.ExecutePreparedStmt(context.TODO(), stmtID1, nil) + require.NoError(t, err) + tk.ResultSetToResult(rs, fmt.Sprintf("%v", rs)).Check(testkit.Rows("1 11 100")) + }) + + doWithCheckPath(t, se, normalPathRecords, func() { + tk.MustQuery("execute s").Check(testkit.Rows("1 11 100")) + }) + + se.SetValue(sessiontxn.AssertTxnInfoSchemaKey, nil) + tk.MustExec("rollback") +} From 46e67db084e4534ef12a48e9ebcea840ca70adef Mon Sep 17 00:00:00 2001 From: tangenta Date: Wed, 22 Dec 2021 14:35:47 +0800 Subject: [PATCH 15/44] Makefile: add `t.Parallel` check to ensure tests are run in serial (#30869) --- Makefile | 9 ++++++++- br/pkg/task/backup_test.go | 4 ---- br/pkg/task/common_test.go | 5 ----- br/pkg/task/restore_test.go | 2 -- br/pkg/utils/backoff_test.go | 4 ---- table/tables/tables_test.go | 1 - 6 files changed, 8 insertions(+), 17 deletions(-) diff --git a/Makefile b/Makefile index 3e7fb1f5be216..d49b0db744da4 100644 --- a/Makefile +++ b/Makefile @@ -34,7 +34,7 @@ dev: checklist check explaintest devgotest gogenerate br_unit_test test_part_par # Install the check tools. check-setup:tools/bin/revive tools/bin/goword -check: fmt unconvert lint tidy testSuite check-static vet errdoc +check: fmt check-parallel unconvert lint tidy testSuite check-static vet errdoc fmt: @echo "gofmt (simplify)" @@ -75,6 +75,13 @@ testSuite: @echo "testSuite" ./tools/check/check_testSuite.sh +check-parallel: +# Make sure no tests are run in parallel to prevent possible unstable tests. +# See https://github.com/pingcap/tidb/pull/30692. + @! find . -name "*_test.go" -not -path "./vendor/*" -print0 | \ + xargs -0 grep -F -n "t.Parallel()" || \ + ! echo "Error: all the go tests should be run in serial." + clean: failpoint-disable $(GO) clean -i ./... diff --git a/br/pkg/task/backup_test.go b/br/pkg/task/backup_test.go index 6db001d609ef9..816d4837203d3 100644 --- a/br/pkg/task/backup_test.go +++ b/br/pkg/task/backup_test.go @@ -11,8 +11,6 @@ import ( ) func TestParseTSString(t *testing.T) { - t.Parallel() - var ( ts uint64 err error @@ -33,8 +31,6 @@ func TestParseTSString(t *testing.T) { } func TestParseCompressionType(t *testing.T) { - t.Parallel() - var ( ct backup.CompressionType err error diff --git a/br/pkg/task/common_test.go b/br/pkg/task/common_test.go index b85d973dea065..b124f6977b9fa 100644 --- a/br/pkg/task/common_test.go +++ b/br/pkg/task/common_test.go @@ -29,7 +29,6 @@ func (f fakeValue) Type() string { } func TestUrlNoQuery(t *testing.T) { - t.Parallel() flag := &pflag.Flag{ Name: flagStorage, Value: fakeValue("s3://some/what?secret=a123456789&key=987654321"), @@ -40,7 +39,6 @@ func TestUrlNoQuery(t *testing.T) { } func TestTiDBConfigUnchanged(t *testing.T) { - t.Parallel() cfg := config.GetGlobalConfig() restoreConfig := enableTiDBConfig() require.NotEqual(t, config.GetGlobalConfig(), cfg) @@ -49,7 +47,6 @@ func TestTiDBConfigUnchanged(t *testing.T) { } func TestStripingPDURL(t *testing.T) { - t.Parallel() nor1, err := normalizePDURL("https://pd:5432", true) require.NoError(t, err) require.Equal(t, "pd:5432", nor1) @@ -68,7 +65,6 @@ func TestStripingPDURL(t *testing.T) { } func TestCheckCipherKeyMatch(t *testing.T) { - t.Parallel() cases := []struct { CipherType encryptionpb.EncryptionMethod CipherKey string @@ -125,7 +121,6 @@ func TestCheckCipherKeyMatch(t *testing.T) { } func TestCheckCipherKey(t *testing.T) { - t.Parallel() cases := []struct { cipherKey string keyFile string diff --git a/br/pkg/task/restore_test.go b/br/pkg/task/restore_test.go index 1abadc3ae6ec4..8b583dcf66e95 100644 --- a/br/pkg/task/restore_test.go +++ b/br/pkg/task/restore_test.go @@ -10,8 +10,6 @@ import ( ) func TestRestoreConfigAdjust(t *testing.T) { - t.Parallel() - cfg := &RestoreConfig{} cfg.adjustRestoreConfig() diff --git a/br/pkg/utils/backoff_test.go b/br/pkg/utils/backoff_test.go index b0c0f640ab677..17aa3f72802e1 100644 --- a/br/pkg/utils/backoff_test.go +++ b/br/pkg/utils/backoff_test.go @@ -125,8 +125,6 @@ func TestPdBackoffWithRetryableError(t *testing.T) { } func TestNewImportSSTBackofferWithSucess(t *testing.T) { - t.Parallel() - var counter int backoffer := utils.NewImportSSTBackoffer() err := utils.WithRetry(context.Background(), func() error { @@ -142,8 +140,6 @@ func TestNewImportSSTBackofferWithSucess(t *testing.T) { } func TestNewDownloadSSTBackofferWithCancel(t *testing.T) { - t.Parallel() - var counter int backoffer := utils.NewDownloadSSTBackoffer() err := utils.WithRetry(context.Background(), func() error { diff --git a/table/tables/tables_test.go b/table/tables/tables_test.go index dcf3507d92c49..a3d13a3bfa89e 100644 --- a/table/tables/tables_test.go +++ b/table/tables/tables_test.go @@ -738,7 +738,6 @@ func TestViewColumns(t *testing.T) { } func TestConstraintCheckForOptimisticUntouched(t *testing.T) { - t.Parallel() store, clean := testkit.CreateMockStore(t) defer clean() From 9d1e29d7cf6388dab8e6774e3bc3f7bbc5a4d5a3 Mon Sep 17 00:00:00 2001 From: xhe Date: Wed, 22 Dec 2021 15:27:48 +0800 Subject: [PATCH 16/44] placement: remove isolationlevel (#30859) close pingcap/tidb#30858 --- ddl/placement/bundle_test.go | 2 +- ddl/placement/rule.go | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/ddl/placement/bundle_test.go b/ddl/placement/bundle_test.go index 225f17f398552..3c11559f06304 100644 --- a/ddl/placement/bundle_test.go +++ b/ddl/placement/bundle_test.go @@ -352,7 +352,7 @@ func (s *testBundleSuite) TestString(c *C) { c.Assert(err, IsNil) bundle.Rules = append(rules1, rules2...) - c.Assert(bundle.String(), Equals, `{"group_id":"TiDB_DDL_1","group_index":0,"group_override":false,"rules":[{"group_id":"","id":"","start_key":"","end_key":"","role":"voter","count":3,"label_constraints":[{"key":"zone","op":"in","values":["sh"]}],"location_labels":["region","zone","rack","host"],"isolation_level":"region"},{"group_id":"","id":"","start_key":"","end_key":"","role":"voter","count":4,"label_constraints":[{"key":"zone","op":"notIn","values":["sh"]},{"key":"zone","op":"in","values":["bj"]}],"location_labels":["region","zone","rack","host"],"isolation_level":"region"}]}`) + c.Assert(bundle.String(), Equals, `{"group_id":"TiDB_DDL_1","group_index":0,"group_override":false,"rules":[{"group_id":"","id":"","start_key":"","end_key":"","role":"voter","count":3,"label_constraints":[{"key":"zone","op":"in","values":["sh"]}],"location_labels":["region","zone","rack","host"]},{"group_id":"","id":"","start_key":"","end_key":"","role":"voter","count":4,"label_constraints":[{"key":"zone","op":"notIn","values":["sh"]},{"key":"zone","op":"in","values":["bj"]}],"location_labels":["region","zone","rack","host"]}]}`) c.Assert(failpoint.Enable("github.com/pingcap/tidb/ddl/placement/MockMarshalFailure", `return(true)`), IsNil) defer func() { diff --git a/ddl/placement/rule.go b/ddl/placement/rule.go index 216714789aec9..88cd5067153f8 100644 --- a/ddl/placement/rule.go +++ b/ddl/placement/rule.go @@ -58,7 +58,6 @@ func NewRule(role PeerRoleType, replicas uint64, cnst Constraints) *Rule { Count: int(replicas), Constraints: cnst, LocationLabels: []string{"region", "zone", "rack", "host"}, - IsolationLevel: "region", } } From beb5451ee3e97470fdc160e0c92f3967bf6e7127 Mon Sep 17 00:00:00 2001 From: Song Gao Date: Wed, 22 Dec 2021 15:47:47 +0800 Subject: [PATCH 17/44] planner: revise the optimize trace output (#30882) --- planner/core/logical_plan_trace_test.go | 70 ++++++++++---------- planner/core/optimizer.go | 8 +-- planner/core/plan.go | 8 +-- planner/core/rule_aggregation_elimination.go | 25 ++++--- planner/core/rule_aggregation_push_down.go | 32 ++++----- planner/core/rule_eliminate_projection.go | 16 +++-- planner/core/rule_join_elimination.go | 12 ++-- planner/core/rule_join_reorder.go | 12 ++-- planner/core/rule_max_min_eliminate.go | 34 +++++----- planner/core/rule_partition_processor.go | 29 +++++--- planner/core/rule_topn_push_down.go | 45 ++++++++++--- 11 files changed, 172 insertions(+), 119 deletions(-) diff --git a/planner/core/logical_plan_trace_test.go b/planner/core/logical_plan_trace_test.go index 0d4577bc1f107..c09ffb1f19b6c 100644 --- a/planner/core/logical_plan_trace_test.go +++ b/planner/core/logical_plan_trace_test.go @@ -138,8 +138,8 @@ func (s *testPlanSuite) TestSingleRuleTraceStep(c *C) { assertRuleName: "partition_processor", assertRuleSteps: []assertTraceStep{ { - assertReason: "Datasource[1] has multiple needed partitions[p1,p2] after pruning", - assertAction: "Datasource[1] becomes PartitionUnion[6] with children[TableScan[1],TableScan[1]]", + assertReason: "DataSource_1 has multiple needed partitions[p1,p2] after pruning", + assertAction: "DataSource_1 becomes PartitionUnion_6 with children[TableScan_1,TableScan_1]", }, }, }, @@ -149,8 +149,8 @@ func (s *testPlanSuite) TestSingleRuleTraceStep(c *C) { assertRuleName: "partition_processor", assertRuleSteps: []assertTraceStep{ { - assertReason: "Datasource[1] has one needed partition[p1] after pruning", - assertAction: "Datasource[1] becomes TableScan[1]", + assertReason: "DataSource_1 has one needed partition[p1] after pruning", + assertAction: "DataSource_1 becomes TableScan_1", }, }, }, @@ -160,8 +160,8 @@ func (s *testPlanSuite) TestSingleRuleTraceStep(c *C) { assertRuleName: "partition_processor", assertRuleSteps: []assertTraceStep{ { - assertReason: "Datasource[1] has multiple needed partitions[p1,p2] after pruning", - assertAction: "Datasource[1] becomes PartitionUnion[7] with children[TableScan[1],TableScan[1]]", + assertReason: "DataSource_1 has multiple needed partitions[p1,p2] after pruning", + assertAction: "DataSource_1 becomes PartitionUnion_7 with children[TableScan_1,TableScan_1]", }, }, }, @@ -171,8 +171,8 @@ func (s *testPlanSuite) TestSingleRuleTraceStep(c *C) { assertRuleName: "partition_processor", assertRuleSteps: []assertTraceStep{ { - assertReason: "Datasource[1] has one needed partition[p2] after pruning", - assertAction: "Datasource[1] becomes TableScan[1]", + assertReason: "DataSource_1 has one needed partition[p2] after pruning", + assertAction: "DataSource_1 becomes TableScan_1", }, }, }, @@ -182,8 +182,8 @@ func (s *testPlanSuite) TestSingleRuleTraceStep(c *C) { assertRuleName: "partition_processor", assertRuleSteps: []assertTraceStep{ { - assertReason: "Datasource[1] doesn't have needed partition table after pruning", - assertAction: "Datasource[1] becomes TableDual[5]", + assertReason: "DataSource_1 doesn't have needed partition table after pruning", + assertAction: "DataSource_1 becomes TableDual_5", }, }, }, @@ -193,8 +193,8 @@ func (s *testPlanSuite) TestSingleRuleTraceStep(c *C) { assertRuleName: "partition_processor", assertRuleSteps: []assertTraceStep{ { - assertReason: "Datasource[1] has multiple needed partitions[p1,p2] after pruning", - assertAction: "Datasource[1] becomes PartitionUnion[7] with children[TableScan[1],TableScan[1]]", + assertReason: "DataSource_1 has multiple needed partitions[p1,p2] after pruning", + assertAction: "DataSource_1 becomes PartitionUnion_7 with children[TableScan_1,TableScan_1]", }, }, }, @@ -204,8 +204,8 @@ func (s *testPlanSuite) TestSingleRuleTraceStep(c *C) { assertRuleName: "partition_processor", assertRuleSteps: []assertTraceStep{ { - assertReason: "Datasource[1] has one needed partition[p1] after pruning", - assertAction: "Datasource[1] becomes TableScan[1]", + assertReason: "DataSource_1 has one needed partition[p1] after pruning", + assertAction: "DataSource_1 becomes TableScan_1", }, }, }, @@ -242,7 +242,7 @@ func (s *testPlanSuite) TestSingleRuleTraceStep(c *C) { }, { assertReason: "[test.t.a] is a unique key", - assertAction: "aggregation is simplified to a projection", + assertAction: "Aggregation_2 is simplified to a Projection_4", }, }, }, @@ -252,8 +252,8 @@ func (s *testPlanSuite) TestSingleRuleTraceStep(c *C) { assertRuleName: "projection_eliminate", assertRuleSteps: []assertTraceStep{ { - assertAction: "Proj[2] is eliminated, Proj[3]'s expressions changed into[plus(1, plus(1, test.t.a))]", - assertReason: "Proj[3]'s child proj[2] is redundant", + assertAction: "Projection_2 is eliminated, Projection_3's expressions changed into[plus(1, plus(1, test.t.a))]", + assertReason: "Projection_3's child Projection_2 is redundant", }, }, }, @@ -263,8 +263,8 @@ func (s *testPlanSuite) TestSingleRuleTraceStep(c *C) { assertRuleName: "aggregation_push_down", assertRuleSteps: []assertTraceStep{ { - assertAction: "agg[6] pushed down across join[5], and join right path becomes agg[8]", - assertReason: "agg[6]'s functions[count(Column#38)] are decomposable with join", + assertAction: "Aggregation_6 pushed down across Join_5, and Join_5 right path becomes Aggregation_8", + assertReason: "Aggregation_6's functions[count(Column#38)] are decomposable with join", }, }, }, @@ -274,16 +274,16 @@ func (s *testPlanSuite) TestSingleRuleTraceStep(c *C) { assertRuleName: "aggregation_push_down", assertRuleSteps: []assertTraceStep{ { - assertAction: "agg[8] pushed down, and union[5]'s children changed into[[id:11,tp:Aggregation],[id:12,tp:Aggregation]]", - assertReason: "agg[8] functions[sum(Column#28)] are decomposable with union", + assertAction: "Aggregation_8 pushed down, and Union_5's children changed into[Aggregation_11,Aggregation_12]", + assertReason: "Aggregation_8 functions[sum(Column#28)] are decomposable with Union_5", }, { - assertAction: "proj[6] is eliminated, and agg[11]'s functions changed into[sum(test.t.c),firstrow(test.t.d)]", - assertReason: "Proj[6] is directly below an agg[11] and has no side effects", + assertAction: "Projection_6 is eliminated, and Aggregation_11's functions changed into[sum(test.t.c),firstrow(test.t.d)]", + assertReason: "Projection_6 is directly below an Aggregation_11 and has no side effects", }, { - assertAction: "proj[7] is eliminated, and agg[12]'s functions changed into[sum(test.t.a),firstrow(test.t.b)]", - assertReason: "Proj[7] is directly below an agg[12] and has no side effects", + assertAction: "Projection_7 is eliminated, and Aggregation_12's functions changed into[sum(test.t.a),firstrow(test.t.b)]", + assertReason: "Projection_7 is directly below an Aggregation_12 and has no side effects", }, }, }, @@ -293,16 +293,16 @@ func (s *testPlanSuite) TestSingleRuleTraceStep(c *C) { assertRuleName: "max_min_eliminate", assertRuleSteps: []assertTraceStep{ { - assertAction: "add sort[8],add limit[9] during eliminating agg[4] max function", - assertReason: "agg[4] has only one function[max] without group by, the columns in agg[4] should be sorted", + assertAction: "add Sort_8,add Limit_9 during eliminating Aggregation_4 max function", + assertReason: "Aggregation_4 has only one function[max] without group by, the columns in Aggregation_4 should be sorted", }, { - assertAction: "add sort[10],add limit[11] during eliminating agg[6] min function", - assertReason: "agg[6] has only one function[min] without group by, the columns in agg[6] should be sorted", + assertAction: "add Sort_10,add Limit_11 during eliminating Aggregation_6 min function", + assertReason: "Aggregation_6 has only one function[min] without group by, the columns in Aggregation_6 should be sorted", }, { - assertAction: "agg[2] splited into aggs[4,6], and add joins[12] to connect them during eliminating agg[2] multi min/max functions", - assertReason: "each column is sorted and can benefit from index/primary key in agg[4,6] and none of them has group by clause", + assertAction: "Aggregation_2 splited into [Aggregation_4,Aggregation_6], and add [Join_12] to connect them during eliminating Aggregation_2 multi min/max functions", + assertReason: "each column is sorted and can benefit from index/primary key in [Aggregation_4,Aggregation_6] and none of them has group by clause", }, }, }, @@ -312,8 +312,8 @@ func (s *testPlanSuite) TestSingleRuleTraceStep(c *C) { assertRuleName: "max_min_eliminate", assertRuleSteps: []assertTraceStep{ { - assertAction: "add selection[4],add sort[5],add limit[6] during eliminating agg[2] max function", - assertReason: "agg[2] has only one function[max] without group by, the columns in agg[2] shouldn't be NULL and needs NULL to be filtered out, the columns in agg[2] should be sorted", + assertAction: "add Selection_4,add Sort_5,add Limit_6 during eliminating Aggregation_2 max function", + assertReason: "Aggregation_2 has only one function[max] without group by, the columns in Aggregation_2 shouldn't be NULL and needs NULL to be filtered out, the columns in Aggregation_2 should be sorted", }, }, }, @@ -323,7 +323,7 @@ func (s *testPlanSuite) TestSingleRuleTraceStep(c *C) { assertRuleName: "outer_join_eliminate", assertRuleSteps: []assertTraceStep{ { - assertAction: "Outer join[3] is eliminated and become DataSource[1]", + assertAction: "Outer Join_3 is eliminated and become DataSource_1", assertReason: "The columns[test.t.b,test.t.c] are from outer table, and the inner join keys[test.t.a] are unique", }, }, @@ -334,7 +334,7 @@ func (s *testPlanSuite) TestSingleRuleTraceStep(c *C) { assertRuleName: "outer_join_eliminate", assertRuleSteps: []assertTraceStep{ { - assertAction: "Outer join[3] is eliminated and become DataSource[1]", + assertAction: "Outer Join_3 is eliminated and become DataSource_1", assertReason: "The columns[test.t.a,test.t.b] in agg are from outer table, and the agg functions are duplicate agnostic", }, }, diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index 89b156e632cea..36ef3b376bf55 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -107,21 +107,21 @@ func (op *logicalOptimizeOp) appendBeforeRuleOptimize(index int, name string, be if op.tracer == nil { return } - op.tracer.AppendRuleTracerBeforeRuleOptimize(index, name, before.buildLogicalPlanTrace(before)) + op.tracer.AppendRuleTracerBeforeRuleOptimize(index, name, before.buildLogicalPlanTrace()) } -func (op *logicalOptimizeOp) appendStepToCurrent(id int, tp, reason, action string) { +func (op *logicalOptimizeOp) appendStepToCurrent(id int, tp string, reason, action func() string) { if op.tracer == nil { return } - op.tracer.AppendRuleTracerStepToCurrent(id, tp, reason, action) + op.tracer.AppendRuleTracerStepToCurrent(id, tp, reason(), action()) } func (op *logicalOptimizeOp) recordFinalLogicalPlan(final LogicalPlan) { if op.tracer == nil { return } - op.tracer.RecordFinalLogicalPlan(final.buildLogicalPlanTrace(final)) + op.tracer.RecordFinalLogicalPlan(final.buildLogicalPlanTrace()) } // logicalOptRule means a logical optimizing rule, which contains decorrelate, ppd, column pruning, etc. diff --git a/planner/core/plan.go b/planner/core/plan.go index bd9d19a7de4e8..4c2c0c6ba93c5 100644 --- a/planner/core/plan.go +++ b/planner/core/plan.go @@ -308,7 +308,7 @@ type LogicalPlan interface { canPushToCop(store kv.StoreType) bool // buildLogicalPlanTrace clone necessary information from LogicalPlan - buildLogicalPlanTrace(p Plan) *tracing.LogicalPlanTrace + buildLogicalPlanTrace() *tracing.LogicalPlanTrace } // PhysicalPlan is a tree of the physical operators. @@ -382,10 +382,10 @@ func (p *baseLogicalPlan) ExplainInfo() string { } // buildLogicalPlanTrace implements LogicalPlan -func (p *baseLogicalPlan) buildLogicalPlanTrace(plan Plan) *tracing.LogicalPlanTrace { - planTrace := &tracing.LogicalPlanTrace{ID: p.ID(), TP: p.TP(), ExplainInfo: plan.ExplainInfo()} +func (p *baseLogicalPlan) buildLogicalPlanTrace() *tracing.LogicalPlanTrace { + planTrace := &tracing.LogicalPlanTrace{ID: p.ID(), TP: p.TP(), ExplainInfo: p.self.ExplainInfo()} for _, child := range p.Children() { - planTrace.Children = append(planTrace.Children, child.buildLogicalPlanTrace(child)) + planTrace.Children = append(planTrace.Children, child.buildLogicalPlanTrace()) } return planTrace } diff --git a/planner/core/rule_aggregation_elimination.go b/planner/core/rule_aggregation_elimination.go index 61d9e0f117e0d..9adca6936099d 100644 --- a/planner/core/rule_aggregation_elimination.go +++ b/planner/core/rule_aggregation_elimination.go @@ -67,7 +67,7 @@ func (a *aggregationEliminateChecker) tryToEliminateAggregation(agg *LogicalAggr // GroupByCols has unique key, so this aggregation can be removed. if ok, proj := ConvertAggToProj(agg, agg.schema); ok { proj.SetChildren(agg.children[0]) - appendAggregationEliminateTraceStep(agg, uniqueKey, opt) + appendAggregationEliminateTraceStep(agg, proj, uniqueKey, opt) return proj } } @@ -116,17 +116,26 @@ func (a *aggregationEliminateChecker) tryToEliminateDistinct(agg *LogicalAggrega } } -func appendAggregationEliminateTraceStep(agg *LogicalAggregation, uniqueKey expression.KeyInfo, opt *logicalOptimizeOp) { - opt.appendStepToCurrent(agg.ID(), agg.TP(), - fmt.Sprintf("%s is a unique key", uniqueKey.String()), - "aggregation is simplified to a projection") +func appendAggregationEliminateTraceStep(agg *LogicalAggregation, proj *LogicalProjection, uniqueKey expression.KeyInfo, opt *logicalOptimizeOp) { + reason := func() string { + return fmt.Sprintf("%s is a unique key", uniqueKey.String()) + } + action := func() string { + return fmt.Sprintf("%v_%v is simplified to a %v_%v", agg.TP(), agg.ID(), proj.TP(), proj.ID()) + } + + opt.appendStepToCurrent(agg.ID(), agg.TP(), reason, action) } func appendDistinctEliminateTraceStep(agg *LogicalAggregation, uniqueKey expression.KeyInfo, af *aggregation.AggFuncDesc, opt *logicalOptimizeOp) { - opt.appendStepToCurrent(agg.ID(), agg.TP(), - fmt.Sprintf("%s is a unique key", uniqueKey.String()), - fmt.Sprintf("%s(distinct ...) is simplified to %s(...)", af.Name, af.Name)) + reason := func() string { + return fmt.Sprintf("%s is a unique key", uniqueKey.String()) + } + action := func() string { + return fmt.Sprintf("%s(distinct ...) is simplified to %s(...)", af.Name, af.Name) + } + opt.appendStepToCurrent(agg.ID(), agg.TP(), reason, action) } // ConvertAggToProj convert aggregation to projection. diff --git a/planner/core/rule_aggregation_push_down.go b/planner/core/rule_aggregation_push_down.go index 7cc0548a7f57d..5f927befb8ee3 100644 --- a/planner/core/rule_aggregation_push_down.go +++ b/planner/core/rule_aggregation_push_down.go @@ -517,7 +517,7 @@ func (*aggregationPushDownSolver) name() string { func appendAggPushDownAcrossJoinTraceStep(oldAgg, newAgg *LogicalAggregation, aggFuncs []*aggregation.AggFuncDesc, join *LogicalJoin, childIdx int, opt *logicalOptimizeOp) { reason := func() string { - buffer := bytes.NewBufferString(fmt.Sprintf("agg[%v]'s functions[", oldAgg.ID())) + buffer := bytes.NewBufferString(fmt.Sprintf("%v_%v's functions[", oldAgg.TP(), oldAgg.ID())) for i, aggFunc := range aggFuncs { if i > 0 { buffer.WriteString(",") @@ -526,23 +526,23 @@ func appendAggPushDownAcrossJoinTraceStep(oldAgg, newAgg *LogicalAggregation, ag } buffer.WriteString("] are decomposable with join") return buffer.String() - }() + } action := func() string { - buffer := bytes.NewBufferString(fmt.Sprintf("agg[%v] pushed down across join[%v], ", oldAgg.ID(), join.ID())) - buffer.WriteString(fmt.Sprintf("and join %v path becomes agg[%v]", func() string { + buffer := bytes.NewBufferString(fmt.Sprintf("%v_%v pushed down across %v_%v, ", oldAgg.TP(), oldAgg.ID(), join.TP(), join.ID())) + buffer.WriteString(fmt.Sprintf("and %v_%v %v path becomes %v_%v", join.TP(), join.ID(), func() string { if childIdx == 0 { return "left" } return "right" - }(), newAgg.ID())) + }(), newAgg.TP(), newAgg.ID())) return buffer.String() - }() + } opt.appendStepToCurrent(join.ID(), join.TP(), reason, action) } func appendAggPushDownAcrossProjTraceStep(agg *LogicalAggregation, proj *LogicalProjection, opt *logicalOptimizeOp) { action := func() string { - buffer := bytes.NewBufferString(fmt.Sprintf("proj[%v] is eliminated, and agg[%v]'s functions changed into[", proj.ID(), agg.ID())) + buffer := bytes.NewBufferString(fmt.Sprintf("%v_%v is eliminated, and %v_%v's functions changed into[", proj.TP(), proj.ID(), agg.TP(), agg.ID())) for i, aggFunc := range agg.AggFuncs { if i > 0 { buffer.WriteString(",") @@ -551,33 +551,35 @@ func appendAggPushDownAcrossProjTraceStep(agg *LogicalAggregation, proj *Logical } buffer.WriteString("]") return buffer.String() - }() - reason := fmt.Sprintf("Proj[%v] is directly below an agg[%v] and has no side effects", proj.ID(), agg.ID()) + } + reason := func() string { + return fmt.Sprintf("%v_%v is directly below an %v_%v and has no side effects", proj.TP(), proj.ID(), agg.TP(), agg.ID()) + } opt.appendStepToCurrent(agg.ID(), agg.TP(), reason, action) } func appendAggPushDownAcrossUnionTraceStep(union *LogicalUnionAll, agg *LogicalAggregation, opt *logicalOptimizeOp) { reason := func() string { - buffer := bytes.NewBufferString(fmt.Sprintf("agg[%v] functions[", agg.ID())) + buffer := bytes.NewBufferString(fmt.Sprintf("%v_%v functions[", agg.TP(), agg.ID())) for i, aggFunc := range agg.AggFuncs { if i > 0 { buffer.WriteString(",") } buffer.WriteString(aggFunc.String()) } - buffer.WriteString("] are decomposable with union") + buffer.WriteString(fmt.Sprintf("] are decomposable with %v_%v", union.TP(), union.ID())) return buffer.String() - }() + } action := func() string { - buffer := bytes.NewBufferString(fmt.Sprintf("agg[%v] pushed down, and union[%v]'s children changed into[", agg.ID(), union.ID())) + buffer := bytes.NewBufferString(fmt.Sprintf("%v_%v pushed down, and %v_%v's children changed into[", agg.TP(), agg.ID(), union.TP(), union.ID())) for i, child := range union.Children() { if i > 0 { buffer.WriteString(",") } - buffer.WriteString(fmt.Sprintf("[id:%v,tp:%s]", child.ID(), child.TP())) + buffer.WriteString(fmt.Sprintf("%v_%v", child.TP(), child.ID())) } buffer.WriteString("]") return buffer.String() - }() + } opt.appendStepToCurrent(union.ID(), union.TP(), reason, action) } diff --git a/planner/core/rule_eliminate_projection.go b/planner/core/rule_eliminate_projection.go index 0c377bd974703..0daaf4616185a 100644 --- a/planner/core/rule_eliminate_projection.go +++ b/planner/core/rule_eliminate_projection.go @@ -300,7 +300,7 @@ func (*projectionEliminator) name() string { func appendDupProjEliminateTraceStep(parent, child *LogicalProjection, opt *logicalOptimizeOp) { action := func() string { buffer := bytes.NewBufferString( - fmt.Sprintf("Proj[%v] is eliminated, Proj[%v]'s expressions changed into[", child.ID(), parent.ID())) + fmt.Sprintf("%v_%v is eliminated, %v_%v's expressions changed into[", child.TP(), child.ID(), parent.TP(), parent.ID())) for i, expr := range parent.Exprs { if i > 0 { buffer.WriteString(",") @@ -309,13 +309,19 @@ func appendDupProjEliminateTraceStep(parent, child *LogicalProjection, opt *logi } buffer.WriteString("]") return buffer.String() - }() - reason := fmt.Sprintf("Proj[%v]'s child proj[%v] is redundant", parent.ID(), child.ID()) + } + reason := func() string { + return fmt.Sprintf("%v_%v's child %v_%v is redundant", parent.TP(), parent.ID(), child.TP(), child.ID()) + } opt.appendStepToCurrent(child.ID(), child.TP(), reason, action) } func appendProjEliminateTraceStep(proj *LogicalProjection, opt *logicalOptimizeOp) { - reason := fmt.Sprintf("Proj[%v]'s Exprs are all Columns", proj.ID()) - action := fmt.Sprintf("Proj[%v] is eliminated", proj.ID()) + reason := func() string { + return fmt.Sprintf("%v_%v's Exprs are all Columns", proj.TP(), proj.ID()) + } + action := func() string { + return fmt.Sprintf("%v_%v is eliminated", proj.TP(), proj.ID()) + } opt.appendStepToCurrent(proj.ID(), proj.TP(), reason, action) } diff --git a/planner/core/rule_join_elimination.go b/planner/core/rule_join_elimination.go index 6d4a750ea4356..c7ed935b8dc99 100644 --- a/planner/core/rule_join_elimination.go +++ b/planner/core/rule_join_elimination.go @@ -258,8 +258,10 @@ func appendOuterJoinEliminateTraceStep(join *LogicalJoin, outerPlan LogicalPlan, } buffer.WriteString("] are unique") return buffer.String() - }() - action := fmt.Sprintf("Outer join[%v] is eliminated and become %v[%v]", join.ID(), outerPlan.TP(), outerPlan.ID()) + } + action := func() string { + return fmt.Sprintf("Outer %v_%v is eliminated and become %v_%v", join.TP(), join.ID(), outerPlan.TP(), outerPlan.ID()) + } opt.appendStepToCurrent(join.ID(), join.TP(), reason, action) } @@ -274,7 +276,9 @@ func appendOuterJoinEliminateAggregationTraceStep(join *LogicalJoin, outerPlan L } buffer.WriteString("] in agg are from outer table, and the agg functions are duplicate agnostic") return buffer.String() - }() - action := fmt.Sprintf("Outer join[%v] is eliminated and become %v[%v]", join.ID(), outerPlan.TP(), outerPlan.ID()) + } + action := func() string { + return fmt.Sprintf("Outer %v_%v is eliminated and become %v_%v", join.TP(), join.ID(), outerPlan.TP(), outerPlan.ID()) + } opt.appendStepToCurrent(join.ID(), join.TP(), reason, action) } diff --git a/planner/core/rule_join_reorder.go b/planner/core/rule_join_reorder.go index dd29f7d3f1f30..e1ff17d70fd98 100644 --- a/planner/core/rule_join_reorder.go +++ b/planner/core/rule_join_reorder.go @@ -209,7 +209,9 @@ func appendJoinReorderTraceStep(tracer *joinReorderTrace, plan LogicalPlan, opt if len(tracer.initial) < 1 || len(tracer.final) < 1 { return } - action := fmt.Sprintf("join order becomes %v from original %v", tracer.final, tracer.initial) + action := func() string { + return fmt.Sprintf("join order becomes %v from original %v", tracer.final, tracer.initial) + } reason := func() string { buffer := bytes.NewBufferString("join cost during reorder: [") var joins []string @@ -225,7 +227,7 @@ func appendJoinReorderTraceStep(tracer *joinReorderTrace, plan LogicalPlan, opt } buffer.WriteString("]") return buffer.String() - }() + } opt.appendStepToCurrent(plan.ID(), plan.TP(), reason, action) } @@ -323,16 +325,16 @@ func (t *joinReorderTrace) traceJoinReorder(p LogicalPlan) { return } if len(t.initial) > 0 { - t.final = allJoinOrderToString(extractJoinAndDataSource(p.buildLogicalPlanTrace(p))) + t.final = allJoinOrderToString(extractJoinAndDataSource(p.buildLogicalPlanTrace())) return } - t.initial = allJoinOrderToString(extractJoinAndDataSource(p.buildLogicalPlanTrace(p))) + t.initial = allJoinOrderToString(extractJoinAndDataSource(p.buildLogicalPlanTrace())) } func (t *joinReorderTrace) appendLogicalJoinCost(join LogicalPlan, cost float64) { if t == nil || t.opt == nil || t.opt.tracer == nil { return } - joinMapKey := allJoinOrderToString(extractJoinAndDataSource(join.buildLogicalPlanTrace(join))) + joinMapKey := allJoinOrderToString(extractJoinAndDataSource(join.buildLogicalPlanTrace())) t.cost[joinMapKey] = cost } diff --git a/planner/core/rule_max_min_eliminate.go b/planner/core/rule_max_min_eliminate.go index 858f9005c2273..20d6fdba2857f 100644 --- a/planner/core/rule_max_min_eliminate.go +++ b/planner/core/rule_max_min_eliminate.go @@ -254,56 +254,56 @@ func appendEliminateSingleMaxMinTrace(agg *LogicalAggregation, sel *LogicalSelec action := func() string { buffer := bytes.NewBufferString("") if sel != nil { - buffer.WriteString(fmt.Sprintf("add selection[%v],", sel.ID())) + buffer.WriteString(fmt.Sprintf("add %v_%v,", sel.TP(), sel.ID())) } if sort != nil { - buffer.WriteString(fmt.Sprintf("add sort[%v],", sort.ID())) + buffer.WriteString(fmt.Sprintf("add %v_%v,", sort.TP(), sort.ID())) } - buffer.WriteString(fmt.Sprintf("add limit[%v] during eliminating agg[%v] %s function", limit.ID(), agg.ID(), agg.AggFuncs[0].Name)) + buffer.WriteString(fmt.Sprintf("add %v_%v during eliminating %v_%v %s function", limit.TP(), limit.ID(), agg.TP(), agg.ID(), agg.AggFuncs[0].Name)) return buffer.String() - }() + } reason := func() string { - buffer := bytes.NewBufferString(fmt.Sprintf("agg[%v] has only one function[%s] without group by", agg.ID(), agg.AggFuncs[0].Name)) + buffer := bytes.NewBufferString(fmt.Sprintf("%v_%v has only one function[%s] without group by", agg.TP(), agg.ID(), agg.AggFuncs[0].Name)) if sel != nil { - buffer.WriteString(fmt.Sprintf(", the columns in agg[%v] shouldn't be NULL and needs NULL to be filtered out", agg.ID())) + buffer.WriteString(fmt.Sprintf(", the columns in %v_%v shouldn't be NULL and needs NULL to be filtered out", agg.TP(), agg.ID())) } if sort != nil { - buffer.WriteString(fmt.Sprintf(", the columns in agg[%v] should be sorted", agg.ID())) + buffer.WriteString(fmt.Sprintf(", the columns in %v_%v should be sorted", agg.TP(), agg.ID())) } return buffer.String() - }() + } opt.appendStepToCurrent(agg.ID(), agg.TP(), reason, action) } func appendEliminateMultiMinMaxTraceStep(originAgg *LogicalAggregation, aggs []*LogicalAggregation, joins []*LogicalJoin, opt *logicalOptimizeOp) { action := func() string { - buffer := bytes.NewBufferString(fmt.Sprintf("agg[%v] splited into aggs[", originAgg.ID())) + buffer := bytes.NewBufferString(fmt.Sprintf("%v_%v splited into [", originAgg.TP(), originAgg.ID())) for i, agg := range aggs { if i > 0 { buffer.WriteString(",") } - buffer.WriteString(fmt.Sprintf("%v", agg.ID())) + buffer.WriteString(fmt.Sprintf("%v_%v", agg.TP(), agg.ID())) } - buffer.WriteString("], and add joins[") + buffer.WriteString("], and add [") for i, join := range joins { if i > 0 { buffer.WriteString(",") } - buffer.WriteString(fmt.Sprintf("%v", join.ID())) + buffer.WriteString(fmt.Sprintf("%v_%v", join.TP(), join.ID())) } - buffer.WriteString(fmt.Sprintf("] to connect them during eliminating agg[%v] multi min/max functions", originAgg.ID())) + buffer.WriteString(fmt.Sprintf("] to connect them during eliminating %v_%v multi min/max functions", originAgg.TP(), originAgg.ID())) return buffer.String() - }() + } reason := func() string { - buffer := bytes.NewBufferString("each column is sorted and can benefit from index/primary key in agg[") + buffer := bytes.NewBufferString("each column is sorted and can benefit from index/primary key in [") for i, agg := range aggs { if i > 0 { buffer.WriteString(",") } - buffer.WriteString(fmt.Sprintf("%v", agg.ID())) + buffer.WriteString(fmt.Sprintf("%v_%v", agg.TP(), agg.ID())) } buffer.WriteString("] and none of them has group by clause") return buffer.String() - }() + } opt.appendStepToCurrent(originAgg.ID(), originAgg.TP(), reason, action) } diff --git a/planner/core/rule_partition_processor.go b/planner/core/rule_partition_processor.go index 04f572200232b..53a655c5ed932 100644 --- a/planner/core/rule_partition_processor.go +++ b/planner/core/rule_partition_processor.go @@ -1580,8 +1580,7 @@ func appendMakeUnionAllChildrenTranceStep(ds *DataSource, usedMap map[int64]mode appendNoPartitionChildTraceStep(ds, plan, opt) return } - action := "" - reason := "" + var action, reason func() string var used []model.PartitionDefinition for _, def := range usedMap { used = append(used, def) @@ -1590,22 +1589,26 @@ func appendMakeUnionAllChildrenTranceStep(ds *DataSource, usedMap map[int64]mode return used[i].ID < used[j].ID }) if len(children) == 1 { - action = fmt.Sprintf("Datasource[%v] becomes %s[%v]", ds.ID(), plan.TP(), plan.ID()) - reason = fmt.Sprintf("Datasource[%v] has one needed partition[%s] after pruning", ds.ID(), used[0].Name) + action = func() string { + return fmt.Sprintf("%v_%v becomes %s_%v", ds.TP(), ds.ID(), plan.TP(), plan.ID()) + } + reason = func() string { + return fmt.Sprintf("%v_%v has one needed partition[%s] after pruning", ds.TP(), ds.ID(), used[0].Name) + } } else { action = func() string { - buffer := bytes.NewBufferString(fmt.Sprintf("Datasource[%v] becomes %s[%v] with children[", ds.ID(), plan.TP(), plan.ID())) + buffer := bytes.NewBufferString(fmt.Sprintf("%v_%v becomes %s_%v with children[", ds.TP(), ds.ID(), plan.TP(), plan.ID())) for i, child := range children { if i > 0 { buffer.WriteString(",") } - buffer.WriteString(fmt.Sprintf("%s[%v]", child.TP(), child.ID())) + buffer.WriteString(fmt.Sprintf("%s_%v", child.TP(), child.ID())) } buffer.WriteString("]") return buffer.String() - }() + } reason = func() string { - buffer := bytes.NewBufferString(fmt.Sprintf("Datasource[%v] has multiple needed partitions[", ds.ID())) + buffer := bytes.NewBufferString(fmt.Sprintf("%v_%v has multiple needed partitions[", ds.TP(), ds.ID())) for i, u := range used { if i > 0 { buffer.WriteString(",") @@ -1614,13 +1617,17 @@ func appendMakeUnionAllChildrenTranceStep(ds *DataSource, usedMap map[int64]mode } buffer.WriteString("] after pruning") return buffer.String() - }() + } } opt.appendStepToCurrent(ds.ID(), ds.TP(), reason, action) } func appendNoPartitionChildTraceStep(ds *DataSource, dual LogicalPlan, opt *logicalOptimizeOp) { - action := fmt.Sprintf("Datasource[%v] becomes %v[%v]", ds.ID(), dual.TP(), dual.ID()) - reason := fmt.Sprintf("Datasource[%v] doesn't have needed partition table after pruning", ds.ID()) + action := func() string { + return fmt.Sprintf("%v_%v becomes %v_%v", ds.TP(), ds.ID(), dual.TP(), dual.ID()) + } + reason := func() string { + return fmt.Sprintf("%v_%v doesn't have needed partition table after pruning", ds.TP(), ds.ID()) + } opt.appendStepToCurrent(dual.ID(), dual.TP(), reason, action) } diff --git a/planner/core/rule_topn_push_down.go b/planner/core/rule_topn_push_down.go index 73ed5749bc57c..ecbb30a9cb17b 100644 --- a/planner/core/rule_topn_push_down.go +++ b/planner/core/rule_topn_push_down.go @@ -87,8 +87,7 @@ func (ls *LogicalSort) pushDownTopN(topN *LogicalTopN, opt *logicalOptimizeOp) L func (p *LogicalLimit) convertToTopN(opt *logicalOptimizeOp) *LogicalTopN { topn := LogicalTopN{Offset: p.Offset, Count: p.Count, limitHints: p.limitHints}.Init(p.ctx, p.blockOffset) - opt.appendStepToCurrent(topn.ID(), topn.TP(), "", fmt.Sprintf("%v_%v is converted into %v_%v", - p.TP(), p.ID(), topn.TP(), topn.ID())) + appendConvertTopNTraceStep(p, topn, opt) return topn } @@ -109,9 +108,7 @@ func (p *LogicalUnionAll) pushDownTopN(topN *LogicalTopN, opt *logicalOptimizeOp newTopN.ByItems = append(newTopN.ByItems, &util.ByItems{Expr: by.Expr, Desc: by.Desc}) } // newTopN to push down Union's child - opt.appendStepToCurrent(newTopN.ID(), newTopN.TP(), "", - fmt.Sprintf("%v_%v is added and pushed down across %v_%v", - newTopN.TP(), newTopN.ID(), p.TP(), p.ID())) + appendNewTopNTraceStep(topN, p, opt) } p.children[i] = child.pushDownTopN(newTopN, opt) } @@ -202,8 +199,12 @@ func (*pushDownTopNOptimizer) name() string { } func appendTopNPushDownTraceStep(parent LogicalPlan, child LogicalPlan, opt *logicalOptimizeOp) { - action := fmt.Sprintf("%v_%v is added as %v_%v's parent", parent.TP(), parent.ID(), child.TP(), child.ID()) - reason := fmt.Sprintf("%v is pushed down", parent.TP()) + action := func() string { + return fmt.Sprintf("%v_%v is added as %v_%v's parent", parent.TP(), parent.ID(), child.TP(), child.ID()) + } + reason := func() string { + return fmt.Sprintf("%v is pushed down", parent.TP()) + } opt.appendStepToCurrent(parent.ID(), parent.TP(), reason, action) } @@ -218,7 +219,7 @@ func appendTopNPushDownJoinTraceStep(p *LogicalJoin, topN *LogicalTopN, idx int, } buffer.WriteString("table") return buffer.String() - }() + } reason := func() string { buffer := bytes.NewBufferString(fmt.Sprintf("%v_%v's joinType is %v, and all ByItems[", p.TP(), p.ID(), p.JoinType.String())) for i, item := range topN.ByItems { @@ -235,7 +236,7 @@ func appendTopNPushDownJoinTraceStep(p *LogicalJoin, topN *LogicalTopN, idx int, } buffer.WriteString("table") return buffer.String() - }() + } opt.appendStepToCurrent(p.ID(), p.TP(), reason, action) } @@ -250,7 +251,29 @@ func appendSortPassByItemsTraceStep(sort *LogicalSort, topN *LogicalTopN, opt *l } buffer.WriteString(fmt.Sprintf("] to %v_%v", topN.TP(), topN.ID())) return buffer.String() - }() - reason := fmt.Sprintf("%v_%v is Limit originally", topN.TP(), topN.ID()) + } + reason := func() string { + return fmt.Sprintf("%v_%v is Limit originally", topN.TP(), topN.ID()) + } opt.appendStepToCurrent(sort.ID(), sort.TP(), reason, action) } + +func appendNewTopNTraceStep(topN *LogicalTopN, union *LogicalUnionAll, opt *logicalOptimizeOp) { + reason := func() string { + return "" + } + action := func() string { + return fmt.Sprintf("%v_%v is added and pushed down across %v_%v", topN.TP(), topN.ID(), union.TP(), union.ID()) + } + opt.appendStepToCurrent(topN.ID(), topN.TP(), reason, action) +} + +func appendConvertTopNTraceStep(p LogicalPlan, topN *LogicalTopN, opt *logicalOptimizeOp) { + reason := func() string { + return "" + } + action := func() string { + return fmt.Sprintf("%v_%v is converted into %v_%v", p.TP(), p.ID(), topN.TP(), topN.ID()) + } + opt.appendStepToCurrent(topN.ID(), topN.TP(), reason, action) +} From 9063d3b837c25dc5c74c339e14cc3306927501b8 Mon Sep 17 00:00:00 2001 From: tangenta Date: Wed, 22 Dec 2021 16:05:47 +0800 Subject: [PATCH 18/44] table: set the datum collation correctly in CastValue() (#30931) close pingcap/tidb#30930 --- executor/write_test.go | 14 ++++++++++++++ table/column.go | 6 +++--- table/column_test.go | 12 ++++++++++++ 3 files changed, 29 insertions(+), 3 deletions(-) diff --git a/executor/write_test.go b/executor/write_test.go index 11e402f446631..ec326d0b6d436 100644 --- a/executor/write_test.go +++ b/executor/write_test.go @@ -1096,6 +1096,20 @@ func TestReplace(t *testing.T) { tk.MustExec("drop table t1, t2") } +func TestReplaceWithCICollation(t *testing.T) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + + tk.MustExec("create table t (a varchar(20) charset utf8mb4 collate utf8mb4_general_ci primary key);") + tk.MustExec("replace into t(a) values (_binary'A '),(_binary'A');") + tk.MustQuery("select a from t use index(primary);").Check(testkit.Rows("A")) + tk.MustQuery("select a from t ignore index(primary);").Check(testkit.Rows("A")) +} + func TestGeneratedColumnForInsert(t *testing.T) { store, clean := testkit.CreateMockStore(t) defer clean() diff --git a/table/column.go b/table/column.go index d7e9a9ec5dadb..cdd296d8c1d0f 100644 --- a/table/column.go +++ b/table/column.go @@ -363,18 +363,18 @@ func validateStringDatum(ctx sessionctx.Context, origin, casted *types.Datum, co src := casted.GetBytes() encBytes, err := enc.Transform(nil, src, charset.OpDecode) if err != nil { - casted.SetBytesAsString(encBytes, charset.CollationUTF8MB4, 0) + casted.SetBytesAsString(encBytes, col.Collate, 0) nSrc := charset.CountValidBytesDecode(enc, src) return handleWrongCharsetValue(ctx, col, src, nSrc) } - casted.SetBytesAsString(encBytes, charset.CollationUTF8MB4, 0) + casted.SetBytesAsString(encBytes, col.Collate, 0) return nil } // Check if the string is valid in the given column charset. str := casted.GetBytes() if !charset.IsValid(enc, str) { replace, _ := enc.Transform(nil, str, charset.OpReplace) - casted.SetBytesAsString(replace, charset.CollationUTF8MB4, 0) + casted.SetBytesAsString(replace, col.Collate, 0) nSrc := charset.CountValidBytes(enc, str) return handleWrongCharsetValue(ctx, col, str, nSrc) } diff --git a/table/column_test.go b/table/column_test.go index 02cbb12237afc..27e35f94757ba 100644 --- a/table/column_test.go +++ b/table/column_test.go @@ -303,6 +303,18 @@ func TestCastValue(t *testing.T) { colInfoS.Charset = charset.CharsetASCII _, err = CastValue(ctx, types.NewDatum([]byte{0x32, 0xf0}), &colInfoS, false, true) require.NoError(t, err) + + colInfoS.Charset = charset.CharsetUTF8MB4 + colInfoS.Collate = "utf8mb4_general_ci" + val, err = CastValue(ctx, types.NewBinaryLiteralDatum([]byte{0xE5, 0xA5, 0xBD}), &colInfoS, false, false) + require.NoError(t, err) + require.Equal(t, "utf8mb4_general_ci", val.Collation()) + val, err = CastValue(ctx, types.NewBinaryLiteralDatum([]byte{0xE5, 0xA5, 0xBD, 0x81}), &colInfoS, false, false) + require.Error(t, err, "[table:1366]Incorrect string value '\\x81' for column ''") + require.Equal(t, "utf8mb4_general_ci", val.Collation()) + val, err = CastValue(ctx, types.NewDatum([]byte{0xE5, 0xA5, 0xBD, 0x81}), &colInfoS, false, false) + require.Error(t, err, "[table:1366]Incorrect string value '\\x81' for column ''") + require.Equal(t, "utf8mb4_general_ci", val.Collation()) } func TestGetDefaultValue(t *testing.T) { From 1f6d2ddfb9b38fd4a033184d95e4d953fdfca7ab Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=8E=8B=E8=B6=85?= Date: Wed, 22 Dec 2021 17:39:47 +0800 Subject: [PATCH 19/44] *: Use TxnManager.GetTxnInfoSchema() to get the txn infoschema (#30934) close pingcap/tidb#30933 --- executor/adapter.go | 2 +- executor/compiler.go | 5 +++-- session/session.go | 8 +++++--- 3 files changed, 9 insertions(+), 6 deletions(-) diff --git a/executor/adapter.go b/executor/adapter.go index be83c42d99940..37998a3bfc7c4 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -315,7 +315,7 @@ func (a *ExecStmt) RebuildPlan(ctx context.Context) (int64, error) { sessiontxn.AssertTxnManagerInfoSchema(a.Ctx, ret.InfoSchema) }) - a.InfoSchema = ret.InfoSchema + a.InfoSchema = sessiontxn.GetTxnManager(a.Ctx).GetTxnInfoSchema() a.SnapshotTS = ret.LastSnapshotTS a.IsStaleness = ret.IsStaleness a.ReplicaReadScope = ret.ReadReplicaScope diff --git a/executor/compiler.go b/executor/compiler.go index baf49979572c2..5debec73f1590 100644 --- a/executor/compiler.go +++ b/executor/compiler.go @@ -73,7 +73,8 @@ func (c *Compiler) Compile(ctx context.Context, stmtNode ast.StmtNode) (*ExecStm sessiontxn.AssertTxnManagerInfoSchema(c.Ctx, ret.InfoSchema) }) - finalPlan, names, err := planner.Optimize(ctx, c.Ctx, stmtNode, ret.InfoSchema) + is := sessiontxn.GetTxnManager(c.Ctx).GetTxnInfoSchema() + finalPlan, names, err := planner.Optimize(ctx, c.Ctx, stmtNode, is) if err != nil { return nil, err } @@ -96,7 +97,7 @@ func (c *Compiler) Compile(ctx context.Context, stmtNode ast.StmtNode) (*ExecStm SnapshotTS: ret.LastSnapshotTS, IsStaleness: ret.IsStaleness, ReplicaReadScope: ret.ReadReplicaScope, - InfoSchema: ret.InfoSchema, + InfoSchema: is, Plan: finalPlan, LowerPriority: lowerPriority, Text: stmtNode.Text(), diff --git a/session/session.go b/session/session.go index cd1ab05da5498..e781a902afdf7 100644 --- a/session/session.go +++ b/session/session.go @@ -2130,7 +2130,9 @@ func (s *session) ExecutePreparedStmt(ctx context.Context, stmtID uint32, args [ txnCtxProvider := &sessiontxn.SimpleTxnContextProvider{ InfoSchema: is, } - if err = sessiontxn.GetTxnManager(s).SetContextProvider(txnCtxProvider); err != nil { + + txnManager := sessiontxn.GetTxnManager(s) + if err = txnManager.SetContextProvider(txnCtxProvider); err != nil { return nil, err } @@ -2143,9 +2145,9 @@ func (s *session) ExecutePreparedStmt(ctx context.Context, stmtID uint32, args [ defer s.txn.onStmtEnd() if ok { - return s.cachedPlanExec(ctx, is, snapshotTS, stmtID, preparedStmt, args) + return s.cachedPlanExec(ctx, txnManager.GetTxnInfoSchema(), snapshotTS, stmtID, preparedStmt, args) } - return s.preparedStmtExec(ctx, is, snapshotTS, stmtID, preparedStmt, args) + return s.preparedStmtExec(ctx, txnManager.GetTxnInfoSchema(), snapshotTS, stmtID, preparedStmt, args) } func (s *session) DropPreparedStmt(stmtID uint32) error { From d543b6046466531cda51b667d70c4d4d0298f407 Mon Sep 17 00:00:00 2001 From: tangenta Date: Wed, 22 Dec 2021 18:19:48 +0800 Subject: [PATCH 20/44] parser: add IsValid() to Encoding to speed up string validation for UTF-8 (#30937) close pingcap/tidb#30936 --- expression/builtin_string.go | 2 +- expression/builtin_string_vec.go | 2 +- expression/collation.go | 3 ++- parser/charset/encoding.go | 17 ++--------------- parser/charset/encoding_ascii.go | 13 ++++++++++++- parser/charset/encoding_base.go | 9 +++++++++ parser/charset/encoding_bin.go | 5 +++++ parser/charset/encoding_latin1.go | 5 +++++ parser/charset/encoding_test.go | 3 +-- parser/charset/encoding_utf8.go | 17 +++++++++++++++-- table/column.go | 2 +- 11 files changed, 54 insertions(+), 24 deletions(-) diff --git a/expression/builtin_string.go b/expression/builtin_string.go index c494d9fcb5c10..acac019139708 100644 --- a/expression/builtin_string.go +++ b/expression/builtin_string.go @@ -1150,7 +1150,7 @@ func (b *builtinConvertSig) evalString(row chunk.Row) (string, bool, error) { return string(ret), false, err } enc := charset.FindEncoding(resultTp.Charset) - if !charset.IsValidString(enc, expr) { + if !enc.IsValid(hack.Slice(expr)) { replace, _ := enc.Transform(nil, hack.Slice(expr), charset.OpReplace) return string(replace), false, nil } diff --git a/expression/builtin_string_vec.go b/expression/builtin_string_vec.go index 3da555f9319ed..202a3d74ed3f1 100644 --- a/expression/builtin_string_vec.go +++ b/expression/builtin_string_vec.go @@ -689,7 +689,7 @@ func (b *builtinConvertSig) vecEvalString(input *chunk.Chunk, result *chunk.Colu continue } exprI := expr.GetBytes(i) - if !charset.IsValid(enc, exprI) { + if !enc.IsValid(exprI) { encBuf, _ = enc.Transform(encBuf, exprI, charset.OpReplace) result.AppendBytes(encBuf) } else { diff --git a/expression/collation.go b/expression/collation.go index 8dc5df02e55e0..813560775e2b4 100644 --- a/expression/collation.go +++ b/expression/collation.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/collate" + "github.com/pingcap/tidb/util/hack" "github.com/pingcap/tidb/util/logutil" ) @@ -315,7 +316,7 @@ func safeConvert(ctx sessionctx.Context, ec *ExprCollation, args ...Expression) if isNull { continue } - if !charset.IsValidString(enc, str) { + if !enc.IsValid(hack.Slice(str)) { return false } } else { diff --git a/parser/charset/encoding.go b/parser/charset/encoding.go index 25257c44e440b..bf3d6b8ff269c 100644 --- a/parser/charset/encoding.go +++ b/parser/charset/encoding.go @@ -57,6 +57,8 @@ type Encoding interface { Tp() EncodingTp // Peek returns the next char. Peek(src []byte) []byte + // IsValid checks whether the utf-8 bytes can be convert to valid string in current encoding. + IsValid(src []byte) bool // Foreach iterates the characters in in current encoding. Foreach(src []byte, op Op, fn func(from, to []byte, ok bool) bool) // Transform map the bytes in src to dest according to Op. @@ -101,21 +103,6 @@ const ( OpDecodeReplace = opToUTF8 | opTruncateReplace | opCollectTo ) -// IsValid checks whether the bytes is valid in current encoding. -func IsValid(e Encoding, src []byte) bool { - isValid := true - e.Foreach(src, opFromUTF8, func(from, to []byte, ok bool) bool { - isValid = ok - return ok - }) - return isValid -} - -// IsValidString is a string version of IsValid. -func IsValidString(e Encoding, str string) bool { - return IsValid(e, Slice(str)) -} - // CountValidBytes counts the first valid bytes in src that // can be encode to the current encoding. func CountValidBytes(e Encoding, src []byte) int { diff --git a/parser/charset/encoding_ascii.go b/parser/charset/encoding_ascii.go index df5fed9c3bce2..34432d5b42e3c 100644 --- a/parser/charset/encoding_ascii.go +++ b/parser/charset/encoding_ascii.go @@ -49,8 +49,19 @@ func (e *encodingASCII) Peek(src []byte) []byte { return src[:1] } +// IsValid implements Encoding interface. +func (e *encodingASCII) IsValid(src []byte) bool { + srcLen := len(src) + for i := 0; i < srcLen; i++ { + if src[i] > go_unicode.MaxASCII { + return false + } + } + return true +} + func (e *encodingASCII) Transform(dest, src []byte, op Op) ([]byte, error) { - if IsValid(e, src) { + if e.IsValid(src) { return src, nil } return e.encodingBase.Transform(dest, src, op) diff --git a/parser/charset/encoding_base.go b/parser/charset/encoding_base.go index 275db24c5a3d6..213596c6aec55 100644 --- a/parser/charset/encoding_base.go +++ b/parser/charset/encoding_base.go @@ -42,6 +42,15 @@ func (b encodingBase) ToLower(src string) string { return strings.ToLower(src) } +func (b encodingBase) IsValid(src []byte) bool { + isValid := true + b.self.Foreach(src, opFromUTF8, func(from, to []byte, ok bool) bool { + isValid = ok + return ok + }) + return isValid +} + func (b encodingBase) Transform(dest, src []byte, op Op) (result []byte, err error) { if dest == nil { dest = make([]byte, len(src)) diff --git a/parser/charset/encoding_bin.go b/parser/charset/encoding_bin.go index 30fd87644c571..30b35ceb1d856 100644 --- a/parser/charset/encoding_bin.go +++ b/parser/charset/encoding_bin.go @@ -47,6 +47,11 @@ func (e *encodingBin) Peek(src []byte) []byte { return src[:1] } +// IsValid implements Encoding interface. +func (e *encodingBin) IsValid(src []byte) bool { + return true +} + // Foreach implements Encoding interface. func (e *encodingBin) Foreach(src []byte, op Op, fn func(from, to []byte, ok bool) bool) { for i := 0; i < len(src); i++ { diff --git a/parser/charset/encoding_latin1.go b/parser/charset/encoding_latin1.go index 1d2992b87642d..d627ed63ec419 100644 --- a/parser/charset/encoding_latin1.go +++ b/parser/charset/encoding_latin1.go @@ -41,6 +41,11 @@ func (e *encodingLatin1) Peek(src []byte) []byte { return src[:1] } +// IsValid implements Encoding interface. +func (e *encodingLatin1) IsValid(src []byte) bool { + return true +} + // Tp implements Encoding interface. func (e *encodingLatin1) Tp() EncodingTp { return EncodingTpLatin1 diff --git a/parser/charset/encoding_test.go b/parser/charset/encoding_test.go index a78aa640d8be5..27d41dbf5ebd2 100644 --- a/parser/charset/encoding_test.go +++ b/parser/charset/encoding_test.go @@ -133,8 +133,7 @@ func TestEncodingValidate(t *testing.T) { enc = charset.EncodingUTF8MB3StrictImpl } strBytes := []byte(tc.str) - ok := charset.IsValid(enc, strBytes) - require.Equal(t, tc.ok, ok, msg) + require.Equal(t, tc.ok, enc.IsValid(strBytes), msg) replace, _ := enc.Transform(nil, strBytes, charset.OpReplace) require.Equal(t, tc.expected, string(replace), msg) } diff --git a/parser/charset/encoding_utf8.go b/parser/charset/encoding_utf8.go index 871a5e5ec33c1..499ce5ea50de7 100644 --- a/parser/charset/encoding_utf8.go +++ b/parser/charset/encoding_utf8.go @@ -67,9 +67,17 @@ func (e *encodingUTF8) Peek(src []byte) []byte { return src[:nextLen] } +// IsValid implements Encoding interface. +func (e *encodingUTF8) IsValid(src []byte) bool { + if utf8.Valid(src) { + return true + } + return e.encodingBase.IsValid(src) +} + // Transform implements Encoding interface. func (e *encodingUTF8) Transform(dest, src []byte, op Op) ([]byte, error) { - if IsValid(e, src) { + if e.IsValid(src) { return src, nil } return e.encodingBase.Transform(dest, src, op) @@ -93,6 +101,11 @@ type encodingUTF8MB3Strict struct { encodingUTF8 } +// IsValid implements Encoding interface. +func (e *encodingUTF8MB3Strict) IsValid(src []byte) bool { + return e.encodingBase.IsValid(src) +} + // Foreach implements Encoding interface. func (e *encodingUTF8MB3Strict) Foreach(src []byte, op Op, fn func(srcCh, dstCh []byte, ok bool) bool) { for i, w := 0, 0; i < len(src); i += w { @@ -107,7 +120,7 @@ func (e *encodingUTF8MB3Strict) Foreach(src []byte, op Op, fn func(srcCh, dstCh // Transform implements Encoding interface. func (e *encodingUTF8MB3Strict) Transform(dest, src []byte, op Op) ([]byte, error) { - if IsValid(e, src) { + if e.IsValid(src) { return src, nil } return e.encodingBase.Transform(dest, src, op) diff --git a/table/column.go b/table/column.go index cdd296d8c1d0f..90404f08cbb69 100644 --- a/table/column.go +++ b/table/column.go @@ -372,7 +372,7 @@ func validateStringDatum(ctx sessionctx.Context, origin, casted *types.Datum, co } // Check if the string is valid in the given column charset. str := casted.GetBytes() - if !charset.IsValid(enc, str) { + if !enc.IsValid(str) { replace, _ := enc.Transform(nil, str, charset.OpReplace) casted.SetBytesAsString(replace, col.Collate, 0) nSrc := charset.CountValidBytes(enc, str) From 713e6994db876fad85684647c854dbcb4baf2ceb Mon Sep 17 00:00:00 2001 From: Chengpeng Yan <41809508+Reminiscent@users.noreply.github.com> Date: Wed, 22 Dec 2021 18:59:48 +0800 Subject: [PATCH 21/44] planner: rename pstmtPlanCacheXX to PlanCacheXX (#30909) --- executor/prepared.go | 2 +- planner/core/cache.go | 26 +++++++++++++++----------- planner/core/cache_test.go | 2 +- planner/core/common_plans.go | 18 +++++++++--------- server/driver_tidb.go | 2 +- session/session.go | 2 +- 6 files changed, 28 insertions(+), 24 deletions(-) diff --git a/executor/prepared.go b/executor/prepared.go index 3b703c75a9cf5..4f63bce491ab9 100644 --- a/executor/prepared.go +++ b/executor/prepared.go @@ -317,7 +317,7 @@ func (e *DeallocateExec) Next(ctx context.Context, req *chunk.Chunk) error { delete(vars.PreparedStmtNameToID, e.Name) if plannercore.PreparedPlanCacheEnabled() { bindSQL := planner.GetBindSQL4PlanCache(e.ctx, prepared.Stmt) - e.ctx.PreparedPlanCache().Delete(plannercore.NewPSTMTPlanCacheKey( + e.ctx.PreparedPlanCache().Delete(plannercore.NewPlanCacheKey( vars, id, prepared.SchemaVersion, bindSQL, )) } diff --git a/planner/core/cache.go b/planner/core/cache.go index 4113f3e911e88..2ef974340c063 100644 --- a/planner/core/cache.go +++ b/planner/core/cache.go @@ -66,7 +66,11 @@ func PreparedPlanCacheEnabled() bool { return isEnabled == preparedPlanCacheEnabled } -type pstmtPlanCacheKey struct { +// planCacheKey is used to access Plan Cache. We put some variables that do not affect the plan into planCacheKey, such as the sql text. +// Put the parameters that may affect the plan in planCacheValue, such as bindSQL. +// However, due to some compatibility reasons, we will temporarily keep some system variable-related values in planCacheKey. +// At the same time, because these variables have a small impact on plan, we will move them to PlanCacheValue later if necessary. +type planCacheKey struct { database string connID uint64 pstmtID uint32 @@ -81,7 +85,7 @@ type pstmtPlanCacheKey struct { } // Hash implements Key interface. -func (key *pstmtPlanCacheKey) Hash() []byte { +func (key *planCacheKey) Hash() []byte { if len(key.hash) == 0 { var ( dbBytes = hack.Slice(key.database) @@ -114,7 +118,7 @@ func (key *pstmtPlanCacheKey) Hash() []byte { // SetPstmtIDSchemaVersion implements PstmtCacheKeyMutator interface to change pstmtID and schemaVersion of cacheKey. // so we can reuse Key instead of new every time. func SetPstmtIDSchemaVersion(key kvcache.Key, pstmtID uint32, schemaVersion int64, isolationReadEngines map[kv.StoreType]struct{}) { - psStmtKey, isPsStmtKey := key.(*pstmtPlanCacheKey) + psStmtKey, isPsStmtKey := key.(*planCacheKey) if !isPsStmtKey { return } @@ -127,13 +131,13 @@ func SetPstmtIDSchemaVersion(key kvcache.Key, pstmtID uint32, schemaVersion int6 psStmtKey.hash = psStmtKey.hash[:0] } -// NewPSTMTPlanCacheKey creates a new pstmtPlanCacheKey object. -func NewPSTMTPlanCacheKey(sessionVars *variable.SessionVars, pstmtID uint32, schemaVersion int64, bindSQL string) kvcache.Key { +// NewPlanCacheKey creates a new planCacheKey object. +func NewPlanCacheKey(sessionVars *variable.SessionVars, pstmtID uint32, schemaVersion int64, bindSQL string) kvcache.Key { timezoneOffset := 0 if sessionVars.TimeZone != nil { _, timezoneOffset = time.Now().In(sessionVars.TimeZone).Zone() } - key := &pstmtPlanCacheKey{ + key := &planCacheKey{ database: sessionVars.CurrentDB, connID: sessionVars.ConnectionID, pstmtID: pstmtID, @@ -175,16 +179,16 @@ func (s FieldSlice) Equal(tps []*types.FieldType) bool { return true } -// PSTMTPlanCacheValue stores the cached Statement and StmtNode. -type PSTMTPlanCacheValue struct { +// PlanCacheValue stores the cached Statement and StmtNode. +type PlanCacheValue struct { Plan Plan OutPutNames []*types.FieldName TblInfo2UnionScan map[*model.TableInfo]bool UserVarTypes FieldSlice } -// NewPSTMTPlanCacheValue creates a SQLCacheValue. -func NewPSTMTPlanCacheValue(plan Plan, names []*types.FieldName, srcMap map[*model.TableInfo]bool, userVarTps []*types.FieldType) *PSTMTPlanCacheValue { +// NewPlanCacheValue creates a SQLCacheValue. +func NewPlanCacheValue(plan Plan, names []*types.FieldName, srcMap map[*model.TableInfo]bool, userVarTps []*types.FieldType) *PlanCacheValue { dstMap := make(map[*model.TableInfo]bool) for k, v := range srcMap { dstMap[k] = v @@ -193,7 +197,7 @@ func NewPSTMTPlanCacheValue(plan Plan, names []*types.FieldName, srcMap map[*mod for i, tp := range userVarTps { userVarTypes[i] = *tp } - return &PSTMTPlanCacheValue{ + return &PlanCacheValue{ Plan: plan, OutPutNames: names, TblInfo2UnionScan: dstMap, diff --git a/planner/core/cache_test.go b/planner/core/cache_test.go index 074d1e4cf2828..c75a4b3963713 100644 --- a/planner/core/cache_test.go +++ b/planner/core/cache_test.go @@ -28,6 +28,6 @@ func TestCacheKey(t *testing.T) { ctx.GetSessionVars().SQLMode = mysql.ModeNone ctx.GetSessionVars().TimeZone = time.UTC ctx.GetSessionVars().ConnectionID = 0 - key := NewPSTMTPlanCacheKey(ctx.GetSessionVars(), 1, 1, "") + key := NewPlanCacheKey(ctx.GetSessionVars(), 1, 1, "") require.Equal(t, []byte{0x74, 0x65, 0x73, 0x74, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x1, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x1, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x74, 0x69, 0x64, 0x62, 0x74, 0x69, 0x6b, 0x76, 0x74, 0x69, 0x66, 0x6c, 0x61, 0x73, 0x68, 0x7f, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff}, key.Hash()) } diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index d3e56600b2c25..54f30d7d998ea 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -404,7 +404,7 @@ func (e *Execute) getPhysicalPlan(ctx context.Context, sctx sessionctx.Context, var bindSQL string if prepared.UseCache { bindSQL = GetBindSQL4PlanCache(sctx, prepared.Stmt) - cacheKey = NewPSTMTPlanCacheKey(sctx.GetSessionVars(), e.ExecID, prepared.SchemaVersion, bindSQL) + cacheKey = NewPlanCacheKey(sctx.GetSessionVars(), e.ExecID, prepared.SchemaVersion, bindSQL) } tps := make([]*types.FieldType, len(e.UsingVars)) for i, param := range e.UsingVars { @@ -445,7 +445,7 @@ func (e *Execute) getPhysicalPlan(ctx context.Context, sctx sessionctx.Context, if err := e.checkPreparedPriv(ctx, sctx, preparedStmt, is); err != nil { return err } - cachedVals := cacheValue.([]*PSTMTPlanCacheValue) + cachedVals := cacheValue.([]*PlanCacheValue) for _, cachedVal := range cachedVals { if !cachedVal.UserVarTypes.Equal(tps) { continue @@ -510,30 +510,30 @@ REBUILD: // rebuild key to exclude kv.TiFlash when stmt is not read only if _, isolationReadContainTiFlash := sessVars.IsolationReadEngines[kv.TiFlash]; isolationReadContainTiFlash && !IsReadOnly(stmt, sessVars) { delete(sessVars.IsolationReadEngines, kv.TiFlash) - cacheKey = NewPSTMTPlanCacheKey(sessVars, e.ExecID, prepared.SchemaVersion, sessVars.StmtCtx.BindSQL) + cacheKey = NewPlanCacheKey(sessVars, e.ExecID, prepared.SchemaVersion, sessVars.StmtCtx.BindSQL) sessVars.IsolationReadEngines[kv.TiFlash] = struct{}{} } else { // We need to reconstruct the plan cache key based on the bindSQL. - cacheKey = NewPSTMTPlanCacheKey(sessVars, e.ExecID, prepared.SchemaVersion, sessVars.StmtCtx.BindSQL) + cacheKey = NewPlanCacheKey(sessVars, e.ExecID, prepared.SchemaVersion, sessVars.StmtCtx.BindSQL) } - cached := NewPSTMTPlanCacheValue(p, names, stmtCtx.TblInfo2UnionScan, tps) + cached := NewPlanCacheValue(p, names, stmtCtx.TblInfo2UnionScan, tps) preparedStmt.NormalizedPlan, preparedStmt.PlanDigest = NormalizePlan(p) stmtCtx.SetPlanDigest(preparedStmt.NormalizedPlan, preparedStmt.PlanDigest) if cacheVals, exists := sctx.PreparedPlanCache().Get(cacheKey); exists { hitVal := false - for i, cacheVal := range cacheVals.([]*PSTMTPlanCacheValue) { + for i, cacheVal := range cacheVals.([]*PlanCacheValue) { if cacheVal.UserVarTypes.Equal(tps) { hitVal = true - cacheVals.([]*PSTMTPlanCacheValue)[i] = cached + cacheVals.([]*PlanCacheValue)[i] = cached break } } if !hitVal { - cacheVals = append(cacheVals.([]*PSTMTPlanCacheValue), cached) + cacheVals = append(cacheVals.([]*PlanCacheValue), cached) } sctx.PreparedPlanCache().Put(cacheKey, cacheVals) } else { - sctx.PreparedPlanCache().Put(cacheKey, []*PSTMTPlanCacheValue{cached}) + sctx.PreparedPlanCache().Put(cacheKey, []*PlanCacheValue{cached}) } } err = e.setFoundInPlanCache(sctx, false) diff --git a/server/driver_tidb.go b/server/driver_tidb.go index 9a13eea632962..dfe88d5364114 100644 --- a/server/driver_tidb.go +++ b/server/driver_tidb.go @@ -167,7 +167,7 @@ func (ts *TiDBStatement) Close() error { } preparedAst := preparedObj.PreparedAst bindSQL := planner.GetBindSQL4PlanCache(ts.ctx, preparedAst.Stmt) - ts.ctx.PreparedPlanCache().Delete(core.NewPSTMTPlanCacheKey( + ts.ctx.PreparedPlanCache().Delete(core.NewPlanCacheKey( ts.ctx.GetSessionVars(), ts.id, preparedObj.PreparedAst.SchemaVersion, bindSQL)) } ts.ctx.GetSessionVars().RemovePreparedStmt(ts.id) diff --git a/session/session.go b/session/session.go index e781a902afdf7..f685b44d0be43 100644 --- a/session/session.go +++ b/session/session.go @@ -314,7 +314,7 @@ func (s *session) cleanRetryInfo() { if ok { preparedAst = preparedObj.PreparedAst bindSQL := planner.GetBindSQL4PlanCache(s, preparedAst.Stmt) - cacheKey = plannercore.NewPSTMTPlanCacheKey(s.sessionVars, firstStmtID, preparedAst.SchemaVersion, bindSQL) + cacheKey = plannercore.NewPlanCacheKey(s.sessionVars, firstStmtID, preparedAst.SchemaVersion, bindSQL) } } } From 2fb260f0eb6ec237d912be5aea23b51a7c8f101a Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Thu, 23 Dec 2021 00:31:49 +0800 Subject: [PATCH 22/44] table/tables: make CI TestCacheTableBasicReadAndWrite more stable (#30924) close pingcap/tidb#30922 --- planner/core/logical_plan_builder.go | 16 ++++++++++++---- table/tables/cache_test.go | 2 ++ 2 files changed, 14 insertions(+), 4 deletions(-) diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index bd7ae44d36cdf..9721cc68d730c 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -58,6 +58,8 @@ import ( "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/plancodec" "github.com/pingcap/tidb/util/set" + "go.uber.org/zap" + "golang.org/x/sync/singleflight" ) const ( @@ -4207,10 +4209,14 @@ func (b *PlanBuilder) buildDataSource(ctx context.Context, tn *ast.TableName, as if r := recover(); r != nil { } }() - err := cachedTable.UpdateLockForRead(ctx, store, startTS) - if err != nil { - log.Warn("Update Lock Info Error") - } + _, err, _ := sf.Do(fmt.Sprintf("%d", tableInfo.ID), func() (interface{}, error) { + err := cachedTable.UpdateLockForRead(ctx, store, startTS) + if err != nil { + log.Warn("Update Lock Info Error", zap.Error(err)) + } + return nil, nil + }) + terror.Log(err) }() } } @@ -4238,6 +4244,8 @@ func (b *PlanBuilder) buildDataSource(ctx context.Context, tn *ast.TableName, as return result, nil } +var sf singleflight.Group + func (b *PlanBuilder) timeRangeForSummaryTable() QueryTimeRange { const defaultSummaryDuration = 30 * time.Minute hints := b.TableHints() diff --git a/table/tables/cache_test.go b/table/tables/cache_test.go index 62e48ccd24c94..a4dc5b4d43d68 100644 --- a/table/tables/cache_test.go +++ b/table/tables/cache_test.go @@ -221,6 +221,8 @@ func TestCacheTableBasicReadAndWrite(t *testing.T) { if tk.HasPlan("select * from write_tmp1", "UnionScan") { break } + // Wait for the cache to be loaded. + time.Sleep(50 * time.Millisecond) } require.True(t, i < 10) From efb53309799d08602d04354a75fecefc5fea0c02 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=B1=B1=E5=B2=9A?= <36239017+YuJuncen@users.noreply.github.com> Date: Thu, 23 Dec 2021 00:45:48 +0800 Subject: [PATCH 23/44] restore: use new ScatterRegions API (#30899) close pingcap/tidb#30425 --- .../backend/local/localhelper_test.go | 5 ++ br/pkg/restore/split.go | 59 ++++++++++++++++--- br/pkg/restore/split_client.go | 20 +++++++ br/pkg/restore/split_test.go | 58 +++++++++++++++--- 4 files changed, 125 insertions(+), 17 deletions(-) diff --git a/br/pkg/lightning/backend/local/localhelper_test.go b/br/pkg/lightning/backend/local/localhelper_test.go index d901b3c2711e6..52a9b71286087 100644 --- a/br/pkg/lightning/backend/local/localhelper_test.go +++ b/br/pkg/lightning/backend/local/localhelper_test.go @@ -69,6 +69,11 @@ func newTestClient( } } +// ScatterRegions scatters regions in a batch. +func (c *testClient) ScatterRegions(ctx context.Context, regionInfo []*restore.RegionInfo) error { + return nil +} + func (c *testClient) GetAllRegions() map[uint64]*restore.RegionInfo { c.mu.RLock() defer c.mu.RUnlock() diff --git a/br/pkg/restore/split.go b/br/pkg/restore/split.go index c962a2109aac6..ada8662522c21 100644 --- a/br/pkg/restore/split.go +++ b/br/pkg/restore/split.go @@ -24,6 +24,8 @@ import ( "github.com/tikv/pd/pkg/codec" "go.uber.org/multierr" "go.uber.org/zap" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" ) // Constants for split retry machinery. @@ -112,6 +114,7 @@ SplitRegions: regionMap[region.Region.GetId()] = region } for regionID, keys := range splitKeyMap { + log.Info("get split keys for region", zap.Int("len", len(keys)), zap.Uint64("region", regionID)) var newRegions []*RegionInfo region := regionMap[regionID] log.Info("split regions", @@ -142,6 +145,7 @@ SplitRegions: logutil.Keys(keys), rtree.ZapRanges(ranges)) continue SplitRegions } + log.Info("scattered regions", zap.Int("count", len(newRegions))) if len(newRegions) != len(keys) { log.Warn("split key count and new region count mismatch", zap.Int("new region count", len(newRegions)), @@ -294,8 +298,6 @@ func (rs *RegionSplitter) ScatterRegionsWithBackoffer(ctx context.Context, newRe log.Info("trying to scatter regions...", zap.Int("remain", len(newRegionSet))) var errs error for _, region := range newRegionSet { - // Wait for a while until the regions successfully split. - rs.waitForSplit(ctx, region.Region.Id) err := rs.client.ScatterRegion(ctx, region) if err == nil { // it is safe accroding to the Go language spec. @@ -328,15 +330,54 @@ func (rs *RegionSplitter) ScatterRegionsWithBackoffer(ctx context.Context, newRe } +// isUnsupportedError checks whether we should fallback to ScatterRegion API when meeting the error. +func isUnsupportedError(err error) bool { + s, ok := status.FromError(errors.Cause(err)) + if !ok { + // Not a gRPC error. Something other went wrong. + return false + } + // In two conditions, we fallback to ScatterRegion: + // (1) If the RPC endpoint returns UNIMPLEMENTED. (This is just for making test cases not be so magic.) + // (2) If the Message is "region 0 not found": + // In fact, PD reuses the gRPC endpoint `ScatterRegion` for the batch version of scattering. + // When the request contains the field `regionIDs`, it would use the batch version, + // Otherwise, it uses the old version and scatter the region with `regionID` in the request. + // When facing 4.x, BR(which uses v5.x PD clients and call `ScatterRegions`!) would set `regionIDs` + // which would be ignored by protocol buffers, and leave the `regionID` be zero. + // Then the older version of PD would try to search the region with ID 0. + // (Then it consistently fails, and returns "region 0 not found".) + return s.Code() == codes.Unimplemented || + strings.Contains(s.Message(), "region 0 not found") +} + // ScatterRegions scatter the regions. func (rs *RegionSplitter) ScatterRegions(ctx context.Context, newRegions []*RegionInfo) { - rs.ScatterRegionsWithBackoffer( - ctx, newRegions, - // backoff about 6s, or we give up scattering this region. - &exponentialBackoffer{ - attempt: 7, - baseBackoff: 100 * time.Millisecond, - }) + for _, region := range newRegions { + // Wait for a while until the regions successfully split. + rs.waitForSplit(ctx, region.Region.Id) + } + + err := utils.WithRetry(ctx, func() error { + err := rs.client.ScatterRegions(ctx, newRegions) + if isUnsupportedError(err) { + log.Warn("batch scatter isn't supported, rollback to old method", logutil.ShortError(err)) + rs.ScatterRegionsWithBackoffer( + ctx, newRegions, + // backoff about 6s, or we give up scattering this region. + &exponentialBackoffer{ + attempt: 7, + baseBackoff: 100 * time.Millisecond, + }) + return nil + } + return err + // the retry is for the temporary network errors during sending request. + }, &exponentialBackoffer{attempt: 3, baseBackoff: 500 * time.Millisecond}) + + if err != nil { + log.Warn("failed to batch scatter region", logutil.ShortError(err)) + } } func CheckRegionConsistency(startKey, endKey []byte, regions []*RegionInfo) error { diff --git a/br/pkg/restore/split_client.go b/br/pkg/restore/split_client.go index 10a9913d8e683..ed24fc3984a52 100755 --- a/br/pkg/restore/split_client.go +++ b/br/pkg/restore/split_client.go @@ -60,6 +60,8 @@ type SplitClient interface { BatchSplitRegionsWithOrigin(ctx context.Context, regionInfo *RegionInfo, keys [][]byte) (*RegionInfo, []*RegionInfo, error) // ScatterRegion scatters a specified region. ScatterRegion(ctx context.Context, regionInfo *RegionInfo) error + // ScatterRegions scatters regions in a batch. + ScatterRegions(ctx context.Context, regionInfo []*RegionInfo) error // GetOperator gets the status of operator of the specified region. GetOperator(ctx context.Context, regionID uint64) (*pdpb.GetOperatorResponse, error) // ScanRegion gets a list of regions, starts from the region that contains key. @@ -114,6 +116,24 @@ func (c *pdClient) needScatter(ctx context.Context) bool { return c.needScatterVal } +// ScatterRegions scatters regions in a batch. +func (c *pdClient) ScatterRegions(ctx context.Context, regionInfo []*RegionInfo) error { + c.mu.Lock() + defer c.mu.Unlock() + regionsID := make([]uint64, 0, len(regionInfo)) + for _, v := range regionInfo { + regionsID = append(regionsID, v.Region.Id) + } + resp, err := c.client.ScatterRegions(ctx, regionsID) + if err != nil { + return err + } + if pbErr := resp.GetHeader().GetError(); pbErr.GetType() != pdpb.ErrorType_OK { + return errors.Annotatef(berrors.ErrPDInvalidResponse, "pd returns error during batch scattering: %s", pbErr) + } + return nil +} + func (c *pdClient) GetStore(ctx context.Context, storeID uint64) (*metapb.Store, error) { c.mu.Lock() defer c.mu.Unlock() diff --git a/br/pkg/restore/split_test.go b/br/pkg/restore/split_test.go index 5e43d3378e579..fdfbba8df54d0 100644 --- a/br/pkg/restore/split_test.go +++ b/br/pkg/restore/split_test.go @@ -21,17 +21,19 @@ import ( "github.com/stretchr/testify/require" "github.com/tikv/pd/server/core" "github.com/tikv/pd/server/schedule/placement" + "go.uber.org/multierr" "google.golang.org/grpc/codes" "google.golang.org/grpc/status" ) type TestClient struct { - mu sync.RWMutex - stores map[uint64]*metapb.Store - regions map[uint64]*restore.RegionInfo - regionsInfo *core.RegionsInfo // For now it's only used in ScanRegions - nextRegionID uint64 - injectInScatter func(*restore.RegionInfo) error + mu sync.RWMutex + stores map[uint64]*metapb.Store + regions map[uint64]*restore.RegionInfo + regionsInfo *core.RegionsInfo // For now it's only used in ScanRegions + nextRegionID uint64 + injectInScatter func(*restore.RegionInfo) error + supportBatchScatter bool scattered map[uint64]bool } @@ -55,6 +57,36 @@ func NewTestClient( } } +func (c *TestClient) InstallBatchScatterSupport() { + c.supportBatchScatter = true +} + +// ScatterRegions scatters regions in a batch. +func (c *TestClient) ScatterRegions(ctx context.Context, regionInfo []*restore.RegionInfo) error { + if !c.supportBatchScatter { + return status.Error(codes.Unimplemented, "Ah, yep") + } + regions := map[uint64]*restore.RegionInfo{} + for _, region := range regionInfo { + regions[region.Region.Id] = region + } + var err error + for i := 0; i < 3; i++ { + if len(regions) == 0 { + return nil + } + for id, region := range regions { + splitErr := c.ScatterRegion(ctx, region) + if splitErr == nil { + delete(regions, id) + } + err = multierr.Append(err, splitErr) + + } + } + return nil +} + func (c *TestClient) GetAllRegions() map[uint64]*restore.RegionInfo { c.mu.RLock() defer c.mu.RUnlock() @@ -282,7 +314,18 @@ func TestScatterFinishInTime(t *testing.T) { // [, aay), [aay, bba), [bba, bbf), [bbf, bbh), [bbh, bbj), // [bbj, cca), [cca, xxe), [xxe, xxz), [xxz, ) func TestSplitAndScatter(t *testing.T) { - client := initTestClient() + t.Run("BatchScatter", func(t *testing.T) { + client := initTestClient() + client.InstallBatchScatterSupport() + runTestSplitAndScatterWith(t, client) + }) + t.Run("BackwardCompatibility", func(t *testing.T) { + client := initTestClient() + runTestSplitAndScatterWith(t, client) + }) +} + +func runTestSplitAndScatterWith(t *testing.T, client *TestClient) { ranges := initRanges() rewriteRules := initRewriteRules() regionSplitter := restore.NewRegionSplitter(client) @@ -320,7 +363,6 @@ func TestSplitAndScatter(t *testing.T) { t.Fatalf("region %d has not been scattered: %#v", key, regions[key]) } } - } // region: [, aay), [aay, bba), [bba, bbh), [bbh, cca), [cca, ) From b53f3499a61e5e9c5f6e9318144bc657e6bf7cd8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=8E=8B=E8=B6=85?= Date: Thu, 23 Dec 2021 01:01:48 +0800 Subject: [PATCH 24/44] *: when placement conflicts with tiflash, cancel the job (#30945) --- ddl/table.go | 3 +++ 1 file changed, 3 insertions(+) diff --git a/ddl/table.go b/ddl/table.go index 83f7ad0b0e58a..625b4f39df759 100644 --- a/ddl/table.go +++ b/ddl/table.go @@ -955,6 +955,7 @@ func (w *worker) onSetTableFlashReplica(t *meta.Meta, job *model.Job) (ver int64 } if replicaInfo.Count > 0 && tableHasPlacementSettings(tblInfo) { + job.State = model.JobStateCancelled return ver, errors.Trace(ErrIncompatibleTiFlashAndPlacement) } @@ -1279,6 +1280,7 @@ func onAlterTablePartitionPlacement(t *meta.Meta, job *model.Job) (ver int64, er } if tblInfo.TiFlashReplica != nil && tblInfo.TiFlashReplica.Count > 0 { + job.State = model.JobStateCancelled return 0, errors.Trace(ErrIncompatibleTiFlashAndPlacement) } @@ -1350,6 +1352,7 @@ func onAlterTablePlacement(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, } if tblInfo.TiFlashReplica != nil && tblInfo.TiFlashReplica.Count > 0 { + job.State = model.JobStateCancelled return 0, errors.Trace(ErrIncompatibleTiFlashAndPlacement) } From ec55c38d14aef78a58d5510f9394c6f4e10c00ee Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Thu, 23 Dec 2021 11:01:47 +0800 Subject: [PATCH 25/44] Makefile,tools: make CI great again! (#30828) close pingcap/tidb#30822 --- .gitignore | 1 + Makefile | 8 + tools/check/go.mod | 1 + tools/check/go.sum | 2 + tools/check/ut.go | 536 +++++++++++++++++++++++++++++++++++++++++++++ 5 files changed, 548 insertions(+) create mode 100644 tools/check/ut.go diff --git a/.gitignore b/.gitignore index c71bd34b114bd..48acac8905ae4 100644 --- a/.gitignore +++ b/.gitignore @@ -6,6 +6,7 @@ coverage.out *.iml *.swp *.log +*.test.bin tags profile.coverprofile explain_test diff --git a/Makefile b/Makefile index d49b0db744da4..30e01b2c1a1e1 100644 --- a/Makefile +++ b/Makefile @@ -126,6 +126,10 @@ devgotest: failpoint-enable $(GOTEST) -ldflags '$(TEST_LDFLAGS)' $(EXTRA_TEST_ARGS) -cover $(PACKAGES_TIDB_TESTS) -check.p true > gotest.log || { $(FAILPOINT_DISABLE); grep -v '^\([[]20\|PASS:\|ok \)' 'gotest.log'; exit 1; } @$(FAILPOINT_DISABLE) +ut: failpoint-enable tools/bin/ut + tools/bin/ut $(X); + @$(FAILPOINT_DISABLE) + gotest: failpoint-enable @echo "Running in native mode." @export log_level=info; export TZ='Asia/Shanghai'; \ @@ -220,6 +224,10 @@ failpoint-disable: tools/bin/failpoint-ctl # Restoring gofail failpoints... @$(FAILPOINT_DISABLE) +tools/bin/ut: tools/check/ut.go + cd tools/check; \ + $(GO) build -o ../bin/ut ut.go + tools/bin/megacheck: tools/check/go.mod cd tools/check; \ $(GO) build -o ../bin/megacheck honnef.co/go/tools/cmd/megacheck diff --git a/tools/check/go.mod b/tools/check/go.mod index 9c9c2b8d3da55..81ee48b2242cd 100644 --- a/tools/check/go.mod +++ b/tools/check/go.mod @@ -16,6 +16,7 @@ require ( github.com/pingcap/failpoint v0.0.0-20200702092429-9f69995143ce // indirect github.com/securego/gosec v0.0.0-20181211171558-12400f9a1ca7 github.com/shurcooL/vfsgen v0.0.0-20181202132449-6a9ea43bcacd + go.uber.org/automaxprocs v1.4.0 // indirect gopkg.in/alecthomas/gometalinter.v2 v2.0.12 // indirect gopkg.in/alecthomas/gometalinter.v3 v3.0.0 // indirect gopkg.in/alecthomas/kingpin.v2 v2.2.6 // indirect diff --git a/tools/check/go.sum b/tools/check/go.sum index ca6214c124823..776ad3f913a32 100644 --- a/tools/check/go.sum +++ b/tools/check/go.sum @@ -96,6 +96,8 @@ github.com/stretchr/testify v1.5.1/go.mod h1:5W2xD1RspED5o8YsWQXVCued0rvSQ+mT+I5 github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= go.uber.org/atomic v1.6.0 h1:Ezj3JGmsOnG1MoRWQkPBsKLe9DwWD9QeXzTRzzldNVk= go.uber.org/atomic v1.6.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= +go.uber.org/automaxprocs v1.4.0 h1:CpDZl6aOlLhReez+8S3eEotD7Jx0Os++lemPlMULQP0= +go.uber.org/automaxprocs v1.4.0/go.mod h1:/mTEdr7LvHhs0v7mjdxDreTz1OG5zdZGqgOnhWiR/+Q= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= diff --git a/tools/check/ut.go b/tools/check/ut.go new file mode 100644 index 0000000000000..6fad0f471fe2c --- /dev/null +++ b/tools/check/ut.go @@ -0,0 +1,536 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package main + +import ( + "bytes" + "fmt" + "math/rand" + "os" + "os/exec" + "path" + "runtime" + "strings" + "sync" + "time" + + // Set the correct when it runs inside docker. + _ "go.uber.org/automaxprocs" +) + +func usage() { + msg := `// run all tests +ut + +// show usage +ut -h + +// list all packages +ut list + +// list test cases of a single package +ut list $package + +// run all tests +ut run + +// run test all cases of a single package +ut run $package + +// run test cases of a single package +ut run $package $test + +// build all test package +ut build + +// build a test package +ut build xxx` + fmt.Println(msg) +} + +const modulePath = "github.com/pingcap/tidb" + +type task struct { + pkg string + test string + old bool +} + +var P int +var workDir string + +func cmdList(args ...string) { + pkgs, err := listPackages() + if err != nil { + fmt.Println("list package error", err) + return + } + + // list all packages + if len(args) == 0 { + for _, pkg := range pkgs { + fmt.Println(pkg) + } + return + } + + // list test case of a single package + if len(args) == 1 { + pkg := args[0] + pkgs = filter(pkgs, func(s string) bool { return s == pkg }) + if len(pkgs) != 1 { + fmt.Println("package not exist", pkg) + return + } + + err := buildTestBinary(pkg) + if err != nil { + fmt.Println("build package error", pkg, err) + return + } + exist, err := testBinaryExist(pkg) + if err != nil { + fmt.Println("check test binary existance error", err) + return + } + if !exist { + fmt.Println("no test case in ", pkg) + return + } + + res, err := listTestCases(pkg, nil) + if err != nil { + fmt.Println("list test cases for package error", err) + return + } + for _, x := range res { + fmt.Println(x.test) + } + } +} + +func cmdBuild(args ...string) { + pkgs, err := listPackages() + if err != nil { + fmt.Println("list package error", err) + return + } + + // build all packages + if len(args) == 0 { + for _, pkg := range pkgs { + err := buildTestBinary(pkg) + if err != nil { + fmt.Println("build package error", pkg, err) + return + } + } + return + } + + // build test binary of a single package + if len(args) >= 1 { + pkg := args[0] + err := buildTestBinary(pkg) + if err != nil { + fmt.Println("build package error", pkg, err) + return + } + } +} + +func cmdRun(args ...string) { + var err error + pkgs, err := listPackages() + if err != nil { + fmt.Println("list packages error", err) + return + } + tasks := make([]task, 0, 5000) + // run all tests + if len(args) == 0 { + for _, pkg := range pkgs { + fmt.Println("handling package", pkg) + err := buildTestBinary(pkg) + if err != nil { + fmt.Println("build package error", pkg, err) + return + } + + exist, err := testBinaryExist(pkg) + if err != nil { + fmt.Println("check test binary existance error", err) + return + } + if !exist { + fmt.Println("no test case in ", pkg) + continue + } + + tasks, err = listTestCases(pkg, tasks) + if err != nil { + fmt.Println("list test cases error", err) + return + } + } + } + + // run tests for a single package + if len(args) == 1 { + pkg := args[0] + err := buildTestBinary(pkg) + if err != nil { + fmt.Println("build package error", pkg, err) + return + } + exist, err := testBinaryExist(pkg) + if err != nil { + fmt.Println("check test binary existance error", err) + return + } + + if !exist { + fmt.Println("no test case in ", pkg) + return + } + tasks, err = listTestCases(pkg, tasks) + if err != nil { + fmt.Println("list test cases error", err) + return + } + } + + // run a single test + if len(args) == 2 { + pkg := args[0] + err := buildTestBinary(pkg) + if err != nil { + fmt.Println("build package error", pkg, err) + return + } + exist, err := testBinaryExist(pkg) + if err != nil { + fmt.Println("check test binary existance error", err) + return + } + if !exist { + fmt.Println("no test case in ", pkg) + return + } + + tasks, err = listTestCases(pkg, tasks) + if err != nil { + fmt.Println("list test cases error", err) + return + } + // filter the test case to run + tmp := tasks[:0] + for _, task := range tasks { + if strings.Contains(task.test, args[1]) { + tmp = append(tmp, task) + } + } + tasks = tmp + } + fmt.Println("building task finish...", len(tasks)) + + numactl := numactlExist() + taskCh := make(chan task, 100) + var wg sync.WaitGroup + for i := 0; i < P; i++ { + n := numa{fmt.Sprintf("%d", i), numactl} + wg.Add(1) + go n.worker(&wg, taskCh) + } + + shuffle(tasks) + for _, task := range tasks { + taskCh <- task + } + close(taskCh) + wg.Wait() +} + +func main() { + // Get the correct count of CPU if it's in docker. + P = runtime.GOMAXPROCS(0) + rand.Seed(time.Now().Unix()) + var err error + workDir, err = os.Getwd() + if err != nil { + fmt.Println("os.Getwd() error", err) + } + + if len(os.Args) == 1 { + // run all tests + cmdRun() + return + } + + if len(os.Args) >= 2 { + switch os.Args[1] { + case "list": + cmdList(os.Args[2:]...) + case "build": + cmdBuild(os.Args[2:]...) + case "run": + cmdRun(os.Args[2:]...) + default: + usage() + } + } +} + +func listTestCases(pkg string, tasks []task) ([]task, error) { + newCases, err := listNewTestCases(pkg) + if err != nil { + fmt.Println("list test case error", pkg, err) + return nil, withTrace(err) + } + for _, c := range newCases { + tasks = append(tasks, task{pkg, c, false}) + } + + oldCases, err := listOldTestCases(pkg) + if err != nil { + fmt.Println("list old test case error", pkg, err) + return nil, withTrace(err) + } + for _, c := range oldCases { + tasks = append(tasks, task{pkg, c, true}) + } + return tasks, nil +} + +func listPackages() ([]string, error) { + cmd := exec.Command("go", "list", "./...") + ss, err := cmdToLines(cmd) + if err != nil { + return nil, withTrace(err) + } + + ret := ss[:0] + for _, s := range ss { + if !strings.HasPrefix(s, modulePath) { + continue + } + pkg := s[len(modulePath)+1:] + if skipDIR(pkg) { + continue + } + ret = append(ret, pkg) + } + return ret, nil +} + +type numa struct { + cpu string + numactl bool +} + +func (n *numa) worker(wg *sync.WaitGroup, ch chan task) { + defer wg.Done() + for t := range ch { + start := time.Now() + if err := n.runTestCase(t.pkg, t.test, t.old); err != nil { + fmt.Println("run test case error", t.pkg, t.test, t.old, time.Since(start), err) + } + } +} + +func (n *numa) runTestCase(pkg string, fn string, old bool) error { + exe := "./" + testFileName(pkg) + var cmd *exec.Cmd + if n.numactl { + cmd = n.testCommandWithNumaCtl(exe, fn, old) + } else { + cmd = n.testCommand(exe, fn, old) + } + cmd.Dir = path.Join(workDir, pkg) + _, err := cmd.CombinedOutput() + if err != nil { + // fmt.Println("run test case error", pkg, fn, string(output)) + return err + } + return nil +} + +func (n *numa) testCommandWithNumaCtl(exe string, fn string, old bool) *exec.Cmd { + if old { + // numactl --physcpubind 3 -- session.test -test.run '^TestT$' -check.f testTxnStateSerialSuite.TestTxnInfoWithPSProtoco + return exec.Command( + "numactl", "--physcpubind", n.cpu, "--", + exe, + "-test.timeout", "20s", + "-test.cpu", "1", "-test.run", "^TestT$", "-check.f", fn) + } + + // numactl --physcpubind 3 -- session.test -test.run TestClusteredPrefixColum + return exec.Command( + "numactl", "--physcpubind", n.cpu, "--", + exe, + "-test.timeout", "20s", + "-test.cpu", "1", "-test.run", fn) +} + +func (n *numa) testCommand(exe string, fn string, old bool) *exec.Cmd { + if old { + // session.test -test.run '^TestT$' -check.f testTxnStateSerialSuite.TestTxnInfoWithPSProtoco + return exec.Command( + exe, + "-test.timeout", "20s", + "-test.cpu", "1", "-test.run", "^TestT$", "-check.f", fn) + } + + // session.test -test.run TestClusteredPrefixColum + return exec.Command( + exe, + "-test.timeout", "20s", + "-test.cpu", "1", "-test.run", fn) +} + +func skipDIR(pkg string) bool { + skipDir := []string{"br", "cmd", "dumpling"} + for _, ignore := range skipDir { + if strings.HasPrefix(pkg, ignore) { + return true + } + } + return false +} + +func buildTestBinary(pkg string) error { + // go test -c + cmd := exec.Command("go", "test", "-c", "-vet", "off", "-o", testFileName(pkg)) + cmd.Dir = path.Join(workDir, pkg) + err := cmd.Run() + return withTrace(err) +} + +func testBinaryExist(pkg string) (bool, error) { + _, err := os.Stat(testFileFullPath(pkg)) + if err != nil { + if _, ok := err.(*os.PathError); ok { + return false, nil + } + } + return true, withTrace(err) +} +func numactlExist() bool { + find, err := exec.Command("which", "numactl").Output() + if err == nil && len(find) > 0 { + return true + } + return false +} + +func testFileName(pkg string) string { + _, file := path.Split(pkg) + return file+".test.bin" +} + +func testFileFullPath(pkg string) string { + return path.Join(workDir, pkg, testFileName(pkg)) +} + +func listNewTestCases(pkg string) ([]string, error) { + exe := "./" + testFileName(pkg) + + // session.test -test.list Test + cmd := exec.Command(exe, "-test.list", "Test") + cmd.Dir = path.Join(workDir, pkg) + res, err := cmdToLines(cmd) + if err != nil { + return nil, withTrace(err) + } + return filter(res, func(s string) bool { + return strings.HasPrefix(s, "Test") && s != "TestT" && s != "TestBenchDaily" + }), nil +} + +func listOldTestCases(pkg string) (res []string, err error) { + exe := "./" + testFileName(pkg) + + // Maybe the restructure is finish on this package. + cmd := exec.Command(exe, "-h") + cmd.Dir = path.Join(workDir, pkg) + buf, err := cmd.CombinedOutput() + if err != nil { + err = withTrace(err) + return + } + if !bytes.Contains(buf, []byte("check.list")) { + // there is no old test case in pkg + return + } + + // session.test -test.run TestT -check.list Test + cmd = exec.Command(exe, "-test.run", "^TestT$", "-check.list", "Test") + cmd.Dir = path.Join(workDir, pkg) + res, err = cmdToLines(cmd) + res = filter(res, func(s string) bool { return strings.Contains(s, "Test") }) + return res, withTrace(err) +} + +func cmdToLines(cmd *exec.Cmd) ([]string, error) { + res, err := cmd.Output() + if err != nil { + return nil, withTrace(err) + } + ss := bytes.Split(res, []byte{'\n'}) + ret := make([]string, len(ss)) + for i, s := range ss { + ret[i] = string(s) + } + return ret, nil +} + +func filter(input []string, f func(string) bool) []string { + ret := input[:0] + for _, s := range input { + if f(s) { + ret = append(ret, s) + } + } + return ret +} + +func shuffle(tasks []task) { + for i := 0; i < len(tasks); i++ { + pos := rand.Intn(len(tasks)) + tasks[i], tasks[pos] = tasks[pos], tasks[i] + } +} + +type errWithStack struct { + err error + buf []byte +} + +func (e *errWithStack) Error() string { + return e.err.Error() + "\n" + string(e.buf) +} + +func withTrace(err error) error { + if err == nil { + return err + } + if _, ok := err.(*errWithStack); ok { + return err + } + var stack [4096]byte + sz := runtime.Stack(stack[:], false) + return &errWithStack{err, stack[:sz]} +} From dfc25ff06bf6955652c6f027a4e8718372d3ac6f Mon Sep 17 00:00:00 2001 From: Yujie Xia Date: Thu, 23 Dec 2021 11:13:47 +0800 Subject: [PATCH 26/44] br/pkg/membuf: remove global buffer pool (#29934) --- br/pkg/lightning/backend/local/local.go | 15 ++-- br/pkg/lightning/backend/local/local_test.go | 6 +- br/pkg/membuf/buffer.go | 81 +++++++++++++++----- br/pkg/membuf/buffer_test.go | 20 +++-- 4 files changed, 91 insertions(+), 31 deletions(-) diff --git a/br/pkg/lightning/backend/local/local.go b/br/pkg/lightning/backend/local/local.go index eb7ab37802e4e..b703acec49395 100644 --- a/br/pkg/lightning/backend/local/local.go +++ b/br/pkg/lightning/backend/local/local.go @@ -150,9 +150,9 @@ type local struct { duplicateDetection bool duplicateDB *pebble.DB errorMgr *errormanager.ErrorManager -} -var bufferPool = membuf.NewPool(1024, manual.Allocator{}) + bufferPool *membuf.Pool +} func openDuplicateDB(storeDir string) (*pebble.DB, error) { dbPath := filepath.Join(storeDir, duplicateDBName) @@ -244,6 +244,8 @@ func NewLocalBackend( checkTiKVAvaliable: cfg.App.CheckRequirements, duplicateDB: duplicateDB, errorMgr: errorMgr, + + bufferPool: membuf.NewPool(membuf.WithAllocator(manual.Allocator{})), } local.conns = common.NewGRPCConns() if err = local.checkMultiIngestSupport(ctx); err != nil { @@ -423,6 +425,7 @@ func (local *local) Close() { engine.unlock() } local.conns.Close() + local.bufferPool.Destroy() if local.duplicateDB != nil { // Check whether there are duplicates. @@ -776,7 +779,7 @@ func (local *local) WriteToTiKV( requests = append(requests, req) } - bytesBuf := bufferPool.NewBuffer() + bytesBuf := local.bufferPool.NewBuffer() defer bytesBuf.Destroy() pairs := make([]*sst.Pair, 0, local.batchWriteKVPairs) count := 0 @@ -1664,14 +1667,14 @@ func (local *local) LocalWriter(ctx context.Context, cfg *backend.LocalWriterCon return nil, errors.Errorf("could not find engine for %s", engineUUID.String()) } engine := e.(*Engine) - return openLocalWriter(cfg, engine, local.localWriterMemCacheSize) + return openLocalWriter(cfg, engine, local.localWriterMemCacheSize, local.bufferPool.NewBuffer()) } -func openLocalWriter(cfg *backend.LocalWriterConfig, engine *Engine, cacheSize int64) (*Writer, error) { +func openLocalWriter(cfg *backend.LocalWriterConfig, engine *Engine, cacheSize int64, kvBuffer *membuf.Buffer) (*Writer, error) { w := &Writer{ engine: engine, memtableSizeLimit: cacheSize, - kvBuffer: bufferPool.NewBuffer(), + kvBuffer: kvBuffer, isKVSorted: cfg.IsKVSorted, isWriteBatchSorted: true, } diff --git a/br/pkg/lightning/backend/local/local_test.go b/br/pkg/lightning/backend/local/local_test.go index 747034068c463..35c13692dce3e 100644 --- a/br/pkg/lightning/backend/local/local_test.go +++ b/br/pkg/lightning/backend/local/local_test.go @@ -46,6 +46,7 @@ import ( "github.com/pingcap/tidb/br/pkg/lightning/backend/kv" "github.com/pingcap/tidb/br/pkg/lightning/common" "github.com/pingcap/tidb/br/pkg/lightning/mydump" + "github.com/pingcap/tidb/br/pkg/membuf" "github.com/pingcap/tidb/br/pkg/mock" "github.com/pingcap/tidb/br/pkg/pdutil" "github.com/pingcap/tidb/br/pkg/restore" @@ -357,7 +358,10 @@ func testLocalWriter(c *C, needSort bool, partitialSort bool) { f.wg.Add(1) go f.ingestSSTLoop() sorted := needSort && !partitialSort - w, err := openLocalWriter(&backend.LocalWriterConfig{IsKVSorted: sorted}, f, 1024) + pool := membuf.NewPool() + defer pool.Destroy() + kvBuffer := pool.NewBuffer() + w, err := openLocalWriter(&backend.LocalWriterConfig{IsKVSorted: sorted}, f, 1024, kvBuffer) c.Assert(err, IsNil) ctx := context.Background() diff --git a/br/pkg/membuf/buffer.go b/br/pkg/membuf/buffer.go index 172d99baec9aa..49ffbae8afdf3 100644 --- a/br/pkg/membuf/buffer.go +++ b/br/pkg/membuf/buffer.go @@ -14,9 +14,11 @@ package membuf -const bigValueSize = 1 << 16 // 64K - -var allocBufLen = 1 << 20 // 1M +const ( + defaultPoolSize = 1024 + defaultBlockSize = 1 << 20 // 1M + defaultLargeAllocThreshold = 1 << 16 // 64K +) // Allocator is the abstract interface for allocating and freeing memory. type Allocator interface { @@ -38,30 +40,71 @@ func (stdAllocator) Free(_ []byte) {} // garbage collector which always release the memory so late. Use a fixed size chan to reuse // can decrease the memory usage to 1/3 compare with sync.Pool. type Pool struct { - allocator Allocator - recycleCh chan []byte + allocator Allocator + blockSize int + blockCache chan []byte + largeAllocThreshold int +} + +// Option configures a pool. +type Option func(p *Pool) + +// WithPoolSize configures how many blocks cached by this pool. +func WithPoolSize(size int) Option { + return func(p *Pool) { + p.blockCache = make(chan []byte, size) + } +} + +// WithBlockSize configures the size of each block. +func WithBlockSize(size int) Option { + return func(p *Pool) { + p.blockSize = size + } +} + +// WithAllocator specifies the allocator used by pool to allocate and free memory. +func WithAllocator(allocator Allocator) Option { + return func(p *Pool) { + p.allocator = allocator + } +} + +// WithLargeAllocThreshold configures the threshold for large allocation of a Buffer. +// If allocate size is larger than this threshold, bytes will be allocated directly +// by the make built-in function and won't be tracked by the pool. +func WithLargeAllocThreshold(threshold int) Option { + return func(p *Pool) { + p.largeAllocThreshold = threshold + } } // NewPool creates a new pool. -func NewPool(size int, allocator Allocator) *Pool { - return &Pool{ - allocator: allocator, - recycleCh: make(chan []byte, size), +func NewPool(opts ...Option) *Pool { + p := &Pool{ + allocator: stdAllocator{}, + blockSize: defaultBlockSize, + blockCache: make(chan []byte, defaultPoolSize), + largeAllocThreshold: defaultLargeAllocThreshold, + } + for _, opt := range opts { + opt(p) } + return p } func (p *Pool) acquire() []byte { select { - case b := <-p.recycleCh: + case b := <-p.blockCache: return b default: - return p.allocator.Alloc(allocBufLen) + return p.allocator.Alloc(p.blockSize) } } func (p *Pool) release(b []byte) { select { - case p.recycleCh <- b: + case p.blockCache <- b: default: p.allocator.Free(b) } @@ -72,10 +115,12 @@ func (p *Pool) NewBuffer() *Buffer { return &Buffer{pool: p, bufs: make([][]byte, 0, 128), curBufIdx: -1} } -var globalPool = NewPool(1024, stdAllocator{}) - -// NewBuffer creates a new buffer in global pool. -func NewBuffer() *Buffer { return globalPool.NewBuffer() } +func (p *Pool) Destroy() { + close(p.blockCache) + for b := range p.blockCache { + p.allocator.Free(b) + } +} // Buffer represents the reuse buffer. type Buffer struct { @@ -123,12 +168,12 @@ func (b *Buffer) Destroy() { // TotalSize represents the total memory size of this Buffer. func (b *Buffer) TotalSize() int64 { - return int64(len(b.bufs) * allocBufLen) + return int64(len(b.bufs) * b.pool.blockSize) } // AllocBytes allocates bytes with the given length. func (b *Buffer) AllocBytes(n int) []byte { - if n > bigValueSize { + if n > b.pool.largeAllocThreshold { return make([]byte, n) } if b.curIdx+n > b.curBufLen { diff --git a/br/pkg/membuf/buffer_test.go b/br/pkg/membuf/buffer_test.go index c5d095d299f9c..fa45c5c4e34b1 100644 --- a/br/pkg/membuf/buffer_test.go +++ b/br/pkg/membuf/buffer_test.go @@ -21,10 +21,6 @@ import ( "github.com/stretchr/testify/require" ) -func init() { - allocBufLen = 1024 -} - type testAllocator struct { allocs int frees int @@ -41,7 +37,13 @@ func (t *testAllocator) Free(_ []byte) { func TestBufferPool(t *testing.T) { allocator := &testAllocator{} - pool := NewPool(2, allocator) + pool := NewPool( + WithPoolSize(2), + WithAllocator(allocator), + WithBlockSize(1024), + WithLargeAllocThreshold(512), + ) + defer pool.Destroy() bytesBuf := pool.NewBuffer() bytesBuf.AllocBytes(256) @@ -53,6 +55,10 @@ func TestBufferPool(t *testing.T) { bytesBuf.AllocBytes(767) require.Equal(t, 2, allocator.allocs) + largeBytes := bytesBuf.AllocBytes(513) + require.Equal(t, 513, len(largeBytes)) + require.Equal(t, 2, allocator.allocs) + require.Equal(t, 0, allocator.frees) bytesBuf.Destroy() require.Equal(t, 0, allocator.frees) @@ -67,7 +73,9 @@ func TestBufferPool(t *testing.T) { } func TestBufferIsolation(t *testing.T) { - bytesBuf := NewBuffer() + pool := NewPool(WithBlockSize(1024)) + defer pool.Destroy() + bytesBuf := pool.NewBuffer() defer bytesBuf.Destroy() b1 := bytesBuf.AllocBytes(16) From b9eb9f67688945616e796a81e215a79cbc57df10 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Thu, 23 Dec 2021 11:59:48 +0800 Subject: [PATCH 27/44] ddl: add format error for incorrect dict syntax in the placement rule (#30919) close pingcap/tidb#30454 --- ddl/placement/errors.go | 4 +++ ddl/placement/rule.go | 16 ++++++++++++ ddl/placement/rule_test.go | 53 ++++++++++++++++++++++++++------------ 3 files changed, 57 insertions(+), 16 deletions(-) diff --git a/ddl/placement/errors.go b/ddl/placement/errors.go index 4f98e0fa4c2ec..b609827bd4ce7 100644 --- a/ddl/placement/errors.go +++ b/ddl/placement/errors.go @@ -43,4 +43,8 @@ var ( ErrNoRulesToDrop = errors.New("no rule of such role to drop") // ErrInvalidPlacementOptions is from bundle.go. ErrInvalidPlacementOptions = errors.New("invalid placement option") + // ErrInvalidConstraintsMappingWrongSeparator is wrong separator in mapping. + ErrInvalidConstraintsMappingWrongSeparator = errors.New("mappings use a colon and space (“: ”) to mark each key/value pair") + // ErrInvalidConstraintsMappingNoColonFound is no colon found in mapping. + ErrInvalidConstraintsMappingNoColonFound = errors.New("no colon found") ) diff --git a/ddl/placement/rule.go b/ddl/placement/rule.go index 88cd5067153f8..518dd369414b2 100644 --- a/ddl/placement/rule.go +++ b/ddl/placement/rule.go @@ -16,6 +16,7 @@ package placement import ( "fmt" + "regexp" "strings" "gopkg.in/yaml.v2" @@ -61,6 +62,18 @@ func NewRule(role PeerRoleType, replicas uint64, cnst Constraints) *Rule { } } +var wrongSeparatorRegexp = regexp.MustCompile(`[^"':]+:\d`) + +func getYamlMapFormatError(str string) error { + if !strings.Contains(str, ":") { + return ErrInvalidConstraintsMappingNoColonFound + } + if wrongSeparatorRegexp.MatchString(str) { + return ErrInvalidConstraintsMappingWrongSeparator + } + return nil +} + // NewRules constructs []*Rule from a yaml-compatible representation of // 'array' or 'dict' constraints. // Refer to https://github.com/pingcap/tidb/blob/master/docs/design/2020-06-24-placement-rules-in-sql.md. @@ -86,6 +99,9 @@ func NewRules(role PeerRoleType, replicas uint64, cnstr string) ([]*Rule, error) ruleCnt := 0 for labels, cnt := range constraints2 { if cnt <= 0 { + if err := getYamlMapFormatError(string(cnstbytes)); err != nil { + return rules, err + } return rules, fmt.Errorf("%w: count of labels '%s' should be positive, but got %d", ErrInvalidConstraintsMapcnt, labels, cnt) } ruleCnt += cnt diff --git a/ddl/placement/rule_test.go b/ddl/placement/rule_test.go index 9432448127a4a..f38819c278998 100644 --- a/ddl/placement/rule_test.go +++ b/ddl/placement/rule_test.go @@ -16,21 +16,20 @@ package placement import ( "errors" + "reflect" + "testing" . "github.com/pingcap/check" + "github.com/stretchr/testify/require" ) -var _ = Suite(&testRuleSuite{}) - -type testRuleSuite struct{} - -func (t *testRuleSuite) TestClone(c *C) { +func TestClone(t *testing.T) { rule := &Rule{ID: "434"} newRule := rule.Clone() newRule.ID = "121" - c.Assert(rule, DeepEquals, &Rule{ID: "434"}) - c.Assert(newRule, DeepEquals, &Rule{ID: "121"}) + require.Equal(t, &Rule{ID: "434"}, rule) + require.Equal(t, &Rule{ID: "121"}, newRule) } func matchRules(t1, t2 []*Rule, prefix string, c *C) { @@ -50,7 +49,22 @@ func matchRules(t1, t2 []*Rule, prefix string, c *C) { } } -func (t *testRuleSuite) TestNewRuleAndNewRules(c *C) { +func matchRulesT(t1, t2 []*Rule, prefix string, t *testing.T) { + require.Equal(t, len(t2), len(t1), prefix) + for i := range t1 { + found := false + for j := range t2 { + ok := reflect.DeepEqual(t2[j], t1[i]) + if ok { + found = true + break + } + } + require.True(t, found, "%s\n\ncan not found %d rule\n%+v\n%+v", prefix, i, t1[i], t2) + } +} + +func TestNewRuleAndNewRules(t *testing.T) { type TestCase struct { name string input string @@ -58,7 +72,7 @@ func (t *testRuleSuite) TestNewRuleAndNewRules(c *C) { output []*Rule err error } - tests := []TestCase{} + var tests []TestCase tests = append(tests, TestCase{ name: "empty constraints", @@ -175,14 +189,21 @@ func (t *testRuleSuite) TestNewRuleAndNewRules(c *C) { err: ErrInvalidConstraintFormat, }) - for _, t := range tests { - comment := Commentf("[%s]", t.name) - output, err := NewRules(Voter, t.replicas, t.input) - if t.err == nil { - c.Assert(err, IsNil, comment) - matchRules(t.output, output, comment.CheckCommentString(), c) + tests = append(tests, TestCase{ + name: "invalid map separator", + input: `{+region=us-east-2:2}`, + replicas: 6, + err: ErrInvalidConstraintsMappingWrongSeparator, + }) + + for _, tt := range tests { + comment := Commentf("[%s]", tt.name) + output, err := NewRules(Voter, tt.replicas, tt.input) + if tt.err == nil { + require.NoError(t, err, comment) + matchRulesT(tt.output, output, comment.CheckCommentString(), t) } else { - c.Assert(errors.Is(err, t.err), IsTrue, Commentf("[%s]\n%s\n%s\n", t.name, err, t.err)) + require.True(t, errors.Is(err, tt.err), "[%s]\n%s\n%s\n", tt.name, err, tt.err) } } } From 7121bf0289734aa71197fb631796048d8e78ac63 Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Thu, 23 Dec 2021 13:39:47 +0800 Subject: [PATCH 28/44] planner: fix index merge plan when expr cannot be pushed to tikv (#30341) close pingcap/tidb#30200 --- cmd/explaintest/r/index_merge.result | 60 +++++------ planner/core/find_best_task.go | 57 ++++++++--- planner/core/integration_test.go | 99 +++++++++++++++++++ planner/core/logical_plans.go | 2 +- planner/core/stats.go | 90 +++++++++++++---- planner/core/task.go | 8 +- .../core/testdata/integration_suite_out.json | 4 +- session/session_test.go | 4 +- table/tables/cache_test.go | 2 +- 9 files changed, 259 insertions(+), 67 deletions(-) diff --git a/cmd/explaintest/r/index_merge.result b/cmd/explaintest/r/index_merge.result index f790569635b28..261acdb49d8a2 100644 --- a/cmd/explaintest/r/index_merge.result +++ b/cmd/explaintest/r/index_merge.result @@ -237,11 +237,11 @@ insert into t1(c1, c2) values(1, 1), (2, 2), (3, 3), (4, 4), (5, 5); explain select /*+ use_index_merge(t1) */ * from t1 where c1 < 10 or c2 < 10 and c3 < 10 order by 1; id estRows task access object operator info Sort_5 4060.74 root test.t1.c1 -└─IndexMerge_12 2250.55 root - ├─IndexRangeScan_8(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo - ├─IndexRangeScan_9(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo - └─Selection_11(Probe) 2250.55 cop[tikv] or(lt(test.t1.c1, 10), and(lt(test.t1.c2, 10), lt(test.t1.c3, 10))) - └─TableRowIDScan_10 5542.21 cop[tikv] table:t1 keep order:false, stats:pseudo +└─Selection_12 2250.55 root or(lt(test.t1.c1, 10), and(lt(test.t1.c2, 10), lt(test.t1.c3, 10))) + └─IndexMerge_11 5542.21 root + ├─IndexRangeScan_8(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo + ├─IndexRangeScan_9(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo + └─TableRowIDScan_10(Probe) 5542.21 cop[tikv] table:t1 keep order:false, stats:pseudo select /*+ use_index_merge(t1) */ * from t1 where c1 < 10 or c2 < 10 and c3 < 10 order by 1; c1 c2 c3 1 1 2 @@ -252,11 +252,11 @@ c1 c2 c3 explain select /*+ use_index_merge(t1) */ * from t1 where c1 < 10 or c2 < 10 and c3 = c1 + c2 order by 1; id estRows task access object operator info Sort_5 5098.44 root test.t1.c1 -└─IndexMerge_12 2825.66 root - ├─IndexRangeScan_8(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo - ├─IndexRangeScan_9(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo - └─Selection_11(Probe) 2825.66 cop[tikv] or(lt(test.t1.c1, 10), and(lt(test.t1.c2, 10), eq(test.t1.c3, plus(test.t1.c1, test.t1.c2)))) - └─TableRowIDScan_10 5542.21 cop[tikv] table:t1 keep order:false, stats:pseudo +└─Selection_12 2825.66 root or(lt(test.t1.c1, 10), and(lt(test.t1.c2, 10), eq(test.t1.c3, plus(test.t1.c1, test.t1.c2)))) + └─IndexMerge_11 5542.21 root + ├─IndexRangeScan_8(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo + ├─IndexRangeScan_9(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo + └─TableRowIDScan_10(Probe) 5542.21 cop[tikv] table:t1 keep order:false, stats:pseudo select /*+ use_index_merge(t1) */ * from t1 where c1 < 10 or c2 < 10 and c3 = c1 + c2 order by 1; c1 c2 c3 1 1 2 @@ -267,11 +267,11 @@ c1 c2 c3 explain select /*+ use_index_merge(t1) */ * from t1 where c1 < 10 or c2 < 10 and substring(c3, c2) order by 1; id estRows task access object operator info Sort_5 5098.44 root test.t1.c1 -└─IndexMerge_12 2825.66 root - ├─IndexRangeScan_8(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo - ├─IndexRangeScan_9(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo - └─Selection_11(Probe) 2825.66 cop[tikv] or(lt(test.t1.c1, 10), and(lt(test.t1.c2, 10), istrue_with_null(cast(substring(cast(test.t1.c3, var_string(20)), test.t1.c2), double BINARY)))) - └─TableRowIDScan_10 5542.21 cop[tikv] table:t1 keep order:false, stats:pseudo +└─Selection_12 2825.66 root or(lt(test.t1.c1, 10), and(lt(test.t1.c2, 10), istrue_with_null(cast(substring(cast(test.t1.c3, var_string(20)), test.t1.c2), double BINARY)))) + └─IndexMerge_11 5542.21 root + ├─IndexRangeScan_8(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo + ├─IndexRangeScan_9(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo + └─TableRowIDScan_10(Probe) 5542.21 cop[tikv] table:t1 keep order:false, stats:pseudo select /*+ use_index_merge(t1) */ * from t1 where c1 < 10 or c2 < 10 and substring(c3, c2) order by 1; c1 c2 c3 1 1 2 @@ -282,11 +282,11 @@ c1 c2 c3 explain select /*+ use_index_merge(t1) */ * from t1 where c1 < 10 or c2 < 10 and c3 order by 1; id estRows task access object operator info Sort_5 4800.37 root test.t1.c1 -└─IndexMerge_12 2660.47 root - ├─IndexRangeScan_8(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo - ├─IndexRangeScan_9(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo - └─Selection_11(Probe) 2660.47 cop[tikv] or(lt(test.t1.c1, 10), and(lt(test.t1.c2, 10), test.t1.c3)) - └─TableRowIDScan_10 5542.21 cop[tikv] table:t1 keep order:false, stats:pseudo +└─Selection_12 2660.47 root or(lt(test.t1.c1, 10), and(lt(test.t1.c2, 10), test.t1.c3)) + └─IndexMerge_11 5542.21 root + ├─IndexRangeScan_8(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo + ├─IndexRangeScan_9(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo + └─TableRowIDScan_10(Probe) 5542.21 cop[tikv] table:t1 keep order:false, stats:pseudo select /*+ use_index_merge(t1) */ * from t1 where c1 < 10 or c2 < 10 and c3 order by 1; c1 c2 c3 1 1 2 @@ -302,11 +302,11 @@ select /*+ use_index_merge(t1) */ * from t1 where c1 < 10 or c2 < 10 and c3 < 10 explain select * from t1 where c1 < 10 or c2 < 10 and c3 < 10 order by 1; id estRows task access object operator info Sort_5 4060.74 root test.t1.c1 -└─IndexMerge_12 2250.55 root - ├─IndexRangeScan_8(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo - ├─IndexRangeScan_9(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo - └─Selection_11(Probe) 2250.55 cop[tikv] or(lt(test.t1.c1, 10), and(lt(test.t1.c2, 10), lt(test.t1.c3, 10))) - └─TableRowIDScan_10 5542.21 cop[tikv] table:t1 keep order:false, stats:pseudo +└─Selection_12 2250.55 root or(lt(test.t1.c1, 10), and(lt(test.t1.c2, 10), lt(test.t1.c3, 10))) + └─IndexMerge_11 5542.21 root + ├─IndexRangeScan_8(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo + ├─IndexRangeScan_9(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo + └─TableRowIDScan_10(Probe) 5542.21 cop[tikv] table:t1 keep order:false, stats:pseudo select * from t1 where c1 < 10 or c2 < 10 and c3 < 10 order by 1; c1 c2 c3 1 1 2 @@ -722,11 +722,11 @@ c1 c2 c3 c4 c5 explain select /*+ use_index_merge(t1) */ * from t1 where (c1 < 10 or c2 < 10) and substring(c3, 1, 1) = '1' order by 1; id estRows task access object operator info Sort_5 4433.77 root test.t1.c1 -└─IndexMerge_12 4433.77 root - ├─IndexRangeScan_8(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo - ├─IndexRangeScan_9(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo - └─Selection_11(Probe) 4433.77 cop[tikv] eq(substring(cast(test.t1.c3, var_string(20)), 1, 1), "1") - └─TableRowIDScan_10 5542.21 cop[tikv] table:t1 keep order:false, stats:pseudo +└─Selection_12 4433.77 root eq(substring(cast(test.t1.c3, var_string(20)), 1, 1), "1") + └─IndexMerge_11 5542.21 root + ├─IndexRangeScan_8(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo + ├─IndexRangeScan_9(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo + └─TableRowIDScan_10(Probe) 5542.21 cop[tikv] table:t1 keep order:false, stats:pseudo select /*+ use_index_merge(t1) */ * from t1 where (c1 < 10 or c2 < 10) and substring(c3, 1, 1) = '1' order by 1; c1 c2 c3 c4 c5 1 1 1 1 1 diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index 166d3adc298b3..2df5aa73eee34 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -28,6 +28,7 @@ import ( "github.com/pingcap/tidb/planner/property" "github.com/pingcap/tidb/planner/util" "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/types" tidbutil "github.com/pingcap/tidb/util" @@ -973,7 +974,7 @@ func (ds *DataSource) convertToIndexMergeScan(prop *property.PhysicalProperty, c if prop.ExpectedCnt < ds.stats.RowCount { totalRowCount *= prop.ExpectedCnt / ds.stats.RowCount } - ts, partialCost, err := ds.buildIndexMergeTableScan(prop, path.TableFilters, totalRowCount) + ts, partialCost, remainingFilters, err := ds.buildIndexMergeTableScan(prop, path.TableFilters, totalRowCount) if err != nil { return nil, err } @@ -981,6 +982,9 @@ func (ds *DataSource) convertToIndexMergeScan(prop *property.PhysicalProperty, c cop.tablePlan = ts cop.idxMergePartPlans = scans cop.cst = totalCost + if remainingFilters != nil { + cop.rootTaskConds = remainingFilters + } task = cop.convertToRootTask(ds.ctx) ds.addSelection4PlanCache(task.(*rootTask), ds.tableStats.ScaleByExpectCnt(totalRowCount), prop) return task, nil @@ -1092,8 +1096,10 @@ func setIndexMergeTableScanHandleCols(ds *DataSource, ts *PhysicalTableScan) (er return } +// buildIndexMergeTableScan() returns Selection that will be pushed to TiKV. +// Filters that cannot be pushed to TiKV are also returned, and an extra Selection above IndexMergeReader will be constructed later. func (ds *DataSource) buildIndexMergeTableScan(prop *property.PhysicalProperty, tableFilters []expression.Expression, - totalRowCount float64) (PhysicalPlan, float64, error) { + totalRowCount float64) (PhysicalPlan, float64, []expression.Expression, error) { var partialCost float64 sessVars := ds.ctx.GetSessionVars() ts := PhysicalTableScan{ @@ -1108,7 +1114,7 @@ func (ds *DataSource) buildIndexMergeTableScan(prop *property.PhysicalProperty, ts.SetSchema(ds.schema.Clone()) err := setIndexMergeTableScanHandleCols(ds, ts) if err != nil { - return nil, 0, err + return nil, 0, nil, err } if ts.Table.PKIsHandle { if pkColInfo := ts.Table.GetPkColInfo(); pkColInfo != nil { @@ -1124,17 +1130,44 @@ func (ds *DataSource) buildIndexMergeTableScan(prop *property.PhysicalProperty, ts.stats.StatsVersion = statistics.PseudoVersion } if len(tableFilters) > 0 { - partialCost += totalRowCount * sessVars.CopCPUFactor - selectivity, _, err := ds.tableStats.HistColl.Selectivity(ds.ctx, tableFilters, nil) - if err != nil { - logutil.BgLogger().Debug("calculate selectivity failed, use selection factor", zap.Error(err)) - selectivity = SelectionFactor + pushedFilters, remainingFilters := extractFiltersForIndexMerge(sessVars.StmtCtx, ds.ctx.GetClient(), tableFilters) + pushedFilters1, remainingFilters1 := SplitSelCondsWithVirtualColumn(pushedFilters) + pushedFilters = pushedFilters1 + remainingFilters = append(remainingFilters, remainingFilters1...) + if len(pushedFilters) != 0 { + partialCost += totalRowCount * sessVars.CopCPUFactor + selectivity, _, err := ds.tableStats.HistColl.Selectivity(ds.ctx, pushedFilters, nil) + if err != nil { + logutil.BgLogger().Debug("calculate selectivity failed, use selection factor", zap.Error(err)) + selectivity = SelectionFactor + } + sel := PhysicalSelection{Conditions: pushedFilters}.Init(ts.ctx, ts.stats.ScaleByExpectCnt(selectivity*totalRowCount), ts.blockOffset) + sel.SetChildren(ts) + return sel, partialCost, remainingFilters, nil + } + return ts, partialCost, remainingFilters, nil + } + return ts, partialCost, nil, nil +} + +// extractFiltersForIndexMerge returns: +// `pushed`: exprs that can be pushed to TiKV. +// `remaining`: exprs that can NOT be pushed to TiKV but can be pushed to other storage engines. +// Why do we need this func? +// IndexMerge only works on TiKV, so we need to find all exprs that cannot be pushed to TiKV, and add a new Selection above IndexMergeReader. +// But the new Selection should exclude the exprs that can NOT be pushed to ALL the storage engines. +// Because these exprs have already been put in another Selection(check rule_predicate_push_down). +func extractFiltersForIndexMerge(sc *stmtctx.StatementContext, client kv.Client, filters []expression.Expression) (pushed []expression.Expression, remaining []expression.Expression) { + for _, expr := range filters { + if expression.CanExprsPushDown(sc, []expression.Expression{expr}, client, kv.TiKV) { + pushed = append(pushed, expr) + continue + } + if expression.CanExprsPushDown(sc, []expression.Expression{expr}, client, kv.UnSpecified) { + remaining = append(remaining, expr) } - sel := PhysicalSelection{Conditions: tableFilters}.Init(ts.ctx, ts.stats.ScaleByExpectCnt(selectivity*totalRowCount), ts.blockOffset) - sel.SetChildren(ts) - return sel, partialCost, nil } - return ts, partialCost, nil + return } func indexCoveringCol(col *expression.Column, indexCols []*expression.Column, idxColLens []int) bool { diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index 0dfc4532a89bc..38048d4d30009 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -4992,6 +4992,82 @@ func (s *testIntegrationSuite) TestIssue30094(c *C) { )) } +func (s *testIntegrationSuite) TestIssue30200(c *C) { + tk := testkit.NewTestKit(c, s.store) + + tk.MustExec("use test") + tk.MustExec("drop table if exists t1;") + tk.MustExec("create table t1(c1 varchar(100), c2 varchar(100), key(c1), key(c2), c3 varchar(100));") + tk.MustExec("insert into t1 values('ab', '10', '10');") + + // lpad has not been pushed to TiKV or TiFlash. + tk.MustQuery("explain format=brief select /*+ use_index_merge(t1) */ * from t1 where c1 = 'ab' or c2 = '10' and char_length(lpad(c1, 10, 'a')) = 10;").Check(testkit.Rows( + "Selection 15.99 root or(eq(test.t1.c1, \"ab\"), and(eq(test.t1.c2, \"10\"), eq(char_length(lpad(test.t1.c1, 10, \"a\")), 10)))", + "└─IndexMerge 19.99 root ", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t1, index:c1(c1) range:[\"ab\",\"ab\"], keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t1, index:c2(c2) range:[\"10\",\"10\"], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 19.99 cop[tikv] table:t1 keep order:false, stats:pseudo")) + tk.MustQuery("select /*+ use_index_merge(t1) */ 1 from t1 where c1 = 'de' or c2 = '10' and char_length(lpad(c1, 10, 'a')) = 10;").Check(testkit.Rows("1")) + + // `left` has not been pushed to TiKV, but it has been pushed to TiFlash. + tk.MustQuery("explain format=brief select /*+ use_index_merge(t1) */ * from t1 where c1 = 'ab' or c2 = '10' and char_length(left(c1, 10)) = 10;").Check(testkit.Rows( + "Selection 0.04 root or(eq(test.t1.c1, \"ab\"), and(eq(test.t1.c2, \"10\"), eq(char_length(left(test.t1.c1, 10)), 10)))", + "└─IndexMerge 19.99 root ", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t1, index:c1(c1) range:[\"ab\",\"ab\"], keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t1, index:c2(c2) range:[\"10\",\"10\"], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 19.99 cop[tikv] table:t1 keep order:false, stats:pseudo")) + tk.MustQuery("select /*+ use_index_merge(t1) */ 1 from t1 where c1 = 'ab' or c2 = '10' and char_length(left(c1, 10)) = 10;").Check(testkit.Rows("1")) + + // If no hint, we cannot use index merge if filter cannot be pushed to any storage. + oriIndexMergeSwitcher := tk.MustQuery("select @@tidb_enable_index_merge;").Rows()[0][0].(string) + tk.MustExec("set tidb_enable_index_merge = on;") + defer func() { + tk.MustExec(fmt.Sprintf("set tidb_enable_index_merge = %s;", oriIndexMergeSwitcher)) + }() + tk.MustQuery("explain format=brief select * from t1 where c1 = 'ab' or c2 = '10' and char_length(lpad(c1, 10, 'a')) = 10;").Check(testkit.Rows( + "Selection 8000.00 root or(eq(test.t1.c1, \"ab\"), and(eq(test.t1.c2, \"10\"), eq(char_length(lpad(test.t1.c1, 10, \"a\")), 10)))", + "└─TableReader 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo")) + + tk.MustExec("use test") + tk.MustExec("drop table if exists t1;") + tk.MustExec("create table t1(c1 varchar(100), c2 varchar(100), c3 varchar(100), c4 varchar(100), key idx_0(c1), key idx_1(c2, c3));") + tk.MustExec("insert into t1 values('ab', '10', '10', '10');") + // c3 is part of idx_1, so it will be put in partial_path's IndexFilters instead of TableFilters. + // But it still cannot be pushed to TiKV. + tk.MustQuery("explain select /*+ use_index_merge(t1) */ 1 from t1 where c1 = 'de' or c2 = '10' and char_length(lpad(c3, 10, 'a')) = 10;").Check(testkit.Rows( + "Projection_4 15.99 root 1->Column#6", + "└─Selection_5 15.99 root or(eq(test.t1.c1, \"de\"), and(eq(test.t1.c2, \"10\"), eq(char_length(lpad(test.t1.c3, 10, \"a\")), 10)))", + " └─IndexMerge_9 19.99 root ", + " ├─IndexRangeScan_6(Build) 10.00 cop[tikv] table:t1, index:idx_0(c1) range:[\"de\",\"de\"], keep order:false, stats:pseudo", + " ├─IndexRangeScan_7(Build) 10.00 cop[tikv] table:t1, index:idx_1(c2, c3) range:[\"10\",\"10\"], keep order:false, stats:pseudo", + " └─TableRowIDScan_8(Probe) 19.99 cop[tikv] table:t1 keep order:false, stats:pseudo")) + tk.MustQuery("select /*+ use_index_merge(t1) */ 1 from t1 where c1 = 'de' or c2 = '10' and char_length(lpad(c3, 10, 'a')) = 10;").Check(testkit.Rows("1")) + + tk.MustExec("drop table if exists t1;") + tk.MustExec("create table t1 (c1 int , pk int, primary key( pk ) , unique key( c1));") + tk.MustExec("insert into t1 values(-3896405, -1), (-2, 1), (-1, -2);") + // to_base64(left(pk, 5)) is in partial_path's TableFilters. But it cannot be pushed to TiKV. So it should be executed in TiDB. + tk.MustQuery("explain select /*+ use_index_merge( t1 ) */ * from t1 where t1.c1 in (-3896405) or t1.pk in (1, 53330) and to_base64(left(pk, 5));").Check(testkit.Rows( + "Selection_5 2.40 root or(eq(test.t1.c1, -3896405), and(in(test.t1.pk, 1, 53330), istrue_with_null(cast(to_base64(left(cast(test.t1.pk, var_string(20)), 5)), double BINARY))))", + "└─IndexMerge_9 3.00 root ", + " ├─IndexRangeScan_6(Build) 1.00 cop[tikv] table:t1, index:c1(c1) range:[-3896405,-3896405], keep order:false, stats:pseudo", + " ├─TableRangeScan_7(Build) 2.00 cop[tikv] table:t1 range:[1,1], [53330,53330], keep order:false, stats:pseudo", + " └─TableRowIDScan_8(Probe) 3.00 cop[tikv] table:t1 keep order:false, stats:pseudo")) + tk.MustQuery("select /*+ use_index_merge( t1 ) */ * from t1 where t1.c1 in (-3896405) or t1.pk in (1, 53330) and to_base64(left(pk, 5));").Check(testkit.Rows("-3896405 -1")) + + tk.MustExec("drop table if exists t1;") + tk.MustExec("create table t1(c1 int, c2 int, c3 int as (c1 + c2), key(c1), key(c2), key(c3));") + tk.MustExec("insert into t1(c1, c2) values(1, 1);") + tk.MustQuery("explain format=brief select /*+ use_index_merge(t1) */ * from t1 where c1 < -10 or c2 < 10 and reverse(c3) = '2';").Check(testkit.Rows( + "Selection 2825.66 root or(lt(test.t1.c1, -10), and(lt(test.t1.c2, 10), eq(reverse(cast(test.t1.c3, var_string(20))), \"2\")))", + "└─IndexMerge 5542.21 root ", + " ├─IndexRangeScan(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,-10), keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 5542.21 cop[tikv] table:t1 keep order:false, stats:pseudo")) + tk.MustQuery("select /*+ use_index_merge(t1) */ * from t1 where c1 < -10 or c2 < 10 and reverse(c3) = '2';").Check(testkit.Rows("1 1 2")) +} + func (s *testIntegrationSuite) TestIssue29705(c *C) { tk := testkit.NewTestKit(c, s.store) origin := tk.MustQuery("SELECT @@session.tidb_partition_prune_mode") @@ -5034,3 +5110,26 @@ func (s *testIntegrationSuite) TestIssue30804(c *C) { c.Assert(core.ErrWindowNoSuchWindow.Equal(err), IsTrue) tk.MustExec("select avg(0) over w1 from t1 where b > (select sum(t2.a) over w2 from t2 window w2 as (partition by t2.b)) window w1 as (partition by t1.b)") } + +func (s *testIntegrationSuite) TestIndexMergeWarning(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + + tk.MustExec("drop table if exists t1") + tk.MustExec("create table t1(c1 int, c2 int)") + tk.MustExec("select /*+ use_index_merge(t1) */ * from t1 where c1 < 1 or c2 < 1") + warningMsg := "Warning 1105 IndexMerge is inapplicable or disabled. No available filter or available index." + tk.MustQuery("show warnings").Check(testkit.Rows(warningMsg)) + + tk.MustExec("drop table if exists t1") + tk.MustExec("create table t1(c1 int, c2 int, key(c1), key(c2))") + tk.MustExec("select /*+ use_index_merge(t1), no_index_merge() */ * from t1 where c1 < 1 or c2 < 1") + warningMsg = "Warning 1105 IndexMerge is inapplicable or disabled. Got no_index_merge hint or tidb_enable_index_merge is off." + tk.MustQuery("show warnings").Check(testkit.Rows(warningMsg)) + + tk.MustExec("drop table if exists t1") + tk.MustExec("create temporary table t1(c1 int, c2 int, key(c1), key(c2))") + tk.MustExec("select /*+ use_index_merge(t1) */ * from t1 where c1 < 1 or c2 < 1") + warningMsg = "Warning 1105 IndexMerge is inapplicable or disabled. Cannot use IndexMerge on temporary table." + tk.MustQuery("show warnings").Check(testkit.Rows(warningMsg)) +} diff --git a/planner/core/logical_plans.go b/planner/core/logical_plans.go index 5fe0426b5c15b..1b0f6c4543985 100644 --- a/planner/core/logical_plans.go +++ b/planner/core/logical_plans.go @@ -549,7 +549,7 @@ type DataSource struct { // pushedDownConds are the conditions that will be pushed down to coprocessor. pushedDownConds []expression.Expression // allConds contains all the filters on this table. For now it's maintained - // in predicate push down and used only in partition pruning. + // in predicate push down and used in partition pruning/index merge. allConds []expression.Expression statisticTable *statistics.Table diff --git a/planner/core/stats.go b/planner/core/stats.go index d3f23427b2f40..4b7ac3e33d00a 100644 --- a/planner/core/stats.go +++ b/planner/core/stats.go @@ -23,6 +23,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/expression" + "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" @@ -409,9 +410,20 @@ func (ds *DataSource) DeriveStats(childStats []*property.StatsInfo, selfSchema * } // Consider the IndexMergePath. Now, we just generate `IndexMergePath` in DNF case. - isPossibleIdxMerge := len(ds.pushedDownConds) > 0 && len(ds.possibleAccessPaths) > 1 - sessionAndStmtPermission := (ds.ctx.GetSessionVars().GetEnableIndexMerge() || len(ds.indexMergeHints) > 0) && !ds.ctx.GetSessionVars().StmtCtx.NoIndexMergeHint - // If there is an index path, we current do not consider `IndexMergePath`. + // Use allConds instread of pushedDownConds, + // because we want to use IndexMerge even if some expr cannot be pushed to TiKV. + // We will create new Selection for exprs that cannot be pushed in convertToIndexMergeScan. + var indexMergeConds []expression.Expression + for _, expr := range ds.allConds { + indexMergeConds = append(indexMergeConds, expression.PushDownNot(ds.ctx, expr)) + } + + stmtCtx := ds.ctx.GetSessionVars().StmtCtx + isPossibleIdxMerge := len(indexMergeConds) > 0 && len(ds.possibleAccessPaths) > 1 + sessionAndStmtPermission := (ds.ctx.GetSessionVars().GetEnableIndexMerge() || len(ds.indexMergeHints) > 0) && !stmtCtx.NoIndexMergeHint + // We current do not consider `IndexMergePath`: + // 1. If there is an index path. + // 2. TODO: If there exists exprs that cannot be pushed down. This is to avoid wrongly estRow of Selection added by rule_predicate_push_down. needConsiderIndexMerge := true if len(ds.indexMergeHints) == 0 { for i := 1; i < len(ds.possibleAccessPaths); i++ { @@ -420,24 +432,45 @@ func (ds *DataSource) DeriveStats(childStats []*property.StatsInfo, selfSchema * break } } + if needConsiderIndexMerge { + // PushDownExprs() will append extra warnings, which is annoying. So we reset warnings here. + warnings := stmtCtx.GetWarnings() + _, remaining := expression.PushDownExprs(stmtCtx, indexMergeConds, ds.ctx.GetClient(), kv.UnSpecified) + stmtCtx.SetWarnings(warnings) + if len(remaining) != 0 { + needConsiderIndexMerge = false + } + } } - readFromTableCache := ds.ctx.GetSessionVars().StmtCtx.ReadFromTableCache + readFromTableCache := stmtCtx.ReadFromTableCache if isPossibleIdxMerge && sessionAndStmtPermission && needConsiderIndexMerge && ds.tableInfo.TempTableType != model.TempTableLocal && !readFromTableCache { - err := ds.generateAndPruneIndexMergePath(ds.indexMergeHints != nil) + err := ds.generateAndPruneIndexMergePath(indexMergeConds, ds.indexMergeHints != nil) if err != nil { return nil, err } } else if len(ds.indexMergeHints) > 0 { ds.indexMergeHints = nil - ds.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("IndexMerge is inapplicable or disabled")) + var msg string + if !isPossibleIdxMerge { + msg = "No available filter or available index." + } else if !sessionAndStmtPermission { + msg = "Got no_index_merge hint or tidb_enable_index_merge is off." + } else if ds.tableInfo.TempTableType == model.TempTableLocal { + msg = "Cannot use IndexMerge on temporary table." + } else if readFromTableCache { + msg = "Cannot use IndexMerge on TableCache." + } + msg = fmt.Sprintf("IndexMerge is inapplicable or disabled. %s", msg) + stmtCtx.AppendWarning(errors.Errorf(msg)) + logutil.BgLogger().Debug(msg) } return ds.stats, nil } -func (ds *DataSource) generateAndPruneIndexMergePath(needPrune bool) error { +func (ds *DataSource) generateAndPruneIndexMergePath(indexMergeConds []expression.Expression, needPrune bool) error { regularPathCount := len(ds.possibleAccessPaths) - err := ds.generateIndexMergeOrPaths() + err := ds.generateIndexMergeOrPaths(indexMergeConds) if err != nil { return err } @@ -448,12 +481,22 @@ func (ds *DataSource) generateAndPruneIndexMergePath(needPrune bool) error { // With hints and without generated IndexMerge paths if regularPathCount == len(ds.possibleAccessPaths) { ds.indexMergeHints = nil - ds.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("IndexMerge is inapplicable or disabled")) + ds.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("IndexMerge is inapplicable.")) return nil } // Do not need to consider the regular paths in find_best_task(). + // So we can use index merge's row count as DataSource's row count. if needPrune { ds.possibleAccessPaths = ds.possibleAccessPaths[regularPathCount:] + minRowCount := ds.possibleAccessPaths[0].CountAfterAccess + for _, path := range ds.possibleAccessPaths { + if minRowCount < path.CountAfterAccess { + minRowCount = path.CountAfterAccess + } + } + if ds.stats.RowCount > minRowCount { + ds.stats = ds.tableStats.ScaleByExpectCnt(minRowCount) + } } return nil } @@ -510,9 +553,9 @@ func (is *LogicalIndexScan) DeriveStats(childStats []*property.StatsInfo, selfSc } // getIndexMergeOrPath generates all possible IndexMergeOrPaths. -func (ds *DataSource) generateIndexMergeOrPaths() error { +func (ds *DataSource) generateIndexMergeOrPaths(filters []expression.Expression) error { usedIndexCount := len(ds.possibleAccessPaths) - for i, cond := range ds.pushedDownConds { + for i, cond := range filters { sf, ok := cond.(*expression.ScalarFunction) if !ok || sf.FuncName.L != ast.LogicOr { continue @@ -548,7 +591,7 @@ func (ds *DataSource) generateIndexMergeOrPaths() error { continue } if len(partialPaths) > 1 { - possiblePath := ds.buildIndexMergeOrPath(partialPaths, i) + possiblePath := ds.buildIndexMergeOrPath(filters, partialPaths, i) if possiblePath == nil { return nil } @@ -686,16 +729,29 @@ func (ds *DataSource) buildIndexMergePartialPath(indexAccessPaths []*util.Access } // buildIndexMergeOrPath generates one possible IndexMergePath. -func (ds *DataSource) buildIndexMergeOrPath(partialPaths []*util.AccessPath, current int) *util.AccessPath { +func (ds *DataSource) buildIndexMergeOrPath(filters []expression.Expression, partialPaths []*util.AccessPath, current int) *util.AccessPath { indexMergePath := &util.AccessPath{PartialIndexPaths: partialPaths} - indexMergePath.TableFilters = append(indexMergePath.TableFilters, ds.pushedDownConds[:current]...) - indexMergePath.TableFilters = append(indexMergePath.TableFilters, ds.pushedDownConds[current+1:]...) + indexMergePath.TableFilters = append(indexMergePath.TableFilters, filters[:current]...) + indexMergePath.TableFilters = append(indexMergePath.TableFilters, filters[current+1:]...) + var addCurrentFilter bool for _, path := range partialPaths { // If any partial path contains table filters, we need to keep the whole DNF filter in the Selection. if len(path.TableFilters) > 0 { - indexMergePath.TableFilters = append(indexMergePath.TableFilters, ds.pushedDownConds[current]) - break + addCurrentFilter = true } + // If any partial path's index filter cannot be pushed to TiKV, we should keep the whole DNF filter. + if len(path.IndexFilters) != 0 && !expression.CanExprsPushDown(ds.ctx.GetSessionVars().StmtCtx, path.IndexFilters, ds.ctx.GetClient(), kv.TiKV) { + addCurrentFilter = true + // Clear IndexFilter, the whole filter will be put in indexMergePath.TableFilters. + path.IndexFilters = nil + } + if len(path.TableFilters) != 0 && !expression.CanExprsPushDown(ds.ctx.GetSessionVars().StmtCtx, path.TableFilters, ds.ctx.GetClient(), kv.TiKV) { + addCurrentFilter = true + path.TableFilters = nil + } + } + if addCurrentFilter { + indexMergePath.TableFilters = append(indexMergePath.TableFilters, filters[current]) } return indexMergePath } diff --git a/planner/core/task.go b/planner/core/task.go index 187140c613aa5..4b9c5692ca29d 100644 --- a/planner/core/task.go +++ b/planner/core/task.go @@ -1005,6 +1005,7 @@ func (t *copTask) convertToRootTaskImpl(ctx sessionctx.Context) *rootTask { setTableScanToTableRowIDScan(p.tablePlan) newTask.p = p p.cost = newTask.cost() + t.handleRootTaskConds(ctx, newTask) if t.needExtraProj { schema := t.originSchema proj := PhysicalProjection{Exprs: expression.Column2Exprs(schema.Columns)}.Init(ctx, p.stats, t.idxMergePartPlans[0].SelectBlockOffset(), nil) @@ -1066,6 +1067,11 @@ func (t *copTask) convertToRootTaskImpl(ctx sessionctx.Context) *rootTask { } } + t.handleRootTaskConds(ctx, newTask) + return newTask +} + +func (t *copTask) handleRootTaskConds(ctx sessionctx.Context, newTask *rootTask) { if len(t.rootTaskConds) > 0 { selectivity, _, err := t.tblColHists.Selectivity(ctx, t.rootTaskConds, nil) if err != nil { @@ -1077,8 +1083,6 @@ func (t *copTask) convertToRootTaskImpl(ctx sessionctx.Context) *rootTask { newTask.p = sel sel.cost = newTask.cost() } - - return newTask } // setTableScanToTableRowIDScan is to update the isChildOfIndexLookUp attribute of PhysicalTableScan child diff --git a/planner/core/testdata/integration_suite_out.json b/planner/core/testdata/integration_suite_out.json index 6c946cdac4d2d..c3bbda1ed3f2c 100644 --- a/planner/core/testdata/integration_suite_out.json +++ b/planner/core/testdata/integration_suite_out.json @@ -1332,7 +1332,7 @@ "└─IndexRangeScan 20.00 cop[tikv] table:tt, index:a(a) range:[10,10], [20,20], keep order:false, stats:pseudo" ], "Warnings": [ - "Warning 1105 IndexMerge is inapplicable or disabled" + "Warning 1105 IndexMerge is inapplicable." ] }, { @@ -1342,7 +1342,7 @@ "└─IndexRangeScan 6666.67 cop[tikv] table:tt, index:a(a) range:[-inf,10), [15,15], (20,+inf], keep order:false, stats:pseudo" ], "Warnings": [ - "Warning 1105 IndexMerge is inapplicable or disabled" + "Warning 1105 IndexMerge is inapplicable." ] } ] diff --git a/session/session_test.go b/session/session_test.go index e70660f172111..67c9fb64aef12 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -5420,7 +5420,7 @@ func (s *testSessionSuite) TestLocalTemporaryTableScan(c *C) { "12 112 1012", "3 113 1003", "14 114 1014", "16 116 1016", "7 117 1007", "18 118 1018", )) - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 IndexMerge is inapplicable or disabled")) + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 IndexMerge is inapplicable or disabled. Cannot use IndexMerge on temporary table.")) } doModify := func() { @@ -5459,7 +5459,7 @@ func (s *testSessionSuite) TestLocalTemporaryTableScan(c *C) { "3 113 1003", "14 114 1014", "7 117 9999", "18 118 1018", "12 132 1012", )) - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 IndexMerge is inapplicable or disabled")) + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 IndexMerge is inapplicable or disabled. Cannot use IndexMerge on temporary table.")) } assertSelectAsUnModified() diff --git a/table/tables/cache_test.go b/table/tables/cache_test.go index a4dc5b4d43d68..788ecd48fd1d7 100644 --- a/table/tables/cache_test.go +++ b/table/tables/cache_test.go @@ -142,7 +142,7 @@ func TestCacheTableBasicScan(t *testing.T) { "12 112 1012", "3 113 1003", "14 114 1014", "16 116 1016", "7 117 1007", "18 118 1018", )) - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 IndexMerge is inapplicable or disabled")) + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 IndexMerge is inapplicable or disabled. Cannot use IndexMerge on TableCache.")) } assertSelect() From d2ed2ae2b2397769b462d343248fdd56078bf7f7 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Thu, 23 Dec 2021 14:45:47 +0800 Subject: [PATCH 29/44] executor: display 'show create table' and INFOSCHEMA for cached table correctly (#30951) close pingcap/tidb#30950 --- executor/infoschema_reader.go | 2 ++ executor/show.go | 6 ++++++ executor/show_test.go | 21 +++++++++++++++++++++ 3 files changed, 29 insertions(+) diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index 06d35d24fce1d..b5eee15e32600 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -543,6 +543,8 @@ func (e *memtableRetriever) setDataFromTables(ctx context.Context, sctx sessionc if !table.IsView() { if table.GetPartitionInfo() != nil { createOptions = "partitioned" + } else if table.TableCacheStatusType == model.TableCacheStatusEnable { + createOptions = "cached=on" } var autoIncID interface{} hasAutoIncID, _ := infoschema.HasAutoIncrementColumn(table) diff --git a/executor/show.go b/executor/show.go index ea4bdf4f1aee8..066d819cc9b80 100644 --- a/executor/show.go +++ b/executor/show.go @@ -1077,6 +1077,12 @@ func ConstructResultOfShowCreateTable(ctx sessionctx.Context, tableInfo *model.T fmt.Fprintf(buf, " /*T![placement] PLACEMENT POLICY=%s */", stringutil.Escape(tableInfo.PlacementPolicyRef.Name.String(), sqlMode)) } + if tableInfo.TableCacheStatusType == model.TableCacheStatusEnable { + // This is not meant to be understand by other components, so it's not written as /*T![cached] */ + // For all external components, cached table is just a normal table. + fmt.Fprintf(buf, " /* CACHED ON */") + } + // add direct placement info here appendDirectPlacementInfo(tableInfo.DirectPlacementOpts, buf) // add partition info here. diff --git a/executor/show_test.go b/executor/show_test.go index 34db4d54ae962..229d4399dae4d 100644 --- a/executor/show_test.go +++ b/executor/show_test.go @@ -1674,3 +1674,24 @@ func (s *testSuite5) TestShowTemporaryTable(c *C) { ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin AUTO_INCREMENT=2" tk.MustQuery("show create table t7").Check(testkit.Rows("t7 " + expect)) } + +func (s *testSuite5) TestShowCachedTable(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("create table t1 (id int)") + tk.MustExec("alter table t1 cache") + tk.MustQuery("show create table t1").Check( + testkit.Rows("t1 CREATE TABLE `t1` (\n" + + " `id` int(11) DEFAULT NULL\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /* CACHED ON */")) + tk.MustQuery("select create_options from information_schema.tables where table_schema = 'test' and table_name = 't1'").Check( + testkit.Rows("cached=on")) + + tk.MustExec("alter table t1 nocache") + tk.MustQuery("show create table t1").Check( + testkit.Rows("t1 CREATE TABLE `t1` (\n" + + " `id` int(11) DEFAULT NULL\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin")) + tk.MustQuery("select create_options from information_schema.tables where table_schema = 'test' and table_name = 't1'").Check( + testkit.Rows("")) +} From 693946bb79f2c3d602f9f63759bdd789a62480b7 Mon Sep 17 00:00:00 2001 From: fengou1 <85682690+fengou1@users.noreply.github.com> Date: Thu, 23 Dec 2021 15:05:48 +0800 Subject: [PATCH 30/44] br: extend the timeout for scan region since 3 seconds is not enough (#30889) close pingcap/tidb#30720 --- br/pkg/restore/split.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/br/pkg/restore/split.go b/br/pkg/restore/split.go index ada8662522c21..b302dfc2487d3 100644 --- a/br/pkg/restore/split.go +++ b/br/pkg/restore/split.go @@ -455,14 +455,14 @@ type scanRegionBackoffer struct { func newScanRegionBackoffer() utils.Backoffer { return &scanRegionBackoffer{ - attempt: 3, + attempt: 30, } } // NextBackoff returns a duration to wait before retrying again func (b *scanRegionBackoffer) NextBackoff(err error) time.Duration { if berrors.ErrPDBatchScanRegion.Equal(err) { - // 500ms * 3 could be enough for splitting remain regions in the hole. + // 500ms * 30 could be enough for splitting remain regions in the hole. b.attempt-- return 500 * time.Millisecond } From 0d899cdd9eee90e4942d87f2b8fcb4854be3c133 Mon Sep 17 00:00:00 2001 From: Chengpeng Yan <41809508+Reminiscent@users.noreply.github.com> Date: Thu, 23 Dec 2021 16:37:47 +0800 Subject: [PATCH 31/44] planner: remove bindSQL from planCacheKey to planCacheValue (#30916) --- executor/prepared.go | 3 +-- planner/core/cache.go | 9 ++++----- planner/core/cache_test.go | 2 +- planner/core/common_plans.go | 16 ++++++++++------ server/driver_tidb.go | 5 +---- session/session.go | 3 +-- 6 files changed, 18 insertions(+), 20 deletions(-) diff --git a/executor/prepared.go b/executor/prepared.go index 4f63bce491ab9..e18495d45c601 100644 --- a/executor/prepared.go +++ b/executor/prepared.go @@ -316,9 +316,8 @@ func (e *DeallocateExec) Next(ctx context.Context, req *chunk.Chunk) error { prepared := preparedObj.PreparedAst delete(vars.PreparedStmtNameToID, e.Name) if plannercore.PreparedPlanCacheEnabled() { - bindSQL := planner.GetBindSQL4PlanCache(e.ctx, prepared.Stmt) e.ctx.PreparedPlanCache().Delete(plannercore.NewPlanCacheKey( - vars, id, prepared.SchemaVersion, bindSQL, + vars, id, prepared.SchemaVersion, )) } vars.RemovePreparedStmt(id) diff --git a/planner/core/cache.go b/planner/core/cache.go index 2ef974340c063..7e741e203d3dc 100644 --- a/planner/core/cache.go +++ b/planner/core/cache.go @@ -79,7 +79,6 @@ type planCacheKey struct { timezoneOffset int isolationReadEngines map[kv.StoreType]struct{} selectLimit uint64 - bindSQL string hash []byte } @@ -110,7 +109,6 @@ func (key *planCacheKey) Hash() []byte { key.hash = append(key.hash, kv.TiFlash.Name()...) } key.hash = codec.EncodeInt(key.hash, int64(key.selectLimit)) - key.hash = append(key.hash, hack.Slice(key.bindSQL)...) } return key.hash } @@ -132,7 +130,7 @@ func SetPstmtIDSchemaVersion(key kvcache.Key, pstmtID uint32, schemaVersion int6 } // NewPlanCacheKey creates a new planCacheKey object. -func NewPlanCacheKey(sessionVars *variable.SessionVars, pstmtID uint32, schemaVersion int64, bindSQL string) kvcache.Key { +func NewPlanCacheKey(sessionVars *variable.SessionVars, pstmtID uint32, schemaVersion int64) kvcache.Key { timezoneOffset := 0 if sessionVars.TimeZone != nil { _, timezoneOffset = time.Now().In(sessionVars.TimeZone).Zone() @@ -146,7 +144,6 @@ func NewPlanCacheKey(sessionVars *variable.SessionVars, pstmtID uint32, schemaVe timezoneOffset: timezoneOffset, isolationReadEngines: make(map[kv.StoreType]struct{}), selectLimit: sessionVars.SelectLimit, - bindSQL: bindSQL, } for k, v := range sessionVars.IsolationReadEngines { key.isolationReadEngines[k] = v @@ -185,10 +182,11 @@ type PlanCacheValue struct { OutPutNames []*types.FieldName TblInfo2UnionScan map[*model.TableInfo]bool UserVarTypes FieldSlice + BindSQL string } // NewPlanCacheValue creates a SQLCacheValue. -func NewPlanCacheValue(plan Plan, names []*types.FieldName, srcMap map[*model.TableInfo]bool, userVarTps []*types.FieldType) *PlanCacheValue { +func NewPlanCacheValue(plan Plan, names []*types.FieldName, srcMap map[*model.TableInfo]bool, userVarTps []*types.FieldType, bindSQL string) *PlanCacheValue { dstMap := make(map[*model.TableInfo]bool) for k, v := range srcMap { dstMap[k] = v @@ -202,6 +200,7 @@ func NewPlanCacheValue(plan Plan, names []*types.FieldName, srcMap map[*model.Ta OutPutNames: names, TblInfo2UnionScan: dstMap, UserVarTypes: userVarTypes, + BindSQL: bindSQL, } } diff --git a/planner/core/cache_test.go b/planner/core/cache_test.go index c75a4b3963713..3527fa9d7ecb2 100644 --- a/planner/core/cache_test.go +++ b/planner/core/cache_test.go @@ -28,6 +28,6 @@ func TestCacheKey(t *testing.T) { ctx.GetSessionVars().SQLMode = mysql.ModeNone ctx.GetSessionVars().TimeZone = time.UTC ctx.GetSessionVars().ConnectionID = 0 - key := NewPlanCacheKey(ctx.GetSessionVars(), 1, 1, "") + key := NewPlanCacheKey(ctx.GetSessionVars(), 1, 1) require.Equal(t, []byte{0x74, 0x65, 0x73, 0x74, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x1, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x1, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x74, 0x69, 0x64, 0x62, 0x74, 0x69, 0x6b, 0x76, 0x74, 0x69, 0x66, 0x6c, 0x61, 0x73, 0x68, 0x7f, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff}, key.Hash()) } diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index 54f30d7d998ea..9dc276fec650b 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -404,7 +404,7 @@ func (e *Execute) getPhysicalPlan(ctx context.Context, sctx sessionctx.Context, var bindSQL string if prepared.UseCache { bindSQL = GetBindSQL4PlanCache(sctx, prepared.Stmt) - cacheKey = NewPlanCacheKey(sctx.GetSessionVars(), e.ExecID, prepared.SchemaVersion, bindSQL) + cacheKey = NewPlanCacheKey(sctx.GetSessionVars(), e.ExecID, prepared.SchemaVersion) } tps := make([]*types.FieldType, len(e.UsingVars)) for i, param := range e.UsingVars { @@ -447,6 +447,13 @@ func (e *Execute) getPhysicalPlan(ctx context.Context, sctx sessionctx.Context, } cachedVals := cacheValue.([]*PlanCacheValue) for _, cachedVal := range cachedVals { + if cachedVal.BindSQL != bindSQL { + // When BindSQL does not match, it means that we have added a new binding, + // and the original cached plan will be invalid, + // so the original cached plan can be cleared directly + sctx.PreparedPlanCache().Delete(cacheKey) + break + } if !cachedVal.UserVarTypes.Equal(tps) { continue } @@ -510,13 +517,10 @@ REBUILD: // rebuild key to exclude kv.TiFlash when stmt is not read only if _, isolationReadContainTiFlash := sessVars.IsolationReadEngines[kv.TiFlash]; isolationReadContainTiFlash && !IsReadOnly(stmt, sessVars) { delete(sessVars.IsolationReadEngines, kv.TiFlash) - cacheKey = NewPlanCacheKey(sessVars, e.ExecID, prepared.SchemaVersion, sessVars.StmtCtx.BindSQL) + cacheKey = NewPlanCacheKey(sessVars, e.ExecID, prepared.SchemaVersion) sessVars.IsolationReadEngines[kv.TiFlash] = struct{}{} - } else { - // We need to reconstruct the plan cache key based on the bindSQL. - cacheKey = NewPlanCacheKey(sessVars, e.ExecID, prepared.SchemaVersion, sessVars.StmtCtx.BindSQL) } - cached := NewPlanCacheValue(p, names, stmtCtx.TblInfo2UnionScan, tps) + cached := NewPlanCacheValue(p, names, stmtCtx.TblInfo2UnionScan, tps, sessVars.StmtCtx.BindSQL) preparedStmt.NormalizedPlan, preparedStmt.PlanDigest = NormalizePlan(p) stmtCtx.SetPlanDigest(preparedStmt.NormalizedPlan, preparedStmt.PlanDigest) if cacheVals, exists := sctx.PreparedPlanCache().Get(cacheKey); exists { diff --git a/server/driver_tidb.go b/server/driver_tidb.go index dfe88d5364114..0840cceb86c0a 100644 --- a/server/driver_tidb.go +++ b/server/driver_tidb.go @@ -25,7 +25,6 @@ import ( "github.com/pingcap/tidb/parser/charset" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/parser/terror" - "github.com/pingcap/tidb/planner" "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx/stmtctx" @@ -165,10 +164,8 @@ func (ts *TiDBStatement) Close() error { if !ok { return errors.Errorf("invalid CachedPrepareStmt type") } - preparedAst := preparedObj.PreparedAst - bindSQL := planner.GetBindSQL4PlanCache(ts.ctx, preparedAst.Stmt) ts.ctx.PreparedPlanCache().Delete(core.NewPlanCacheKey( - ts.ctx.GetSessionVars(), ts.id, preparedObj.PreparedAst.SchemaVersion, bindSQL)) + ts.ctx.GetSessionVars(), ts.id, preparedObj.PreparedAst.SchemaVersion)) } ts.ctx.GetSessionVars().RemovePreparedStmt(ts.id) } diff --git a/session/session.go b/session/session.go index f685b44d0be43..399032bd6cbfd 100644 --- a/session/session.go +++ b/session/session.go @@ -313,8 +313,7 @@ func (s *session) cleanRetryInfo() { preparedObj, ok := preparedPointer.(*plannercore.CachedPrepareStmt) if ok { preparedAst = preparedObj.PreparedAst - bindSQL := planner.GetBindSQL4PlanCache(s, preparedAst.Stmt) - cacheKey = plannercore.NewPlanCacheKey(s.sessionVars, firstStmtID, preparedAst.SchemaVersion, bindSQL) + cacheKey = plannercore.NewPlanCacheKey(s.sessionVars, firstStmtID, preparedAst.SchemaVersion) } } } From df58e6c36c3f35599c5c3f8e1afc2032b5f3c299 Mon Sep 17 00:00:00 2001 From: Yilong Li Date: Thu, 23 Dec 2021 16:53:47 +0800 Subject: [PATCH 32/44] execution: refine precision of cast as decimal in agg func (#30805) --- cmd/explaintest/r/explain_complex.result | 2 +- .../r/explain_complex_stats.result | 2 +- cmd/explaintest/r/explain_easy.result | 128 +++++++++++++++++- cmd/explaintest/r/generated_columns.result | 8 +- cmd/explaintest/r/index_merge.result | 2 +- cmd/explaintest/r/select.result | 4 +- cmd/explaintest/r/tpch.result | 2 +- cmd/explaintest/r/window_function.result | 32 ++--- cmd/explaintest/t/explain_easy.test | 19 +++ executor/testdata/agg_suite_out.json | 2 +- expression/aggregation/base_func.go | 35 ++++- expression/builtin.go | 2 +- parser/mysql/type.go | 10 +- .../testdata/integration_suite_out.json | 4 +- planner/core/preprocess.go | 2 +- .../integration_serial_suite_out.json | 60 ++++---- .../ordered_result_mode_suite_out.json | 2 +- planner/core/testdata/plan_suite_out.json | 8 +- .../testdata/plan_suite_unexported_out.json | 38 +++--- planner/core/testdata/stats_suite_out.json | 2 +- 20 files changed, 267 insertions(+), 97 deletions(-) diff --git a/cmd/explaintest/r/explain_complex.result b/cmd/explaintest/r/explain_complex.result index b3c79948a142e..43f7c0f915f14 100644 --- a/cmd/explaintest/r/explain_complex.result +++ b/cmd/explaintest/r/explain_complex.result @@ -182,7 +182,7 @@ CREATE TABLE `tbl_009` (`a` int, `b` int); explain format = 'brief' select sum(a) from (select * from tbl_001 union all select * from tbl_002 union all select * from tbl_003 union all select * from tbl_004 union all select * from tbl_005 union all select * from tbl_006 union all select * from tbl_007 union all select * from tbl_008 union all select * from tbl_009) x group by b; id estRows task access object operator info HashAgg 72000.00 root group by:Column#32, funcs:sum(Column#31)->Column#30 -└─Projection 90000.00 root cast(Column#28, decimal(32,0) BINARY)->Column#31, Column#29 +└─Projection 90000.00 root cast(Column#28, decimal(10,0) BINARY)->Column#31, Column#29 └─Union 90000.00 root ├─TableReader 10000.00 root data:TableFullScan │ └─TableFullScan 10000.00 cop[tikv] table:tbl_001 keep order:false, stats:pseudo diff --git a/cmd/explaintest/r/explain_complex_stats.result b/cmd/explaintest/r/explain_complex_stats.result index 4365be0c45336..ed7021dbbfba2 100644 --- a/cmd/explaintest/r/explain_complex_stats.result +++ b/cmd/explaintest/r/explain_complex_stats.result @@ -205,7 +205,7 @@ load stats 's/explain_complex_stats_tbl_009.json'; explain format = 'brief' select sum(a) from (select * from tbl_001 union all select * from tbl_002 union all select * from tbl_003 union all select * from tbl_004 union all select * from tbl_005 union all select * from tbl_006 union all select * from tbl_007 union all select * from tbl_008 union all select * from tbl_009) x group by b; id estRows task access object operator info HashAgg 18000.00 root group by:Column#32, funcs:sum(Column#31)->Column#30 -└─Projection 18000.00 root cast(Column#28, decimal(32,0) BINARY)->Column#31, Column#29 +└─Projection 18000.00 root cast(Column#28, decimal(10,0) BINARY)->Column#31, Column#29 └─Union 18000.00 root ├─TableReader 2000.00 root data:TableFullScan │ └─TableFullScan 2000.00 cop[tikv] table:tbl_001 keep order:false diff --git a/cmd/explaintest/r/explain_easy.result b/cmd/explaintest/r/explain_easy.result index 2eec1954d46df..3eb32472170e5 100644 --- a/cmd/explaintest/r/explain_easy.result +++ b/cmd/explaintest/r/explain_easy.result @@ -91,7 +91,7 @@ Selection 0.33 root gt(test.t1.c2, 1) explain format = 'brief' select sum(t1.c1 in (select c1 from t2)) from t1; id estRows task access object operator info StreamAgg 1.00 root funcs:sum(Column#13)->Column#11 -└─Projection 10000.00 root cast(Column#10, decimal(65,0) BINARY)->Column#13 +└─Projection 10000.00 root cast(Column#10, decimal(3,0) BINARY)->Column#13 └─HashJoin 10000.00 root CARTESIAN left outer semi join, other cond:eq(test.t1.c1, test.t2.c1) ├─IndexReader(Build) 10000.00 root index:IndexFullScan │ └─IndexFullScan 10000.00 cop[tikv] table:t2, index:c1(c1) keep order:false, stats:pseudo @@ -225,7 +225,7 @@ set @@session.tidb_opt_insubq_to_join_and_agg=0; explain format = 'brief' select sum(t1.c1 in (select c1 from t2)) from t1; id estRows task access object operator info StreamAgg 1.00 root funcs:sum(Column#13)->Column#11 -└─Projection 10000.00 root cast(Column#10, decimal(65,0) BINARY)->Column#13 +└─Projection 10000.00 root cast(Column#10, decimal(3,0) BINARY)->Column#13 └─HashJoin 10000.00 root CARTESIAN left outer semi join, other cond:eq(test.t1.c1, test.t2.c1) ├─IndexReader(Build) 10000.00 root index:IndexFullScan │ └─IndexFullScan 10000.00 cop[tikv] table:t2, index:c1(c1) keep order:false, stats:pseudo @@ -241,7 +241,7 @@ HashJoin 10000.00 root CARTESIAN left outer semi join, other cond:eq(1, test.t2 explain format = 'brief' select sum(6 in (select c2 from t2)) from t1; id estRows task access object operator info StreamAgg 1.00 root funcs:sum(Column#13)->Column#11 -└─Projection 10000.00 root cast(Column#10, decimal(65,0) BINARY)->Column#13 +└─Projection 10000.00 root cast(Column#10, decimal(3,0) BINARY)->Column#13 └─HashJoin 10000.00 root CARTESIAN left outer semi join, other cond:eq(6, test.t2.c2) ├─TableReader(Build) 10000.00 root data:TableFullScan │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo @@ -809,7 +809,7 @@ Projection 1.00 root Column#7 └─HashAgg(Probe) 1.00 root funcs:sum(Column#12)->Column#7 └─HashJoin 1.00 root CARTESIAN left outer join ├─HashAgg(Build) 1.00 root group by:1, funcs:sum(Column#14)->Column#12 - │ └─Projection 1.00 root cast(Column#6, decimal(42,0) BINARY)->Column#14 + │ └─Projection 1.00 root cast(Column#6, decimal(20,0) BINARY)->Column#14 │ └─MaxOneRow 1.00 root │ └─Projection 1.00 root Column#5 │ └─TableDual 1.00 root rows:1 @@ -831,7 +831,7 @@ Projection 1.00 root Column#5 │ └─TableReader 10000.00 root data:TableFullScan │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo └─StreamAgg(Probe) 1.00 root funcs:sum(Column#7)->Column#5 - └─Projection 1.00 root cast(Column#4, decimal(42,0) BINARY)->Column#7 + └─Projection 1.00 root cast(Column#4, decimal(20,0) BINARY)->Column#7 └─TableDual 1.00 root rows:1 explain format = 'brief' select sum(a), (select sum(a)), count(a) from t group by b order by (select count(a)); id estRows task access object operator info @@ -846,3 +846,121 @@ Projection 8000.00 root Column#4, Column#4, Column#5 └─HashAgg 8000.00 cop[tikv] group by:test.t.b, funcs:sum(test.t.a)->Column#13, funcs:count(test.t.a)->Column#14 └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo drop table if exists t; +create table t(a tinyint, b smallint, c mediumint, d int, e bigint); +insert into mysql.opt_rule_blacklist VALUES("aggregation_push_down"); +admin reload opt_rule_blacklist; + +explain format = 'brief' select sum(t1.a) from t t1 join t t2 on t1.a=t2.a; +id estRows task access object operator info +StreamAgg 1.00 root funcs:sum(Column#14)->Column#13 +└─Projection 12487.50 root cast(test.t.a, decimal(3,0) BINARY)->Column#14 + └─HashJoin 12487.50 root inner join, equal:[eq(test.t.a, test.t.a)] + ├─TableReader(Build) 9990.00 root data:Selection + │ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a)) + │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo + └─TableReader(Probe) 9990.00 root data:Selection + └─Selection 9990.00 cop[tikv] not(isnull(test.t.a)) + └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo +explain format = 'brief' select sum(t1.b) from t t1 join t t2 on t1.b=t2.b; +id estRows task access object operator info +StreamAgg 1.00 root funcs:sum(Column#14)->Column#13 +└─Projection 12487.50 root cast(test.t.b, decimal(5,0) BINARY)->Column#14 + └─HashJoin 12487.50 root inner join, equal:[eq(test.t.b, test.t.b)] + ├─TableReader(Build) 9990.00 root data:Selection + │ └─Selection 9990.00 cop[tikv] not(isnull(test.t.b)) + │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo + └─TableReader(Probe) 9990.00 root data:Selection + └─Selection 9990.00 cop[tikv] not(isnull(test.t.b)) + └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo +explain format = 'brief' select sum(t1.c) from t t1 join t t2 on t1.c=t2.c; +id estRows task access object operator info +StreamAgg 1.00 root funcs:sum(Column#14)->Column#13 +└─Projection 12487.50 root cast(test.t.c, decimal(8,0) BINARY)->Column#14 + └─HashJoin 12487.50 root inner join, equal:[eq(test.t.c, test.t.c)] + ├─TableReader(Build) 9990.00 root data:Selection + │ └─Selection 9990.00 cop[tikv] not(isnull(test.t.c)) + │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo + └─TableReader(Probe) 9990.00 root data:Selection + └─Selection 9990.00 cop[tikv] not(isnull(test.t.c)) + └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo +explain format = 'brief' select sum(t1.d) from t t1 join t t2 on t1.d=t2.d; +id estRows task access object operator info +StreamAgg 1.00 root funcs:sum(Column#14)->Column#13 +└─Projection 12487.50 root cast(test.t.d, decimal(10,0) BINARY)->Column#14 + └─HashJoin 12487.50 root inner join, equal:[eq(test.t.d, test.t.d)] + ├─TableReader(Build) 9990.00 root data:Selection + │ └─Selection 9990.00 cop[tikv] not(isnull(test.t.d)) + │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo + └─TableReader(Probe) 9990.00 root data:Selection + └─Selection 9990.00 cop[tikv] not(isnull(test.t.d)) + └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo +explain format = 'brief' select sum(t1.e) from t t1 join t t2 on t1.e=t2.e; +id estRows task access object operator info +StreamAgg 1.00 root funcs:sum(Column#14)->Column#13 +└─Projection 12487.50 root cast(test.t.e, decimal(20,0) BINARY)->Column#14 + └─HashJoin 12487.50 root inner join, equal:[eq(test.t.e, test.t.e)] + ├─TableReader(Build) 9990.00 root data:Selection + │ └─Selection 9990.00 cop[tikv] not(isnull(test.t.e)) + │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo + └─TableReader(Probe) 9990.00 root data:Selection + └─Selection 9990.00 cop[tikv] not(isnull(test.t.e)) + └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo +explain format = 'brief' select avg(t1.a) from t t1 join t t2 on t1.a=t2.a; +id estRows task access object operator info +StreamAgg 1.00 root funcs:avg(Column#14)->Column#13 +└─Projection 12487.50 root cast(test.t.a, decimal(8,4) BINARY)->Column#14 + └─HashJoin 12487.50 root inner join, equal:[eq(test.t.a, test.t.a)] + ├─TableReader(Build) 9990.00 root data:Selection + │ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a)) + │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo + └─TableReader(Probe) 9990.00 root data:Selection + └─Selection 9990.00 cop[tikv] not(isnull(test.t.a)) + └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo +explain format = 'brief' select avg(t1.b) from t t1 join t t2 on t1.b=t2.b; +id estRows task access object operator info +StreamAgg 1.00 root funcs:avg(Column#14)->Column#13 +└─Projection 12487.50 root cast(test.t.b, decimal(10,4) BINARY)->Column#14 + └─HashJoin 12487.50 root inner join, equal:[eq(test.t.b, test.t.b)] + ├─TableReader(Build) 9990.00 root data:Selection + │ └─Selection 9990.00 cop[tikv] not(isnull(test.t.b)) + │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo + └─TableReader(Probe) 9990.00 root data:Selection + └─Selection 9990.00 cop[tikv] not(isnull(test.t.b)) + └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo +explain format = 'brief' select avg(t1.c) from t t1 join t t2 on t1.c=t2.c; +id estRows task access object operator info +StreamAgg 1.00 root funcs:avg(Column#14)->Column#13 +└─Projection 12487.50 root cast(test.t.c, decimal(13,4) BINARY)->Column#14 + └─HashJoin 12487.50 root inner join, equal:[eq(test.t.c, test.t.c)] + ├─TableReader(Build) 9990.00 root data:Selection + │ └─Selection 9990.00 cop[tikv] not(isnull(test.t.c)) + │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo + └─TableReader(Probe) 9990.00 root data:Selection + └─Selection 9990.00 cop[tikv] not(isnull(test.t.c)) + └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo +explain format = 'brief' select avg(t1.d) from t t1 join t t2 on t1.d=t2.d; +id estRows task access object operator info +StreamAgg 1.00 root funcs:avg(Column#14)->Column#13 +└─Projection 12487.50 root cast(test.t.d, decimal(15,4) BINARY)->Column#14 + └─HashJoin 12487.50 root inner join, equal:[eq(test.t.d, test.t.d)] + ├─TableReader(Build) 9990.00 root data:Selection + │ └─Selection 9990.00 cop[tikv] not(isnull(test.t.d)) + │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo + └─TableReader(Probe) 9990.00 root data:Selection + └─Selection 9990.00 cop[tikv] not(isnull(test.t.d)) + └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo +explain format = 'brief' select avg(t1.e) from t t1 join t t2 on t1.e=t2.e; +id estRows task access object operator info +StreamAgg 1.00 root funcs:avg(Column#14)->Column#13 +└─Projection 12487.50 root cast(test.t.e, decimal(24,4) BINARY)->Column#14 + └─HashJoin 12487.50 root inner join, equal:[eq(test.t.e, test.t.e)] + ├─TableReader(Build) 9990.00 root data:Selection + │ └─Selection 9990.00 cop[tikv] not(isnull(test.t.e)) + │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo + └─TableReader(Probe) 9990.00 root data:Selection + └─Selection 9990.00 cop[tikv] not(isnull(test.t.e)) + └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo +drop table if exists t; +delete from mysql.opt_rule_blacklist where name="aggregation_push_down"; +admin reload opt_rule_blacklist; + diff --git a/cmd/explaintest/r/generated_columns.result b/cmd/explaintest/r/generated_columns.result index d7f120eb28f3f..970f00880ac5d 100644 --- a/cmd/explaintest/r/generated_columns.result +++ b/cmd/explaintest/r/generated_columns.result @@ -182,7 +182,7 @@ INSERT INTO t1 (a) VALUES (2), (1), (1), (3), (NULL); EXPLAIN format = 'brief' SELECT sum(a) FROM t1 GROUP BY b; id estRows task access object operator info HashAgg 8000.00 root group by:Column#7, funcs:sum(Column#6)->Column#5 -└─Projection 10000.00 root cast(test.t1.a, decimal(32,0) BINARY)->Column#6, test.t1.b +└─Projection 10000.00 root cast(test.t1.a, decimal(10,0) BINARY)->Column#6, test.t1.b └─TableReader 10000.00 root data:TableFullScan └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo EXPLAIN format = 'brief' SELECT sum(a) FROM t1 GROUP BY c; @@ -194,13 +194,13 @@ HashAgg 8000.00 root group by:test.t1.c, funcs:sum(Column#6)->Column#5 EXPLAIN format = 'brief' SELECT sum(b) FROM t1 GROUP BY a; id estRows task access object operator info HashAgg 8000.00 root group by:Column#7, funcs:sum(Column#6)->Column#5 -└─Projection 10000.00 root cast(test.t1.b, decimal(32,0) BINARY)->Column#6, test.t1.a +└─Projection 10000.00 root cast(test.t1.b, decimal(10,0) BINARY)->Column#6, test.t1.a └─TableReader 10000.00 root data:TableFullScan └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo EXPLAIN format = 'brief' SELECT sum(b) FROM t1 GROUP BY c; id estRows task access object operator info HashAgg 8000.00 root group by:Column#9, funcs:sum(Column#8)->Column#5 -└─Projection 10000.00 root cast(test.t1.b, decimal(32,0) BINARY)->Column#8, test.t1.c +└─Projection 10000.00 root cast(test.t1.b, decimal(10,0) BINARY)->Column#8, test.t1.c └─Projection 10000.00 root test.t1.b, test.t1.c └─TableReader 10000.00 root data:TableFullScan └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo @@ -213,7 +213,7 @@ HashAgg 8000.00 root group by:test.t1.a, funcs:sum(Column#6)->Column#5 EXPLAIN format = 'brief' SELECT sum(c) FROM t1 GROUP BY b; id estRows task access object operator info HashAgg 8000.00 root group by:Column#7, funcs:sum(Column#6)->Column#5 -└─Projection 10000.00 root cast(test.t1.c, decimal(32,0) BINARY)->Column#6, test.t1.b +└─Projection 10000.00 root cast(test.t1.c, decimal(10,0) BINARY)->Column#6, test.t1.b └─Projection 10000.00 root test.t1.b, test.t1.c └─TableReader 10000.00 root data:TableFullScan └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo diff --git a/cmd/explaintest/r/index_merge.result b/cmd/explaintest/r/index_merge.result index 261acdb49d8a2..f4d6571fd8fe3 100644 --- a/cmd/explaintest/r/index_merge.result +++ b/cmd/explaintest/r/index_merge.result @@ -481,7 +481,7 @@ explain select /*+ use_index_merge(t1) */ sum(c1) from t1 where (c1 < 10 or c2 < id estRows task access object operator info Sort_6 1473.49 root Column#5 └─HashAgg_11 1473.49 root group by:Column#10, funcs:sum(Column#9)->Column#5 - └─Projection_18 1841.86 root cast(test.t1.c1, decimal(32,0) BINARY)->Column#9, test.t1.c1 + └─Projection_18 1841.86 root cast(test.t1.c1, decimal(10,0) BINARY)->Column#9, test.t1.c1 └─IndexMerge_16 1841.86 root ├─IndexRangeScan_12(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo ├─IndexRangeScan_13(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo diff --git a/cmd/explaintest/r/select.result b/cmd/explaintest/r/select.result index 959e761aad086..f364a577e1071 100644 --- a/cmd/explaintest/r/select.result +++ b/cmd/explaintest/r/select.result @@ -385,7 +385,7 @@ id estRows task access object operator info Projection 10000.00 root and(or(or(gt(Column#11, 1), ne(test.t.a, Column#10)), if(ne(Column#12, 0), , 0)), and(ne(Column#13, 0), if(isnull(test.t.a), , 1)))->Column#14 └─HashJoin 10000.00 root CARTESIAN inner join ├─StreamAgg(Build) 1.00 root funcs:max(Column#16)->Column#10, funcs:count(distinct Column#17)->Column#11, funcs:sum(Column#18)->Column#12, funcs:count(1)->Column#13 - │ └─Projection 10000.00 root test.t.a, test.t.a, cast(isnull(test.t.a), decimal(22,0) BINARY)->Column#18 + │ └─Projection 10000.00 root test.t.a, test.t.a, cast(isnull(test.t.a), decimal(20,0) BINARY)->Column#18 │ └─TableReader 10000.00 root data:TableFullScan │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo └─TableReader(Probe) 10000.00 root data:TableFullScan @@ -395,7 +395,7 @@ id estRows task access object operator info Projection 10000.00 root or(and(and(le(Column#11, 1), eq(test.t.a, Column#10)), if(ne(Column#12, 0), , 1)), or(eq(Column#13, 0), if(isnull(test.t.a), , 0)))->Column#14 └─HashJoin 10000.00 root CARTESIAN inner join ├─StreamAgg(Build) 1.00 root funcs:firstrow(Column#16)->Column#10, funcs:count(distinct Column#17)->Column#11, funcs:sum(Column#18)->Column#12, funcs:count(1)->Column#13 - │ └─Projection 10000.00 root test.t.a, test.t.a, cast(isnull(test.t.a), decimal(22,0) BINARY)->Column#18 + │ └─Projection 10000.00 root test.t.a, test.t.a, cast(isnull(test.t.a), decimal(20,0) BINARY)->Column#18 │ └─TableReader 10000.00 root data:TableFullScan │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo └─TableReader(Probe) 10000.00 root data:TableFullScan diff --git a/cmd/explaintest/r/tpch.result b/cmd/explaintest/r/tpch.result index 319713d946aa2..c9edd22189514 100644 --- a/cmd/explaintest/r/tpch.result +++ b/cmd/explaintest/r/tpch.result @@ -773,7 +773,7 @@ id estRows task access object operator info Sort 1.00 root tpch.lineitem.l_shipmode └─Projection 1.00 root tpch.lineitem.l_shipmode, Column#27, Column#28 └─HashAgg 1.00 root group by:Column#40, funcs:sum(Column#37)->Column#27, funcs:sum(Column#38)->Column#28, funcs:firstrow(Column#39)->tpch.lineitem.l_shipmode - └─Projection 10023369.01 root cast(case(or(eq(tpch.orders.o_orderpriority, 1-URGENT), eq(tpch.orders.o_orderpriority, 2-HIGH)), 1, 0), decimal(22,0) BINARY)->Column#37, cast(case(and(ne(tpch.orders.o_orderpriority, 1-URGENT), ne(tpch.orders.o_orderpriority, 2-HIGH)), 1, 0), decimal(22,0) BINARY)->Column#38, tpch.lineitem.l_shipmode, tpch.lineitem.l_shipmode + └─Projection 10023369.01 root cast(case(or(eq(tpch.orders.o_orderpriority, 1-URGENT), eq(tpch.orders.o_orderpriority, 2-HIGH)), 1, 0), decimal(20,0) BINARY)->Column#37, cast(case(and(ne(tpch.orders.o_orderpriority, 1-URGENT), ne(tpch.orders.o_orderpriority, 2-HIGH)), 1, 0), decimal(20,0) BINARY)->Column#38, tpch.lineitem.l_shipmode, tpch.lineitem.l_shipmode └─Projection 10023369.01 root tpch.orders.o_orderpriority, tpch.lineitem.l_shipmode └─IndexJoin 10023369.01 root inner join, inner:TableReader, outer key:tpch.lineitem.l_orderkey, inner key:tpch.orders.o_orderkey, equal cond:eq(tpch.lineitem.l_orderkey, tpch.orders.o_orderkey) ├─TableReader(Build) 10023369.01 root data:Selection diff --git a/cmd/explaintest/r/window_function.result b/cmd/explaintest/r/window_function.result index 6c92b63dcd0d3..b29d1e5d3fba7 100644 --- a/cmd/explaintest/r/window_function.result +++ b/cmd/explaintest/r/window_function.result @@ -6,47 +6,47 @@ set @@session.tidb_window_concurrency = 1; explain format = 'brief' select sum(a) over() from t; id estRows task access object operator info Projection 10000.00 root Column#6 -└─Window 10000.00 root sum(cast(test.t.a, decimal(32,0) BINARY))->Column#6 over() +└─Window 10000.00 root sum(cast(test.t.a, decimal(10,0) BINARY))->Column#6 over() └─IndexReader 10000.00 root index:IndexFullScan └─IndexFullScan 10000.00 cop[tikv] table:t, index:idx(a) keep order:false, stats:pseudo explain format = 'brief' select sum(a) over(partition by a) from t; id estRows task access object operator info Projection 10000.00 root Column#6 -└─Window 10000.00 root sum(cast(test.t.a, decimal(32,0) BINARY))->Column#6 over(partition by test.t.a) +└─Window 10000.00 root sum(cast(test.t.a, decimal(10,0) BINARY))->Column#6 over(partition by test.t.a) └─IndexReader 10000.00 root index:IndexFullScan └─IndexFullScan 10000.00 cop[tikv] table:t, index:idx(a) keep order:true, stats:pseudo explain format = 'brief' select sum(a) over(partition by a order by b) from t; id estRows task access object operator info Projection 10000.00 root Column#6 -└─Window 10000.00 root sum(cast(test.t.a, decimal(32,0) BINARY))->Column#6 over(partition by test.t.a order by test.t.b range between unbounded preceding and current row) +└─Window 10000.00 root sum(cast(test.t.a, decimal(10,0) BINARY))->Column#6 over(partition by test.t.a order by test.t.b range between unbounded preceding and current row) └─Sort 10000.00 root test.t.a, test.t.b └─TableReader 10000.00 root data:TableFullScan └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo explain format = 'brief' select sum(a) over(partition by a order by b rows unbounded preceding) from t; id estRows task access object operator info Projection 10000.00 root Column#6 -└─Window 10000.00 root sum(cast(test.t.a, decimal(32,0) BINARY))->Column#6 over(partition by test.t.a order by test.t.b rows between unbounded preceding and current row) +└─Window 10000.00 root sum(cast(test.t.a, decimal(10,0) BINARY))->Column#6 over(partition by test.t.a order by test.t.b rows between unbounded preceding and current row) └─Sort 10000.00 root test.t.a, test.t.b └─TableReader 10000.00 root data:TableFullScan └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo explain format = 'brief' select sum(a) over(partition by a order by b rows between 1 preceding and 1 following) from t; id estRows task access object operator info Projection 10000.00 root Column#6 -└─Window 10000.00 root sum(cast(test.t.a, decimal(32,0) BINARY))->Column#6 over(partition by test.t.a order by test.t.b rows between 1 preceding and 1 following) +└─Window 10000.00 root sum(cast(test.t.a, decimal(10,0) BINARY))->Column#6 over(partition by test.t.a order by test.t.b rows between 1 preceding and 1 following) └─Sort 10000.00 root test.t.a, test.t.b └─TableReader 10000.00 root data:TableFullScan └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo explain format = 'brief' select sum(a) over(partition by a order by b range between 1 preceding and 1 following) from t; id estRows task access object operator info Projection 10000.00 root Column#6 -└─Window 10000.00 root sum(cast(test.t.a, decimal(32,0) BINARY))->Column#6 over(partition by test.t.a order by test.t.b range between 1 preceding and 1 following) +└─Window 10000.00 root sum(cast(test.t.a, decimal(10,0) BINARY))->Column#6 over(partition by test.t.a order by test.t.b range between 1 preceding and 1 following) └─Sort 10000.00 root test.t.a, test.t.b └─TableReader 10000.00 root data:TableFullScan └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo explain format = 'brief' select sum(a) over(partition by a order by c range between interval '2:30' minute_second preceding and interval '2:30' minute_second following) from t; id estRows task access object operator info Projection 10000.00 root Column#6 -└─Window 10000.00 root sum(cast(test.t.a, decimal(32,0) BINARY))->Column#6 over(partition by test.t.a order by test.t.c range between interval "2:30" "MINUTE_SECOND" preceding and interval "2:30" "MINUTE_SECOND" following) +└─Window 10000.00 root sum(cast(test.t.a, decimal(10,0) BINARY))->Column#6 over(partition by test.t.a order by test.t.c range between interval "2:30" "MINUTE_SECOND" preceding and interval "2:30" "MINUTE_SECOND" following) └─Sort 10000.00 root test.t.a, test.t.c └─TableReader 10000.00 root data:TableFullScan └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo @@ -54,20 +54,20 @@ set @@session.tidb_window_concurrency = 4; explain format = 'brief' select sum(a) over() from t; id estRows task access object operator info Projection 10000.00 root Column#6 -└─Window 10000.00 root sum(cast(test.t.a, decimal(32,0) BINARY))->Column#6 over() +└─Window 10000.00 root sum(cast(test.t.a, decimal(10,0) BINARY))->Column#6 over() └─IndexReader 10000.00 root index:IndexFullScan └─IndexFullScan 10000.00 cop[tikv] table:t, index:idx(a) keep order:false, stats:pseudo explain format = 'brief' select sum(a) over(partition by a) from t; id estRows task access object operator info Projection 10000.00 root Column#6 -└─Window 10000.00 root sum(cast(test.t.a, decimal(32,0) BINARY))->Column#6 over(partition by test.t.a) +└─Window 10000.00 root sum(cast(test.t.a, decimal(10,0) BINARY))->Column#6 over(partition by test.t.a) └─IndexReader 10000.00 root index:IndexFullScan └─IndexFullScan 10000.00 cop[tikv] table:t, index:idx(a) keep order:true, stats:pseudo explain format = 'brief' select sum(a) over(partition by a order by b) from t; id estRows task access object operator info Projection 10000.00 root Column#6 └─Shuffle 10000.00 root execution info: concurrency:4, data sources:[TableReader] - └─Window 10000.00 root sum(cast(test.t.a, decimal(32,0) BINARY))->Column#6 over(partition by test.t.a order by test.t.b range between unbounded preceding and current row) + └─Window 10000.00 root sum(cast(test.t.a, decimal(10,0) BINARY))->Column#6 over(partition by test.t.a order by test.t.b range between unbounded preceding and current row) └─Sort 10000.00 root test.t.a, test.t.b └─TableReader 10000.00 root data:TableFullScan └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo @@ -75,7 +75,7 @@ explain format = 'brief' select sum(a) over(partition by a order by b rows unbou id estRows task access object operator info Projection 10000.00 root Column#6 └─Shuffle 10000.00 root execution info: concurrency:4, data sources:[TableReader] - └─Window 10000.00 root sum(cast(test.t.a, decimal(32,0) BINARY))->Column#6 over(partition by test.t.a order by test.t.b rows between unbounded preceding and current row) + └─Window 10000.00 root sum(cast(test.t.a, decimal(10,0) BINARY))->Column#6 over(partition by test.t.a order by test.t.b rows between unbounded preceding and current row) └─Sort 10000.00 root test.t.a, test.t.b └─TableReader 10000.00 root data:TableFullScan └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo @@ -83,7 +83,7 @@ explain format = 'brief' select sum(a) over(partition by a order by b rows betwe id estRows task access object operator info Projection 10000.00 root Column#6 └─Shuffle 10000.00 root execution info: concurrency:4, data sources:[TableReader] - └─Window 10000.00 root sum(cast(test.t.a, decimal(32,0) BINARY))->Column#6 over(partition by test.t.a order by test.t.b rows between 1 preceding and 1 following) + └─Window 10000.00 root sum(cast(test.t.a, decimal(10,0) BINARY))->Column#6 over(partition by test.t.a order by test.t.b rows between 1 preceding and 1 following) └─Sort 10000.00 root test.t.a, test.t.b └─TableReader 10000.00 root data:TableFullScan └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo @@ -91,7 +91,7 @@ explain format = 'brief' select sum(a) over(partition by a order by b range betw id estRows task access object operator info Projection 10000.00 root Column#6 └─Shuffle 10000.00 root execution info: concurrency:4, data sources:[TableReader] - └─Window 10000.00 root sum(cast(test.t.a, decimal(32,0) BINARY))->Column#6 over(partition by test.t.a order by test.t.b range between 1 preceding and 1 following) + └─Window 10000.00 root sum(cast(test.t.a, decimal(10,0) BINARY))->Column#6 over(partition by test.t.a order by test.t.b range between 1 preceding and 1 following) └─Sort 10000.00 root test.t.a, test.t.b └─TableReader 10000.00 root data:TableFullScan └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo @@ -99,7 +99,7 @@ explain format = 'brief' select sum(a) over(partition by a order by c range betw id estRows task access object operator info Projection 10000.00 root Column#6 └─Shuffle 10000.00 root execution info: concurrency:4, data sources:[TableReader] - └─Window 10000.00 root sum(cast(test.t.a, decimal(32,0) BINARY))->Column#6 over(partition by test.t.a order by test.t.c range between interval "2:30" "MINUTE_SECOND" preceding and interval "2:30" "MINUTE_SECOND" following) + └─Window 10000.00 root sum(cast(test.t.a, decimal(10,0) BINARY))->Column#6 over(partition by test.t.a order by test.t.c range between interval "2:30" "MINUTE_SECOND" preceding and interval "2:30" "MINUTE_SECOND" following) └─Sort 10000.00 root test.t.a, test.t.c └─TableReader 10000.00 root data:TableFullScan └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo @@ -110,7 +110,7 @@ analyze table t1; explain format = 'brief' select sum(a) over(partition by b) from t1; id estRows task access object operator info Projection 2.00 root Column#4 -└─Window 2.00 root sum(cast(test.t1.a, decimal(32,0) BINARY))->Column#4 over(partition by test.t1.b) +└─Window 2.00 root sum(cast(test.t1.a, decimal(10,0) BINARY))->Column#4 over(partition by test.t1.b) └─Sort 2.00 root test.t1.b └─TableReader 2.00 root data:TableFullScan └─TableFullScan 2.00 cop[tikv] table:t1 keep order:false @@ -120,7 +120,7 @@ explain format = 'brief' select sum(a) over(partition by b) from t1; id estRows task access object operator info Projection 3.00 root Column#4 └─Shuffle 3.00 root execution info: concurrency:2, data sources:[TableReader] - └─Window 3.00 root sum(cast(test.t1.a, decimal(32,0) BINARY))->Column#4 over(partition by test.t1.b) + └─Window 3.00 root sum(cast(test.t1.a, decimal(10,0) BINARY))->Column#4 over(partition by test.t1.b) └─Sort 3.00 root test.t1.b └─TableReader 3.00 root data:TableFullScan └─TableFullScan 3.00 cop[tikv] table:t1 keep order:false diff --git a/cmd/explaintest/t/explain_easy.test b/cmd/explaintest/t/explain_easy.test index 4d3b698bd7d72..161b716f5a666 100644 --- a/cmd/explaintest/t/explain_easy.test +++ b/cmd/explaintest/t/explain_easy.test @@ -218,3 +218,22 @@ explain format = 'brief' select count(a) from t group by b order by (select coun explain format = 'brief' select (select sum(count(a))) from t; explain format = 'brief' select sum(a), (select sum(a)), count(a) from t group by b order by (select count(a)); drop table if exists t; + +# lower precision for cast to decimal for integer type variables in sum function +create table t(a tinyint, b smallint, c mediumint, d int, e bigint); +insert into mysql.opt_rule_blacklist VALUES("aggregation_push_down"); +admin reload opt_rule_blacklist; +explain format = 'brief' select sum(t1.a) from t t1 join t t2 on t1.a=t2.a; +explain format = 'brief' select sum(t1.b) from t t1 join t t2 on t1.b=t2.b; +explain format = 'brief' select sum(t1.c) from t t1 join t t2 on t1.c=t2.c; +explain format = 'brief' select sum(t1.d) from t t1 join t t2 on t1.d=t2.d; +explain format = 'brief' select sum(t1.e) from t t1 join t t2 on t1.e=t2.e; +# note that avg will be converted to count and sum, and .decimal field will be non-zero +explain format = 'brief' select avg(t1.a) from t t1 join t t2 on t1.a=t2.a; +explain format = 'brief' select avg(t1.b) from t t1 join t t2 on t1.b=t2.b; +explain format = 'brief' select avg(t1.c) from t t1 join t t2 on t1.c=t2.c; +explain format = 'brief' select avg(t1.d) from t t1 join t t2 on t1.d=t2.d; +explain format = 'brief' select avg(t1.e) from t t1 join t t2 on t1.e=t2.e; +drop table if exists t; +delete from mysql.opt_rule_blacklist where name="aggregation_push_down"; +admin reload opt_rule_blacklist; diff --git a/executor/testdata/agg_suite_out.json b/executor/testdata/agg_suite_out.json index abfb0f9d102ef..1937b7f4cc358 100644 --- a/executor/testdata/agg_suite_out.json +++ b/executor/testdata/agg_suite_out.json @@ -54,7 +54,7 @@ " ├─Apply(Build) 1.00 root CARTESIAN left outer join", " │ ├─Apply(Build) 1.00 root CARTESIAN left outer join", " │ │ ├─HashAgg(Build) 1.00 root funcs:sum(Column#28)->Column#9, funcs:firstrow(Column#29)->test.test.a", - " │ │ │ └─Projection 10000.00 root cast(test.test.a, decimal(32,0) BINARY)->Column#28, test.test.a", + " │ │ │ └─Projection 10000.00 root cast(test.test.a, decimal(10,0) BINARY)->Column#28, test.test.a", " │ │ │ └─TableReader 10000.00 root data:TableFullScan", " │ │ │ └─TableFullScan 10000.00 cop[tikv] table:tt keep order:false, stats:pseudo", " │ │ └─Projection(Probe) 1.00 root ->Column#12", diff --git a/expression/aggregation/base_func.go b/expression/aggregation/base_func.go index 1c639eeb4f14d..a185910c82b97 100644 --- a/expression/aggregation/base_func.go +++ b/expression/aggregation/base_func.go @@ -178,7 +178,7 @@ func (a *baseFuncDesc) typeInfer4ApproxPercentile(ctx sessionctx.Context) error return nil } -// typeInfer4Sum should returns a "decimal", otherwise it returns a "double". +// typeInfer4Sum should return a "decimal", otherwise it returns a "double". // Because child returns integer or decimal type. func (a *baseFuncDesc) typeInfer4Sum(ctx sessionctx.Context) { switch a.Args[0].GetType().Tp { @@ -421,6 +421,7 @@ func (a *baseFuncDesc) WrapCastForAggArgs(ctx sessionctx.Context) { if a.Args[i].GetType().Tp == mysql.TypeNull { continue } + tpOld := a.Args[i].GetType().Tp a.Args[i] = castFunc(ctx, a.Args[i]) if a.Name != ast.AggFuncAvg && a.Name != ast.AggFuncSum { continue @@ -443,5 +444,37 @@ func (a *baseFuncDesc) WrapCastForAggArgs(ctx sessionctx.Context) { originTp := a.Args[i].GetType().Tp *(a.Args[i].GetType()) = *(a.RetTp) a.Args[i].GetType().Tp = originTp + + // refine each mysql integer type to the needed decimal precision for sum + if a.Name == ast.AggFuncSum { + adjustDecimalLenForSumInteger(a.Args[i].GetType(), tpOld) + } + } +} + +func adjustDecimalLenForSumInteger(ft *types.FieldType, tpOld byte) { + if types.IsTypeInteger(tpOld) && ft.Tp == mysql.TypeNewDecimal { + if flen, err := minimalDecimalLenForHoldingInteger(tpOld); err == nil { + ft.Flen = mathutil.Min(ft.Flen, flen+ft.Decimal) + } + } +} + +func minimalDecimalLenForHoldingInteger(tp byte) (int, error) { + switch tp { + case mysql.TypeTiny: + return 3, nil + case mysql.TypeShort: + return 5, nil + case mysql.TypeInt24: + return 8, nil + case mysql.TypeLong: + return 10, nil + case mysql.TypeLonglong: + return 20, nil + case mysql.TypeYear: + return 4, nil + default: + return -1, errors.Errorf("Invalid type: %v", tp) } } diff --git a/expression/builtin.go b/expression/builtin.go index 1b67239613498..71fde7647a598 100644 --- a/expression/builtin.go +++ b/expression/builtin.go @@ -114,7 +114,7 @@ func newBaseBuiltinFunc(ctx sessionctx.Context, funcName string, args []Expressi // newBaseBuiltinFuncWithTp creates a built-in function signature with specified types of arguments and the return type of the function. // argTps indicates the types of the args, retType indicates the return type of the built-in function. -// Every built-in function needs determined argTps and retType when we create it. +// Every built-in function needs to be determined argTps and retType when we create it. func newBaseBuiltinFuncWithTp(ctx sessionctx.Context, funcName string, args []Expression, retType types.EvalType, argTps ...types.EvalType) (bf baseBuiltinFunc, err error) { if len(args) != len(argTps) { panic("unexpected length of args and argTps") diff --git a/parser/mysql/type.go b/parser/mysql/type.go index be030bd9c81d3..c54d0f8984b63 100644 --- a/parser/mysql/type.go +++ b/parser/mysql/type.go @@ -16,15 +16,15 @@ package mysql // MySQL type information. const ( TypeUnspecified byte = 0 - TypeTiny byte = 1 - TypeShort byte = 2 - TypeLong byte = 3 + TypeTiny byte = 1 // TINYINT + TypeShort byte = 2 // SMALLINT + TypeLong byte = 3 // INT TypeFloat byte = 4 TypeDouble byte = 5 TypeNull byte = 6 TypeTimestamp byte = 7 - TypeLonglong byte = 8 - TypeInt24 byte = 9 + TypeLonglong byte = 8 // BIGINT + TypeInt24 byte = 9 // MEDIUMINT TypeDate byte = 10 /* TypeDuration original name was TypeTime, renamed to TypeDuration to resolve the conflict with Go type Time.*/ TypeDuration byte = 11 diff --git a/planner/cascades/testdata/integration_suite_out.json b/planner/cascades/testdata/integration_suite_out.json index 3b2719c02d293..21b601bfc9198 100644 --- a/planner/cascades/testdata/integration_suite_out.json +++ b/planner/cascades/testdata/integration_suite_out.json @@ -197,7 +197,7 @@ " └─Sort_29 6400.00 root test.t.b", " └─Selection_28 6400.00 root gt(Column#4, 1)", " └─HashAgg_19 8000.00 root group by:test.t.b, funcs:avg(Column#11)->Column#3, funcs:sum(Column#12)->Column#4, funcs:firstrow(test.t.b)->test.t.b", - " └─Projection_20 10000.00 root cast(test.t.a, decimal(15,4) BINARY)->Column#11, cast(test.t.a, decimal(32,0) BINARY)->Column#12, test.t.b, test.t.b", + " └─Projection_20 10000.00 root cast(test.t.a, decimal(15,4) BINARY)->Column#11, cast(test.t.a, decimal(10,0) BINARY)->Column#12, test.t.b, test.t.b", " └─TableReader_21 10000.00 root data:TableFullScan_22", " └─TableFullScan_22 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" ], @@ -636,7 +636,7 @@ "Plan": [ "Projection_8 8000.00 root Column#5, test.t.c, Column#5, Column#6, Column#7, Column#8, Column#9", "└─HashAgg_9 8000.00 root group by:test.t.c, funcs:avg(Column#10)->Column#5, funcs:count(distinct test.t.a, test.t.b)->Column#6, funcs:count(distinct test.t.a)->Column#7, funcs:count(distinct test.t.c)->Column#8, funcs:sum(Column#11)->Column#9, funcs:firstrow(test.t.c)->test.t.c", - " └─Projection_10 10000.00 root cast(test.t.b, decimal(15,4) BINARY)->Column#10, test.t.a, test.t.b, test.t.a, test.t.c, cast(test.t.b, decimal(32,0) BINARY)->Column#11, test.t.c, test.t.c", + " └─Projection_10 10000.00 root cast(test.t.b, decimal(15,4) BINARY)->Column#10, test.t.a, test.t.b, test.t.a, test.t.c, cast(test.t.b, decimal(10,0) BINARY)->Column#11, test.t.c, test.t.c", " └─TableReader_11 10000.00 root data:TableFullScan_12", " └─TableFullScan_12 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" ], diff --git a/planner/core/preprocess.go b/planner/core/preprocess.go index 6e2f9aaed4f53..7eb2a3c041c52 100644 --- a/planner/core/preprocess.go +++ b/planner/core/preprocess.go @@ -109,7 +109,7 @@ func TryAddExtraLimit(ctx sessionctx.Context, node ast.StmtNode) ast.StmtNode { return node } -// Preprocess resolves table names of the node, and checks some statements validation. +// Preprocess resolves table names of the node, and checks some statements' validation. // preprocessReturn used to extract the infoschema for the tableName and the timestamp from the asof clause. func Preprocess(ctx sessionctx.Context, node ast.Node, preprocessOpt ...PreprocessOpt) error { v := preprocessor{ctx: ctx, tableAliasInJoin: make([]map[string]interface{}, 0), withName: make(map[string]interface{})} diff --git a/planner/core/testdata/integration_serial_suite_out.json b/planner/core/testdata/integration_serial_suite_out.json index fa07e7323f022..8e88ca5157d3f 100644 --- a/planner/core/testdata/integration_serial_suite_out.json +++ b/planner/core/testdata/integration_serial_suite_out.json @@ -1463,7 +1463,7 @@ "StreamAgg 1.00 root funcs:avg(Column#7, Column#8)->Column#4", "└─TableReader 1.00 root data:StreamAgg", " └─StreamAgg 1.00 batchCop[tiflash] funcs:count(Column#9)->Column#7, funcs:sum(Column#10)->Column#8", - " └─Projection 10000.00 batchCop[tiflash] test.t.a, cast(test.t.a, decimal(37,4) BINARY)->Column#10", + " └─Projection 10000.00 batchCop[tiflash] test.t.a, cast(test.t.a, decimal(14,4) BINARY)->Column#10", " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" ], "Warn": null @@ -1474,7 +1474,7 @@ "StreamAgg 1.00 root funcs:avg(Column#7, Column#8)->Column#4", "└─TableReader 1.00 root data:StreamAgg", " └─StreamAgg 1.00 batchCop[tiflash] funcs:count(Column#9)->Column#7, funcs:sum(Column#10)->Column#8", - " └─Projection 10000.00 batchCop[tiflash] test.t.a, cast(test.t.a, decimal(37,4) BINARY)->Column#10", + " └─Projection 10000.00 batchCop[tiflash] test.t.a, cast(test.t.a, decimal(14,4) BINARY)->Column#10", " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" ], "Warn": null @@ -1485,7 +1485,7 @@ "StreamAgg 1.00 root funcs:sum(Column#6)->Column#4", "└─TableReader 1.00 root data:StreamAgg", " └─StreamAgg 1.00 batchCop[tiflash] funcs:sum(Column#7)->Column#6", - " └─Projection 10000.00 batchCop[tiflash] cast(test.t.a, decimal(32,0) BINARY)->Column#7", + " └─Projection 10000.00 batchCop[tiflash] cast(test.t.a, decimal(10,0) BINARY)->Column#7", " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" ], "Warn": null @@ -1496,7 +1496,7 @@ "StreamAgg 1.00 root funcs:sum(Column#6)->Column#4", "└─TableReader 1.00 root data:StreamAgg", " └─StreamAgg 1.00 batchCop[tiflash] funcs:sum(Column#7)->Column#6", - " └─Projection 10000.00 batchCop[tiflash] cast(plus(test.t.a, 1), decimal(41,0) BINARY)->Column#7", + " └─Projection 10000.00 batchCop[tiflash] cast(plus(test.t.a, 1), decimal(20,0) BINARY)->Column#7", " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" ], "Warn": null @@ -1507,7 +1507,7 @@ "StreamAgg 1.00 root funcs:sum(Column#6)->Column#4", "└─TableReader 1.00 root data:StreamAgg", " └─StreamAgg 1.00 batchCop[tiflash] funcs:sum(Column#7)->Column#6", - " └─Projection 10000.00 batchCop[tiflash] cast(isnull(test.t.a), decimal(22,0) BINARY)->Column#7", + " └─Projection 10000.00 batchCop[tiflash] cast(isnull(test.t.a), decimal(20,0) BINARY)->Column#7", " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" ], "Warn": null @@ -1837,7 +1837,7 @@ "HashAgg 1.00 root funcs:sum(Column#8)->Column#6", "└─TableReader 1.00 root data:HashAgg", " └─HashAgg 1.00 batchCop[tiflash] funcs:sum(Column#10)->Column#8", - " └─Projection 10000.00 batchCop[tiflash] cast(plus(test.t.id, 1), decimal(41,0) BINARY)->Column#10", + " └─Projection 10000.00 batchCop[tiflash] cast(plus(test.t.id, 1), decimal(20,0) BINARY)->Column#10", " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" ] }, @@ -1866,7 +1866,7 @@ "StreamAgg 1.00 root funcs:sum(Column#8)->Column#6", "└─TableReader 1.00 root data:StreamAgg", " └─StreamAgg 1.00 batchCop[tiflash] funcs:sum(Column#10)->Column#8", - " └─Projection 10000.00 batchCop[tiflash] cast(plus(test.t.id, 1), decimal(41,0) BINARY)->Column#10", + " └─Projection 10000.00 batchCop[tiflash] cast(plus(test.t.id, 1), decimal(20,0) BINARY)->Column#10", " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" ] }, @@ -1992,7 +1992,7 @@ "└─TableReader 1.00 root data:ExchangeSender", " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", " └─HashAgg 1.00 batchCop[tiflash] funcs:sum(Column#11)->Column#9", - " └─Projection 10000.00 batchCop[tiflash] cast(plus(test.t.id, 1), decimal(41,0) BINARY)->Column#11", + " └─Projection 10000.00 batchCop[tiflash] cast(plus(test.t.id, 1), decimal(20,0) BINARY)->Column#11", " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" ] }, @@ -2024,7 +2024,7 @@ "└─TableReader 1.00 root data:ExchangeSender", " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", " └─HashAgg 1.00 batchCop[tiflash] funcs:sum(Column#11)->Column#10", - " └─Projection 10000.00 batchCop[tiflash] cast(plus(test.t.id, 1), decimal(41,0) BINARY)->Column#11", + " └─Projection 10000.00 batchCop[tiflash] cast(plus(test.t.id, 1), decimal(20,0) BINARY)->Column#11", " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" ] }, @@ -2442,7 +2442,7 @@ "└─TableReader 1.00 root data:ExchangeSender", " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", " └─HashAgg 1.00 batchCop[tiflash] funcs:sum(Column#9)->Column#8", - " └─Projection 10000.00 batchCop[tiflash] cast(plus(test.t.id, 1), decimal(41,0) BINARY)->Column#9", + " └─Projection 10000.00 batchCop[tiflash] cast(plus(test.t.id, 1), decimal(20,0) BINARY)->Column#9", " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" ] }, @@ -2595,7 +2595,7 @@ "└─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: PassThrough", " └─Projection 7992.00 batchCop[tiflash] Column#7", " └─HashAgg 7992.00 batchCop[tiflash] group by:Column#11, funcs:sum(Column#10)->Column#7", - " └─Projection 12487.50 batchCop[tiflash] cast(test.t.id, decimal(32,0) BINARY)->Column#10, test.t.id", + " └─Projection 12487.50 batchCop[tiflash] cast(test.t.id, decimal(10,0) BINARY)->Column#10, test.t.id", " └─HashJoin 12487.50 batchCop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", " ├─ExchangeReceiver(Build) 9990.00 batchCop[tiflash] ", " │ └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: N/A]", @@ -2622,7 +2622,7 @@ " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", " └─Projection(Probe) 7992.00 batchCop[tiflash] Column#11, test.t.id", " └─HashAgg 7992.00 batchCop[tiflash] group by:Column#39, funcs:sum(Column#37)->Column#11, funcs:firstrow(Column#38)->test.t.id", - " └─Projection 9990.00 batchCop[tiflash] cast(test.t.id, decimal(32,0) BINARY)->Column#37, test.t.id, test.t.id", + " └─Projection 9990.00 batchCop[tiflash] cast(test.t.id, decimal(10,0) BINARY)->Column#37, test.t.id, test.t.id", " └─HashJoin 9990.00 batchCop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", " ├─Projection(Build) 7992.00 batchCop[tiflash] test.t.id, Column#13", " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:firstrow(test.t.id)->test.t.id, funcs:sum(Column#17)->Column#13", @@ -2704,7 +2704,7 @@ "└─ExchangeSender 6400.00 batchCop[tiflash] ExchangeType: PassThrough", " └─Projection 6400.00 batchCop[tiflash] Column#4", " └─HashAgg 6400.00 batchCop[tiflash] group by:Column#22, funcs:sum(Column#21)->Column#4", - " └─Projection 6400.00 batchCop[tiflash] cast(test.t.id, decimal(32,0) BINARY)->Column#21, test.t.value", + " └─Projection 6400.00 batchCop[tiflash] cast(test.t.id, decimal(10,0) BINARY)->Column#21, test.t.value", " └─Projection 6400.00 batchCop[tiflash] test.t.id, test.t.value", " └─HashAgg 6400.00 batchCop[tiflash] group by:test.t.id, test.t.value, funcs:firstrow(test.t.id)->test.t.id, funcs:firstrow(test.t.value)->test.t.value", " └─ExchangeReceiver 6400.00 batchCop[tiflash] ", @@ -2868,7 +2868,7 @@ " └─ExchangeReceiver 7992.00 batchCop[tiflash] ", " └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: N/A]", " └─HashAgg 7992.00 batchCop[tiflash] group by:Column#11, funcs:sum(Column#10)->Column#8", - " └─Projection 12487.50 batchCop[tiflash] cast(test.t.id, decimal(32,0) BINARY)->Column#10, test.t.id", + " └─Projection 12487.50 batchCop[tiflash] cast(test.t.id, decimal(10,0) BINARY)->Column#10, test.t.id", " └─HashJoin 12487.50 batchCop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", " ├─ExchangeReceiver(Build) 9990.00 batchCop[tiflash] ", " │ └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: Broadcast", @@ -2922,7 +2922,7 @@ " └─ExchangeReceiver 7992.00 batchCop[tiflash] ", " └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: N/A]", " └─HashAgg 7992.00 batchCop[tiflash] group by:Column#33, funcs:sum(Column#32)->Column#17", - " └─Projection 9990.00 batchCop[tiflash] cast(test.t.id, decimal(32,0) BINARY)->Column#32, test.t.id", + " └─Projection 9990.00 batchCop[tiflash] cast(test.t.id, decimal(10,0) BINARY)->Column#32, test.t.id", " └─HashJoin 9990.00 batchCop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", " ├─ExchangeReceiver(Build) 7992.00 batchCop[tiflash] ", " │ └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: Broadcast", @@ -3157,7 +3157,7 @@ " │ └─TableReader 1.00 root data:ExchangeSender", " │ └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", " │ └─HashAgg 1.00 batchCop[tiflash] funcs:min(Column#42)->Column#36, funcs:sum(Column#43)->Column#37, funcs:count(1)->Column#38", - " │ └─Projection 10000.00 batchCop[tiflash] test.ts.col_varchar_64, cast(isnull(test.ts.col_varchar_64), decimal(22,0) BINARY)->Column#43", + " │ └─Projection 10000.00 batchCop[tiflash] test.ts.col_varchar_64, cast(isnull(test.ts.col_varchar_64), decimal(20,0) BINARY)->Column#43", " │ └─TableFullScan 10000.00 batchCop[tiflash] table:SUBQUERY4_t1 keep order:false, stats:pseudo", " └─TableReader(Probe) 12487.50 root data:ExchangeSender", " └─ExchangeSender 12487.50 cop[tiflash] ExchangeType: PassThrough", @@ -3379,7 +3379,7 @@ "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", " └─Projection 8000.00 batchCop[tiflash] Column#5, Column#6, Column#7, div(Column#8, cast(case(eq(Column#11, 0), 1, Column#11), decimal(20,0) BINARY))->Column#8", " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#20, funcs:group_concat(Column#13, Column#14, Column#15 order by Column#16 separator \",\")->Column#5, funcs:count(1)->Column#6, funcs:min(Column#17)->Column#7, funcs:count(Column#18)->Column#11, funcs:sum(Column#19)->Column#8", - " └─Projection 10000.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#15, test.ts.col_0, test.ts.col_0, test.ts.id, cast(test.ts.id, decimal(37,4) BINARY)->Column#19, test.ts.col_2", + " └─Projection 10000.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#15, test.ts.col_0, test.ts.col_0, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#19, test.ts.col_2", " └─ExchangeReceiver 10000.00 batchCop[tiflash] ", " └─ExchangeSender 10000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_2, collate: N/A]", " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" @@ -3399,7 +3399,7 @@ " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_2, collate: N/A]", " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#32, Column#33, Column#34, Column#35, funcs:count(1)->Column#25, funcs:max(Column#29)->Column#26, funcs:count(Column#30)->Column#27, funcs:sum(Column#31)->Column#28", - " └─Projection 10000.00 batchCop[tiflash] test.ts.col_1, test.ts.id, cast(test.ts.id, decimal(37,4) BINARY)->Column#31, test.ts.col_2, test.ts.col_0, test.ts.col_1, test.ts.id", + " └─Projection 10000.00 batchCop[tiflash] test.ts.col_1, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#31, test.ts.col_2, test.ts.col_0, test.ts.col_1, test.ts.id", " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" ], "Warning": [ @@ -3413,7 +3413,7 @@ "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", " └─Projection 1.00 batchCop[tiflash] Column#5, Column#6, Column#7, div(Column#8, cast(case(eq(Column#10, 0), 1, Column#10), decimal(20,0) BINARY))->Column#8", " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(Column#11, Column#12, Column#13 order by Column#14 separator \",\")->Column#5, funcs:count(Column#15)->Column#6, funcs:min(Column#16)->Column#7, funcs:count(Column#17)->Column#10, funcs:sum(Column#18)->Column#8", - " └─Projection 10000.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#13, test.ts.col_0, test.ts.id, test.ts.col_0, test.ts.id, cast(test.ts.id, decimal(37,4) BINARY)->Column#18", + " └─Projection 10000.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#13, test.ts.col_0, test.ts.id, test.ts.col_0, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#18", " └─ExchangeReceiver 10000.00 batchCop[tiflash] ", " └─ExchangeSender 10000.00 batchCop[tiflash] ExchangeType: PassThrough", " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" @@ -3433,7 +3433,7 @@ " └─ExchangeReceiver 1.00 batchCop[tiflash] ", " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", " └─HashAgg 1.00 batchCop[tiflash] group by:Column#27, Column#28, Column#29, funcs:count(Column#23)->Column#19, funcs:max(Column#24)->Column#20, funcs:count(Column#25)->Column#21, funcs:sum(Column#26)->Column#22", - " └─Projection 10000.00 batchCop[tiflash] test.ts.id, test.ts.col_1, test.ts.id, cast(test.ts.id, decimal(37,4) BINARY)->Column#26, test.ts.col_0, test.ts.col_1, test.ts.id", + " └─Projection 10000.00 batchCop[tiflash] test.ts.id, test.ts.col_1, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#26, test.ts.col_0, test.ts.col_1, test.ts.id", " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" ], "Warning": [ @@ -3450,7 +3450,7 @@ " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_2, collate: N/A]", " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#40, funcs:group_concat(Column#33, Column#34, Column#35 separator \",\")->Column#28, funcs:count(Column#36)->Column#29, funcs:min(Column#37)->Column#30, funcs:count(Column#38)->Column#31, funcs:sum(Column#39)->Column#32", - " └─Projection 10000.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#35, test.ts.id, test.ts.col_0, test.ts.id, cast(test.ts.id, decimal(37,4) BINARY)->Column#39, test.ts.col_2", + " └─Projection 10000.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#35, test.ts.id, test.ts.col_0, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#39, test.ts.col_2", " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" ], "Warning": [ @@ -3468,7 +3468,7 @@ " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_2, collate: N/A]", " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#33, Column#34, Column#35, Column#36, funcs:count(Column#29)->Column#25, funcs:max(Column#30)->Column#26, funcs:count(Column#31)->Column#27, funcs:sum(Column#32)->Column#28", - " └─Projection 10000.00 batchCop[tiflash] test.ts.id, test.ts.col_1, test.ts.id, cast(test.ts.id, decimal(37,4) BINARY)->Column#32, test.ts.col_2, test.ts.col_0, test.ts.col_1, test.ts.id", + " └─Projection 10000.00 batchCop[tiflash] test.ts.id, test.ts.col_1, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#32, test.ts.col_2, test.ts.col_0, test.ts.col_1, test.ts.id", " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" ], "Warning": [ @@ -3482,7 +3482,7 @@ "└─TableReader 1.00 root data:ExchangeSender", " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(Column#24, Column#25, Column#26 separator \",\")->Column#14, funcs:count(Column#27)->Column#15, funcs:min(Column#28)->Column#16, funcs:count(Column#29)->Column#17, funcs:sum(Column#30)->Column#18", - " └─Projection 10000.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#26, test.ts.id, test.ts.col_0, test.ts.id, cast(test.ts.id, decimal(37,4) BINARY)->Column#30", + " └─Projection 10000.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#26, test.ts.id, test.ts.col_0, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#30", " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" ], "Warning": [ @@ -3500,7 +3500,7 @@ " └─ExchangeReceiver 1.00 batchCop[tiflash] ", " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", " └─HashAgg 1.00 batchCop[tiflash] group by:Column#27, Column#28, Column#29, funcs:count(Column#23)->Column#19, funcs:max(Column#24)->Column#20, funcs:count(Column#25)->Column#21, funcs:sum(Column#26)->Column#22", - " └─Projection 10000.00 batchCop[tiflash] test.ts.id, test.ts.col_1, test.ts.id, cast(test.ts.id, decimal(37,4) BINARY)->Column#26, test.ts.col_0, test.ts.col_1, test.ts.id", + " └─Projection 10000.00 batchCop[tiflash] test.ts.id, test.ts.col_1, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#26, test.ts.col_0, test.ts.col_1, test.ts.id", " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" ], "Warning": [ @@ -3514,7 +3514,7 @@ "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", " └─Projection 8000.00 batchCop[tiflash] Column#5, Column#6, Column#7, div(Column#8, cast(case(eq(Column#17, 0), 1, Column#17), decimal(20,0) BINARY))->Column#8", " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#29, funcs:group_concat(Column#21, Column#22, Column#23 separator \",\")->Column#5, funcs:count(Column#24)->Column#6, funcs:group_concat(Column#25 order by Column#26 separator \",\")->Column#7, funcs:count(Column#27)->Column#17, funcs:sum(Column#28)->Column#8", - " └─Projection 10000.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#23, test.ts.id, test.ts.col_0, test.ts.col_0, test.ts.id, cast(test.ts.id, decimal(37,4) BINARY)->Column#28, test.ts.col_2", + " └─Projection 10000.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#23, test.ts.id, test.ts.col_0, test.ts.col_0, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#28, test.ts.col_2", " └─ExchangeReceiver 10000.00 batchCop[tiflash] ", " └─ExchangeSender 10000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_2, collate: N/A]", " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" @@ -3530,7 +3530,7 @@ "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", " └─Projection 8000.00 batchCop[tiflash] Column#5, Column#6, Column#7, div(Column#8, cast(case(eq(Column#14, 0), 1, Column#14), decimal(20,0) BINARY))->Column#8", " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#25, funcs:group_concat(distinct Column#17 separator \",\")->Column#5, funcs:count(Column#18)->Column#6, funcs:group_concat(Column#19, Column#20 order by Column#21, Column#22 separator \",\")->Column#7, funcs:count(Column#23)->Column#14, funcs:sum(Column#24)->Column#8", - " └─Projection 10000.00 batchCop[tiflash] test.ts.col_0, test.ts.id, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#20, test.ts.col_1, test.ts.id, test.ts.id, cast(test.ts.id, decimal(37,4) BINARY)->Column#24, test.ts.col_2", + " └─Projection 10000.00 batchCop[tiflash] test.ts.col_0, test.ts.id, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#20, test.ts.col_1, test.ts.id, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#24, test.ts.col_2", " └─ExchangeReceiver 10000.00 batchCop[tiflash] ", " └─ExchangeSender 10000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_2, collate: N/A]", " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" @@ -3546,7 +3546,7 @@ "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", " └─Projection 1.00 batchCop[tiflash] Column#5, Column#6, Column#7, Column#8, div(Column#9, cast(case(eq(Column#15, 0), 1, Column#15), decimal(20,0) BINARY))->Column#9", " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(Column#18, Column#19 separator \",\")->Column#5, funcs:count(Column#20)->Column#6, funcs:group_concat(Column#21, Column#22 order by Column#23, Column#24 separator \",\")->Column#7, funcs:min(Column#25)->Column#8, funcs:count(Column#26)->Column#15, funcs:sum(Column#27)->Column#9", - " └─Projection 10000.00 batchCop[tiflash] test.ts.col_0, cast(test.ts.id, var_string(20))->Column#19, test.ts.id, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#22, test.ts.col_1, test.ts.id, test.ts.col_0, test.ts.id, cast(test.ts.id, decimal(37,4) BINARY)->Column#27", + " └─Projection 10000.00 batchCop[tiflash] test.ts.col_0, cast(test.ts.id, var_string(20))->Column#19, test.ts.id, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#22, test.ts.col_1, test.ts.id, test.ts.col_0, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#27", " └─ExchangeReceiver 10000.00 batchCop[tiflash] ", " └─ExchangeSender 10000.00 batchCop[tiflash] ExchangeType: PassThrough", " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" @@ -3562,7 +3562,7 @@ "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", " └─Projection 1.00 batchCop[tiflash] Column#5, Column#6, Column#7, Column#8, div(Column#9, cast(case(eq(Column#13, 0), 1, Column#13), decimal(20,0) BINARY))->Column#9", " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(distinct Column#15, Column#16, Column#17 separator \",\")->Column#5, funcs:count(Column#18)->Column#6, funcs:group_concat(Column#19, Column#20 order by Column#21, Column#22 separator \",\")->Column#7, funcs:max(Column#23)->Column#8, funcs:count(Column#24)->Column#13, funcs:sum(Column#25)->Column#9", - " └─Projection 10000.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#17, test.ts.id, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#20, test.ts.col_1, test.ts.id, test.ts.col_1, test.ts.id, cast(test.ts.id, decimal(37,4) BINARY)->Column#25", + " └─Projection 10000.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#17, test.ts.id, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#20, test.ts.col_1, test.ts.id, test.ts.col_1, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#25", " └─ExchangeReceiver 10000.00 batchCop[tiflash] ", " └─ExchangeSender 10000.00 batchCop[tiflash] ExchangeType: PassThrough", " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" @@ -3582,7 +3582,7 @@ " └─ExchangeReceiver 1.00 batchCop[tiflash] ", " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", " └─HashAgg 1.00 batchCop[tiflash] group by:Column#29, Column#30, Column#31, Column#32, funcs:group_concat(Column#24, Column#25 separator \",\")->Column#20, funcs:max(Column#26)->Column#21, funcs:count(Column#27)->Column#22, funcs:sum(Column#28)->Column#23", - " └─Projection 10000.00 batchCop[tiflash] test.ts.col_1, cast(test.ts.id, var_string(20))->Column#25, test.ts.col_1, test.ts.id, cast(test.ts.id, decimal(37,4) BINARY)->Column#28, test.ts.col_0, test.ts.col_1, test.ts.id, test.ts.col_2", + " └─Projection 10000.00 batchCop[tiflash] test.ts.col_1, cast(test.ts.id, var_string(20))->Column#25, test.ts.col_1, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#28, test.ts.col_0, test.ts.col_1, test.ts.id, test.ts.col_2", " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" ], "Warning": [ @@ -3600,7 +3600,7 @@ " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_0, collate: N/A]", " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#35, Column#36, Column#37, Column#38, funcs:group_concat(Column#30, Column#31 separator \",\")->Column#26, funcs:max(Column#32)->Column#27, funcs:count(Column#33)->Column#28, funcs:sum(Column#34)->Column#29", - " └─Projection 10000.00 batchCop[tiflash] test.ts.col_1, cast(test.ts.id, var_string(20))->Column#31, test.ts.col_1, test.ts.id, cast(test.ts.id, decimal(37,4) BINARY)->Column#34, test.ts.col_0, test.ts.col_1, test.ts.id, test.ts.col_2", + " └─Projection 10000.00 batchCop[tiflash] test.ts.col_1, cast(test.ts.id, var_string(20))->Column#31, test.ts.col_1, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#34, test.ts.col_0, test.ts.col_1, test.ts.id, test.ts.col_2", " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" ], "Warning": [ diff --git a/planner/core/testdata/ordered_result_mode_suite_out.json b/planner/core/testdata/ordered_result_mode_suite_out.json index 80d8b06a86fd6..8016c33bed6f4 100644 --- a/planner/core/testdata/ordered_result_mode_suite_out.json +++ b/planner/core/testdata/ordered_result_mode_suite_out.json @@ -401,7 +401,7 @@ "Plan": [ "Projection_8 10000.00 root Column#6", "└─Sort_9 10000.00 root test.t1.b, test.t1.a, Column#6", - " └─Window_11 10000.00 root sum(cast(test.t1.b, decimal(32,0) BINARY))->Column#6 over(partition by test.t1.a)", + " └─Window_11 10000.00 root sum(cast(test.t1.b, decimal(10,0) BINARY))->Column#6 over(partition by test.t1.a)", " └─TableReader_13 10000.00 root data:TableFullScan_12", " └─TableFullScan_12 10000.00 cop[tikv] table:t1 keep order:true, stats:pseudo" ] diff --git a/planner/core/testdata/plan_suite_out.json b/planner/core/testdata/plan_suite_out.json index 75db6cb736e99..f70d1eb7e5550 100644 --- a/planner/core/testdata/plan_suite_out.json +++ b/planner/core/testdata/plan_suite_out.json @@ -1705,7 +1705,7 @@ "SQL": "select /*+ HASH_AGG(), AGG_TO_COP() */ sum(distinct b) from pt;", "Plan": [ "HashAgg 1.00 root funcs:sum(distinct Column#9)->Column#4", - "└─Projection 16000.00 root cast(test.pt.b, decimal(32,0) BINARY)->Column#9", + "└─Projection 16000.00 root cast(test.pt.b, decimal(10,0) BINARY)->Column#9", " └─PartitionUnion 16000.00 root ", " ├─HashAgg 8000.00 root group by:test.pt.b, funcs:firstrow(test.pt.b)->test.pt.b, funcs:firstrow(test.pt.b)->test.pt.b", " │ └─TableReader 8000.00 root data:HashAgg", @@ -1788,7 +1788,7 @@ "Plan": [ "Projection 8000.00 root Column#5, test.t.c, Column#5, Column#6, Column#7, Column#8, Column#9", "└─HashAgg 8000.00 root group by:Column#17, funcs:avg(Column#10)->Column#5, funcs:count(distinct Column#11, Column#12)->Column#6, funcs:count(distinct Column#13)->Column#7, funcs:count(distinct Column#14)->Column#8, funcs:sum(Column#15)->Column#9, funcs:firstrow(Column#16)->test.t.c", - " └─Projection 10000.00 root cast(test.t.b, decimal(15,4) BINARY)->Column#10, test.t.a, test.t.b, test.t.a, test.t.c, cast(test.t.b, decimal(32,0) BINARY)->Column#15, test.t.c, test.t.c", + " └─Projection 10000.00 root cast(test.t.b, decimal(15,4) BINARY)->Column#10, test.t.a, test.t.b, test.t.a, test.t.c, cast(test.t.b, decimal(10,0) BINARY)->Column#15, test.t.c, test.t.c", " └─TableReader 10000.00 root data:TableFullScan", " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" ], @@ -1850,7 +1850,7 @@ "SQL": "select /*+ HASH_AGG(), AGG_TO_COP() */ sum(distinct b) from pt;", "Plan": [ "HashAgg 1.00 root funcs:sum(distinct Column#9)->Column#4", - "└─Projection 16000.00 root cast(test.pt.b, decimal(32,0) BINARY)->Column#9", + "└─Projection 16000.00 root cast(test.pt.b, decimal(10,0) BINARY)->Column#9", " └─PartitionUnion 16000.00 root ", " ├─HashAgg 8000.00 root group by:test.pt.b, funcs:firstrow(test.pt.b)->test.pt.b, funcs:firstrow(test.pt.b)->test.pt.b", " │ └─TableReader 8000.00 root data:HashAgg", @@ -1892,7 +1892,7 @@ "SQL": "select /*+ HASH_AGG(), AGG_TO_COP() */ sum(distinct b) from pt;", "Plan": [ "HashAgg 1.00 root funcs:sum(distinct Column#9)->Column#4", - "└─Projection 16000.00 root cast(test.pt.b, decimal(32,0) BINARY)->Column#9", + "└─Projection 16000.00 root cast(test.pt.b, decimal(10,0) BINARY)->Column#9", " └─PartitionUnion 16000.00 root ", " ├─HashAgg 8000.00 root group by:test.pt.b, funcs:firstrow(test.pt.b)->test.pt.b, funcs:firstrow(test.pt.b)->test.pt.b", " │ └─TableReader 8000.00 root data:HashAgg", diff --git a/planner/core/testdata/plan_suite_unexported_out.json b/planner/core/testdata/plan_suite_unexported_out.json index 6401df6aa9264..391797fa59644 100644 --- a/planner/core/testdata/plan_suite_unexported_out.json +++ b/planner/core/testdata/plan_suite_unexported_out.json @@ -93,7 +93,7 @@ "Join{DataScan(t1)->DataScan(t2)}(test.t.a,test.t.a)->Projection", "Dual->Projection", "DataScan(t)->Projection->Projection->Window(min(test.t.a)->Column#14)->Sel([lt(test.t.a, 10) eq(test.t.b, Column#14)])->Projection->Projection", - "DataScan(t)->Projection->Projection->Window(sum(cast(test.t.a, decimal(65,0) BINARY))->Column#14)->Sel([gt(Column#14, cast(test.t.b, decimal(20,0) BINARY))])->Projection->Projection" + "DataScan(t)->Projection->Projection->Window(sum(cast(test.t.a, decimal(10,0) BINARY))->Column#14)->Sel([gt(Column#14, cast(test.t.b, decimal(20,0) BINARY))])->Projection->Projection" ] }, { @@ -189,12 +189,12 @@ "TableReader(Table(t))->Sort->Window(avg(cast(test.t.a, decimal(15,4) BINARY))->Column#14 over(order by test.t.a, test.t.b desc range between unbounded preceding and current row))->Projection", "TableReader(Table(t))->Window(avg(cast(test.t.a, decimal(15,4) BINARY))->Column#14 over(partition by test.t.a))->Projection", "[planner:1054]Unknown column 'z' in 'field list'", - "TableReader(Table(t))->Window(sum(cast(test.t.b, decimal(65,0) BINARY))->Column#14 over())->Sort->Projection", + "TableReader(Table(t))->Window(sum(cast(test.t.b, decimal(10,0) BINARY))->Column#14 over())->Sort->Projection", "IndexReader(Index(t.f)[[NULL,+inf]]->StreamAgg)->StreamAgg->Window(sum(Column#13)->Column#15 over())->Projection", - "TableReader(Table(t))->Window(sum(cast(test.t.a, decimal(65,0) BINARY))->Column#14 over())->Sort->Projection", - "TableReader(Table(t))->Window(sum(cast(test.t.a, decimal(65,0) BINARY))->Column#14 over(partition by test.t.a))->Sort->Projection", + "TableReader(Table(t))->Window(sum(cast(test.t.a, decimal(10,0) BINARY))->Column#14 over())->Sort->Projection", + "TableReader(Table(t))->Window(sum(cast(test.t.a, decimal(10,0) BINARY))->Column#14 over(partition by test.t.a))->Sort->Projection", "TableReader(Table(t)->StreamAgg)->StreamAgg->Window(sum(Column#13)->Column#15 over())->Sort->Projection", - "Apply{IndexReader(Index(t.f)[[NULL,+inf]])->IndexReader(Index(t.f)[[NULL,+inf]]->Sel([gt(test.t.a, test.t.a)]))->Window(sum(cast(test.t.a, decimal(65,0) BINARY))->Column#38 over())->MaxOneRow}->Sel([Column#38])->Projection", + "Apply{IndexReader(Index(t.f)[[NULL,+inf]])->IndexReader(Index(t.f)[[NULL,+inf]]->Sel([gt(test.t.a, test.t.a)]))->Window(sum(cast(test.t.a, decimal(10,0) BINARY))->Column#38 over())->MaxOneRow}->Sel([Column#38])->Projection", "[planner:3594]You cannot use the alias 'w' of an expression containing a window function in this context.'", "[planner:1247]Reference 'sum_a' not supported (reference to window function)", "[planner:3579]Window name 'w2' is not defined.", @@ -203,11 +203,11 @@ "[planner:3581]A window which depends on another cannot define partitioning.", "[planner:3581]A window which depends on another cannot define partitioning.", "[planner:3582]Window 'w' has a frame definition, so cannot be referenced by another window.", - "IndexReader(Index(t.f)[[NULL,+inf]])->Window(sum(cast(test.t.a, decimal(65,0) BINARY))->Column#14 over(rows between 1 preceding and 1 following))->Projection", + "IndexReader(Index(t.f)[[NULL,+inf]])->Window(sum(cast(test.t.a, decimal(10,0) BINARY))->Column#14 over(rows between 1 preceding and 1 following))->Projection", "[planner:3583]Window '' cannot inherit 'w' since both contain an ORDER BY clause.", "[planner:3591]Window 'w1' is defined twice.", "TableReader(Table(t))->Window(avg(cast(test.t.a, decimal(15,4) BINARY))->Column#14 over(partition by test.t.a))->Projection", - "TableReader(Table(t))->Window(sum(cast(test.t.a, decimal(65,0) BINARY))->Column#14 over(partition by test.t.a))->Sort->Projection", + "TableReader(Table(t))->Window(sum(cast(test.t.a, decimal(10,0) BINARY))->Column#14 over(partition by test.t.a))->Sort->Projection", "[planner:1235]This version of TiDB doesn't yet support 'GROUPS'", "[planner:3584]Window '': frame start cannot be UNBOUNDED FOLLOWING.", "[planner:3585]Window '': frame end cannot be UNBOUNDED PRECEDING.", @@ -219,7 +219,7 @@ "[planner:3590]Window '' has a non-constant frame bound.", "[planner:3586]Window '': frame start or end is negative, NULL or of non-integral type", "[planner:3588]Window '' with RANGE frame has ORDER BY expression of datetime type. Only INTERVAL bound value allowed.", - "TableReader(Table(t))->Window(sum(cast(test.t.a, decimal(65,0) BINARY))->Column#14 over(order by test.t.a range between 1.0 preceding and 1 following))->Projection", + "TableReader(Table(t))->Window(sum(cast(test.t.a, decimal(10,0) BINARY))->Column#14 over(order by test.t.a range between 1.0 preceding and 1 following))->Projection", "IndexReader(Index(t.f)[[NULL,+inf]])->Window(row_number()->Column#14 over())->Projection", "TableReader(Table(t))->HashAgg->Window(max(Column#13)->Column#15 over(rows between 1 preceding and 1 following))->Projection", "[planner:1210]Incorrect arguments to nth_value", @@ -229,11 +229,11 @@ "IndexReader(Index(t.f)[[NULL,+inf]])->Window(ntile()->Column#14 over())->Projection", "TableReader(Table(t))->Sort->Window(avg(cast(test.t.a, decimal(15,4) BINARY))->Column#14 over(partition by test.t.b))->Projection", "TableReader(Table(t))->Window(nth_value(test.t.i_date, 1)->Column#14 over())->Projection", - "TableReader(Table(t))->Window(sum(cast(test.t.b, decimal(65,0) BINARY))->Column#15, sum(cast(test.t.c, decimal(65,0) BINARY))->Column#16 over(order by test.t.a range between unbounded preceding and current row))->Projection", + "TableReader(Table(t))->Window(sum(cast(test.t.b, decimal(10,0) BINARY))->Column#15, sum(cast(test.t.c, decimal(10,0) BINARY))->Column#16 over(order by test.t.a range between unbounded preceding and current row))->Projection", "[planner:3593]You cannot use the window function 'sum' in this context.'", "[planner:3593]You cannot use the window function 'sum' in this context.'", "[planner:3593]You cannot use the window function 'row_number' in this context.'", - "TableReader(Table(t))->Sort->Window(sum(cast(test.t.c, decimal(65,0) BINARY))->Column#17 over(partition by test.t.a order by test.t.c range between unbounded preceding and current row))->Sort->Window(sum(cast(test.t.b, decimal(65,0) BINARY))->Column#18 over(order by test.t.a, test.t.b, test.t.c range between unbounded preceding and current row))->Window(sum(cast(test.t.a, decimal(65,0) BINARY))->Column#19 over(partition by test.t.a order by test.t.b range between unbounded preceding and current row))->Window(sum(cast(test.t.d, decimal(65,0) BINARY))->Column#20 over())->Projection", + "TableReader(Table(t))->Sort->Window(sum(cast(test.t.c, decimal(10,0) BINARY))->Column#17 over(partition by test.t.a order by test.t.c range between unbounded preceding and current row))->Sort->Window(sum(cast(test.t.b, decimal(10,0) BINARY))->Column#18 over(order by test.t.a, test.t.b, test.t.c range between unbounded preceding and current row))->Window(sum(cast(test.t.a, decimal(10,0) BINARY))->Column#19 over(partition by test.t.a order by test.t.b range between unbounded preceding and current row))->Window(sum(cast(test.t.d, decimal(10,0) BINARY))->Column#20 over())->Projection", "[planner:3587]Window 'w1' with RANGE N PRECEDING/FOLLOWING frame requires exactly one ORDER BY expression, of numeric or temporal type", "TableReader(Table(t))->Sort->Window(dense_rank()->Column#14 over(partition by test.t.b order by test.t.a desc, test.t.b desc))->Projection", "[planner:3587]Window 'w1' with RANGE N PRECEDING/FOLLOWING frame requires exactly one ORDER BY expression, of numeric or temporal type", @@ -262,12 +262,12 @@ "TableReader(Table(t))->Sort->Window(avg(cast(test.t.a, decimal(15,4) BINARY))->Column#14 over(order by test.t.a, test.t.b desc range between unbounded preceding and current row))->Projection", "TableReader(Table(t))->Window(avg(cast(test.t.a, decimal(15,4) BINARY))->Column#14 over(partition by test.t.a))->Projection", "[planner:1054]Unknown column 'z' in 'field list'", - "TableReader(Table(t))->Window(sum(cast(test.t.b, decimal(65,0) BINARY))->Column#14 over())->Sort->Projection", + "TableReader(Table(t))->Window(sum(cast(test.t.b, decimal(10,0) BINARY))->Column#14 over())->Sort->Projection", "IndexReader(Index(t.f)[[NULL,+inf]]->StreamAgg)->StreamAgg->Window(sum(Column#13)->Column#15 over())->Projection", - "TableReader(Table(t))->Window(sum(cast(test.t.a, decimal(65,0) BINARY))->Column#14 over())->Sort->Projection", - "TableReader(Table(t))->Window(sum(cast(test.t.a, decimal(65,0) BINARY))->Column#14 over(partition by test.t.a))->Sort->Projection", + "TableReader(Table(t))->Window(sum(cast(test.t.a, decimal(10,0) BINARY))->Column#14 over())->Sort->Projection", + "TableReader(Table(t))->Window(sum(cast(test.t.a, decimal(10,0) BINARY))->Column#14 over(partition by test.t.a))->Sort->Projection", "TableReader(Table(t)->StreamAgg)->StreamAgg->Window(sum(Column#13)->Column#15 over())->Sort->Projection", - "Apply{IndexReader(Index(t.f)[[NULL,+inf]])->IndexReader(Index(t.f)[[NULL,+inf]]->Sel([gt(test.t.a, test.t.a)]))->Window(sum(cast(test.t.a, decimal(65,0) BINARY))->Column#38 over())->MaxOneRow}->Sel([Column#38])->Projection", + "Apply{IndexReader(Index(t.f)[[NULL,+inf]])->IndexReader(Index(t.f)[[NULL,+inf]]->Sel([gt(test.t.a, test.t.a)]))->Window(sum(cast(test.t.a, decimal(10,0) BINARY))->Column#38 over())->MaxOneRow}->Sel([Column#38])->Projection", "[planner:3594]You cannot use the alias 'w' of an expression containing a window function in this context.'", "[planner:1247]Reference 'sum_a' not supported (reference to window function)", "[planner:3579]Window name 'w2' is not defined.", @@ -276,11 +276,11 @@ "[planner:3581]A window which depends on another cannot define partitioning.", "[planner:3581]A window which depends on another cannot define partitioning.", "[planner:3582]Window 'w' has a frame definition, so cannot be referenced by another window.", - "IndexReader(Index(t.f)[[NULL,+inf]])->Window(sum(cast(test.t.a, decimal(65,0) BINARY))->Column#14 over(rows between 1 preceding and 1 following))->Projection", + "IndexReader(Index(t.f)[[NULL,+inf]])->Window(sum(cast(test.t.a, decimal(10,0) BINARY))->Column#14 over(rows between 1 preceding and 1 following))->Projection", "[planner:3583]Window '' cannot inherit 'w' since both contain an ORDER BY clause.", "[planner:3591]Window 'w1' is defined twice.", "TableReader(Table(t))->Window(avg(cast(test.t.a, decimal(15,4) BINARY))->Column#14 over(partition by test.t.a))->Projection", - "TableReader(Table(t))->Window(sum(cast(test.t.a, decimal(65,0) BINARY))->Column#14 over(partition by test.t.a))->Sort->Projection", + "TableReader(Table(t))->Window(sum(cast(test.t.a, decimal(10,0) BINARY))->Column#14 over(partition by test.t.a))->Sort->Projection", "[planner:1235]This version of TiDB doesn't yet support 'GROUPS'", "[planner:3584]Window '': frame start cannot be UNBOUNDED FOLLOWING.", "[planner:3585]Window '': frame end cannot be UNBOUNDED PRECEDING.", @@ -292,7 +292,7 @@ "[planner:3590]Window '' has a non-constant frame bound.", "[planner:3586]Window '': frame start or end is negative, NULL or of non-integral type", "[planner:3588]Window '' with RANGE frame has ORDER BY expression of datetime type. Only INTERVAL bound value allowed.", - "TableReader(Table(t))->Window(sum(cast(test.t.a, decimal(65,0) BINARY))->Column#14 over(order by test.t.a range between 1.0 preceding and 1 following))->Projection", + "TableReader(Table(t))->Window(sum(cast(test.t.a, decimal(10,0) BINARY))->Column#14 over(order by test.t.a range between 1.0 preceding and 1 following))->Projection", "IndexReader(Index(t.f)[[NULL,+inf]])->Window(row_number()->Column#14 over())->Projection", "TableReader(Table(t))->HashAgg->Window(max(Column#13)->Column#15 over(rows between 1 preceding and 1 following))->Projection", "[planner:1210]Incorrect arguments to nth_value", @@ -302,11 +302,11 @@ "IndexReader(Index(t.f)[[NULL,+inf]])->Window(ntile()->Column#14 over())->Projection", "TableReader(Table(t))->Sort->Window(avg(cast(test.t.a, decimal(15,4) BINARY))->Column#14 over(partition by test.t.b))->Partition(execution info: concurrency:4, data sources:[TableReader_10])->Projection", "TableReader(Table(t))->Window(nth_value(test.t.i_date, 1)->Column#14 over())->Projection", - "TableReader(Table(t))->Window(sum(cast(test.t.b, decimal(65,0) BINARY))->Column#15, sum(cast(test.t.c, decimal(65,0) BINARY))->Column#16 over(order by test.t.a range between unbounded preceding and current row))->Projection", + "TableReader(Table(t))->Window(sum(cast(test.t.b, decimal(10,0) BINARY))->Column#15, sum(cast(test.t.c, decimal(10,0) BINARY))->Column#16 over(order by test.t.a range between unbounded preceding and current row))->Projection", "[planner:3593]You cannot use the window function 'sum' in this context.'", "[planner:3593]You cannot use the window function 'sum' in this context.'", "[planner:3593]You cannot use the window function 'row_number' in this context.'", - "TableReader(Table(t))->Sort->Window(sum(cast(test.t.c, decimal(65,0) BINARY))->Column#17 over(partition by test.t.a order by test.t.c range between unbounded preceding and current row))->Sort->Window(sum(cast(test.t.b, decimal(65,0) BINARY))->Column#18 over(order by test.t.a, test.t.b, test.t.c range between unbounded preceding and current row))->Window(sum(cast(test.t.a, decimal(65,0) BINARY))->Column#19 over(partition by test.t.a order by test.t.b range between unbounded preceding and current row))->Window(sum(cast(test.t.d, decimal(65,0) BINARY))->Column#20 over())->Projection", + "TableReader(Table(t))->Sort->Window(sum(cast(test.t.c, decimal(10,0) BINARY))->Column#17 over(partition by test.t.a order by test.t.c range between unbounded preceding and current row))->Sort->Window(sum(cast(test.t.b, decimal(10,0) BINARY))->Column#18 over(order by test.t.a, test.t.b, test.t.c range between unbounded preceding and current row))->Window(sum(cast(test.t.a, decimal(10,0) BINARY))->Column#19 over(partition by test.t.a order by test.t.b range between unbounded preceding and current row))->Window(sum(cast(test.t.d, decimal(10,0) BINARY))->Column#20 over())->Projection", "[planner:3587]Window 'w1' with RANGE N PRECEDING/FOLLOWING frame requires exactly one ORDER BY expression, of numeric or temporal type", "TableReader(Table(t))->Sort->Window(dense_rank()->Column#14 over(partition by test.t.b order by test.t.a desc, test.t.b desc))->Partition(execution info: concurrency:4, data sources:[TableReader_9])->Projection", "[planner:3587]Window 'w1' with RANGE N PRECEDING/FOLLOWING frame requires exactly one ORDER BY expression, of numeric or temporal type", diff --git a/planner/core/testdata/stats_suite_out.json b/planner/core/testdata/stats_suite_out.json index f7e40ddcc0e61..369fbe4fb0adf 100644 --- a/planner/core/testdata/stats_suite_out.json +++ b/planner/core/testdata/stats_suite_out.json @@ -275,7 +275,7 @@ "SQL": "select count(tmp.a_sum) from (select t1.a as a, t1.b as b, sum(a) over() as a_sum from t1) tmp group by tmp.a, tmp.b", "Plan": [ "HashAgg 4.00 root group by:test.t1.a, test.t1.b, funcs:count(Column#5)->Column#6", - "└─Window 4.00 root sum(cast(test.t1.a, decimal(32,0) BINARY))->Column#5 over()", + "└─Window 4.00 root sum(cast(test.t1.a, decimal(10,0) BINARY))->Column#5 over()", " └─TableReader 4.00 root data:TableFullScan", " └─TableFullScan 4.00 cop[tikv] table:t1 keep order:false" ] From dfaabf4d31b87a020513ec25630d2902fcd747a9 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Thu, 23 Dec 2021 17:11:49 +0800 Subject: [PATCH 33/44] *: fix data race in the tikv_client (#30964) close pingcap/tidb#30658 --- go.mod | 2 +- go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/go.mod b/go.mod index f11eab1cf0456..6a60bdbeb67df 100644 --- a/go.mod +++ b/go.mod @@ -65,7 +65,7 @@ require ( github.com/spf13/pflag v1.0.5 github.com/stretchr/testify v1.7.0 github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2 - github.com/tikv/client-go/v2 v2.0.0-rc.0.20211221041211-e9de5625c45c + github.com/tikv/client-go/v2 v2.0.0-rc.0.20211223062159-300275dee63e github.com/tikv/pd v1.1.0-beta.0.20211118054146-02848d2660ee github.com/twmb/murmur3 v1.1.3 github.com/uber/jaeger-client-go v2.22.1+incompatible diff --git a/go.sum b/go.sum index 2a4c23d9d70bc..087c367fd7482 100644 --- a/go.sum +++ b/go.sum @@ -712,8 +712,8 @@ github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2/go.mod h1:2PfK github.com/tidwall/gjson v1.3.5/go.mod h1:P256ACg0Mn+j1RXIDXoss50DeIABTYK1PULOJHhxOls= github.com/tidwall/match v1.0.1/go.mod h1:LujAq0jyVjBy028G1WhWfIzbpQfMO8bBZ6Tyb0+pL9E= github.com/tidwall/pretty v1.0.0/go.mod h1:XNkn88O1ChpSDQmQeStsy+sBenx6DDtFZJxhVysOjyk= -github.com/tikv/client-go/v2 v2.0.0-rc.0.20211221041211-e9de5625c45c h1:1P6iN1csRSZNHXuaylArmG3/bA5MpYVzc9ZkdHK/L2Y= -github.com/tikv/client-go/v2 v2.0.0-rc.0.20211221041211-e9de5625c45c/go.mod h1:wRuh+W35daKTiYBld0oBlT6PSkzEVr+pB/vChzJZk+8= +github.com/tikv/client-go/v2 v2.0.0-rc.0.20211223062159-300275dee63e h1:UildvukO7gTs4/bW+h6jNnpv6syWmh2VMQxD5sMm9II= +github.com/tikv/client-go/v2 v2.0.0-rc.0.20211223062159-300275dee63e/go.mod h1:wRuh+W35daKTiYBld0oBlT6PSkzEVr+pB/vChzJZk+8= github.com/tikv/pd v1.1.0-beta.0.20211029083450-e65f0c55b6ae/go.mod h1:varH0IE0jJ9E9WN2Ei/N6pajMlPkcXdDEf7f5mmsUVQ= github.com/tikv/pd v1.1.0-beta.0.20211118054146-02848d2660ee h1:rAAdvQ8Hh36syHr92g0VmZEpkH+40RGQBpFL2121xMs= github.com/tikv/pd v1.1.0-beta.0.20211118054146-02848d2660ee/go.mod h1:lRbwxBAhnTQR5vqbTzeI/Bj62bD2OvYYuFezo2vrmeI= From 586eed998a8c8ff8bfef3ede5e803c30bba2e69c Mon Sep 17 00:00:00 2001 From: tuuuuuu <83738345+MiaoMiaoGarden@users.noreply.github.com> Date: Thu, 23 Dec 2021 18:39:47 +0800 Subject: [PATCH 34/44] ddl: migrate test-infra to testify for ddl/db_partition_test.go (#30952) close pingcap/tidb#28635 --- ddl/partition_test.go | 62 +++++++++++++++++-------------------------- 1 file changed, 24 insertions(+), 38 deletions(-) diff --git a/ddl/partition_test.go b/ddl/partition_test.go index f2cdd1c4e3597..0b993a7de0a8c 100644 --- a/ddl/partition_test.go +++ b/ddl/partition_test.go @@ -16,55 +16,41 @@ package ddl import ( "context" + "testing" - . "github.com/pingcap/check" - "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/types" + "github.com/stretchr/testify/require" ) -var _ = SerialSuites(&testPartitionSuite{}) - -type testPartitionSuite struct { - store kv.Storage -} - -func (s *testPartitionSuite) SetUpSuite(c *C) { - s.store = testCreateStore(c, "test_store") -} - -func (s *testPartitionSuite) TearDownSuite(c *C) { - err := s.store.Close() - c.Assert(err, IsNil) -} - -func (s *testPartitionSuite) TestDropAndTruncatePartition(c *C) { +func TestDropAndTruncatePartition(t *testing.T) { + store := testCreateStoreT(t, "test_store") d, err := testNewDDLAndStart( context.Background(), - WithStore(s.store), + WithStore(store), WithLease(testLease), ) - c.Assert(err, IsNil) + require.NoError(t, err) defer func() { err := d.Stop() - c.Assert(err, IsNil) + require.NoError(t, err) }() dbInfo, err := testSchemaInfo(d, "test_partition") - c.Assert(err, IsNil) - testCreateSchema(c, testNewContext(d), d, dbInfo) + require.NoError(t, err) + testCreateSchemaT(t, testNewContext(d), d, dbInfo) // generate 5 partition in tableInfo. - tblInfo, partIDs := buildTableInfoWithPartition(c, d) + tblInfo, partIDs := buildTableInfoWithPartition(t, d) ctx := testNewContext(d) - testCreateTable(c, ctx, d, dbInfo, tblInfo) + testCreateTableT(t, ctx, d, dbInfo, tblInfo) - testDropPartition(c, ctx, d, dbInfo, tblInfo, []string{"p0", "p1"}) + testDropPartition(t, ctx, d, dbInfo, tblInfo, []string{"p0", "p1"}) - testTruncatePartition(c, ctx, d, dbInfo, tblInfo, []int64{partIDs[3], partIDs[4]}) + testTruncatePartition(t, ctx, d, dbInfo, tblInfo, []int64{partIDs[3], partIDs[4]}) } -func buildTableInfoWithPartition(c *C, d *ddl) (*model.TableInfo, []int64) { +func buildTableInfoWithPartition(t *testing.T, d *ddl) (*model.TableInfo, []int64) { tbl := &model.TableInfo{ Name: model.NewCIStr("t"), } @@ -76,14 +62,14 @@ func buildTableInfoWithPartition(c *C, d *ddl) (*model.TableInfo, []int64) { ID: allocateColumnID(tbl), } genIDs, err := d.genGlobalIDs(1) - c.Assert(err, IsNil) + require.NoError(t, err) tbl.ID = genIDs[0] tbl.Columns = []*model.ColumnInfo{col} tbl.Charset = "utf8" tbl.Collate = "utf8_bin" partIDs, err := d.genGlobalIDs(5) - c.Assert(err, IsNil) + require.NoError(t, err) partInfo := &model.PartitionInfo{ Type: model.PartitionTypeRange, Expr: tbl.Columns[0].Name.L, @@ -130,12 +116,12 @@ func buildDropPartitionJob(dbInfo *model.DBInfo, tblInfo *model.TableInfo, partN } } -func testDropPartition(c *C, ctx sessionctx.Context, d *ddl, dbInfo *model.DBInfo, tblInfo *model.TableInfo, partNames []string) *model.Job { +func testDropPartition(t *testing.T, ctx sessionctx.Context, d *ddl, dbInfo *model.DBInfo, tblInfo *model.TableInfo, partNames []string) *model.Job { job := buildDropPartitionJob(dbInfo, tblInfo, partNames) err := d.doDDLJob(ctx, job) - c.Assert(err, IsNil) - v := getSchemaVer(c, ctx) - checkHistoryJobArgs(c, ctx, job.ID, &historyJobArgs{ver: v, tbl: tblInfo}) + require.NoError(t, err) + v := getSchemaVerT(t, ctx) + checkHistoryJobArgsT(t, ctx, job.ID, &historyJobArgs{ver: v, tbl: tblInfo}) return job } @@ -149,11 +135,11 @@ func buildTruncatePartitionJob(dbInfo *model.DBInfo, tblInfo *model.TableInfo, p } } -func testTruncatePartition(c *C, ctx sessionctx.Context, d *ddl, dbInfo *model.DBInfo, tblInfo *model.TableInfo, pids []int64) *model.Job { +func testTruncatePartition(t *testing.T, ctx sessionctx.Context, d *ddl, dbInfo *model.DBInfo, tblInfo *model.TableInfo, pids []int64) *model.Job { job := buildTruncatePartitionJob(dbInfo, tblInfo, pids) err := d.doDDLJob(ctx, job) - c.Assert(err, IsNil) - v := getSchemaVer(c, ctx) - checkHistoryJobArgs(c, ctx, job.ID, &historyJobArgs{ver: v, tbl: tblInfo}) + require.NoError(t, err) + v := getSchemaVerT(t, ctx) + checkHistoryJobArgsT(t, ctx, job.ID, &historyJobArgs{ver: v, tbl: tblInfo}) return job } From e24493f411e601fb6c5dfb178c25b01abe03c4be Mon Sep 17 00:00:00 2001 From: Zhou Kunqin <25057648+time-and-fate@users.noreply.github.com> Date: Thu, 23 Dec 2021 19:13:47 +0800 Subject: [PATCH 35/44] planner: fix `AccessPath.TableFilters` got modified unexpectedly (#30966) close pingcap/tidb#30965 --- planner/core/find_best_task.go | 12 ++++++------ planner/core/physical_plan_test.go | 23 +++++++++++++++++++++++ 2 files changed, 29 insertions(+), 6 deletions(-) diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index 2df5aa73eee34..5157ffb738b24 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -1442,15 +1442,15 @@ func (is *PhysicalIndexScan) addPushedDownSelection(copTask *copTask, p *DataSou } // SplitSelCondsWithVirtualColumn filter the select conditions which contain virtual column -func SplitSelCondsWithVirtualColumn(conds []expression.Expression) ([]expression.Expression, []expression.Expression) { - var filterConds []expression.Expression - for i := len(conds) - 1; i >= 0; i-- { +func SplitSelCondsWithVirtualColumn(conds []expression.Expression) (withoutVirt []expression.Expression, withVirt []expression.Expression) { + for i := range conds { if expression.ContainVirtualColumn(conds[i : i+1]) { - filterConds = append(filterConds, conds[i]) - conds = append(conds[:i], conds[i+1:]...) + withVirt = append(withVirt, conds[i]) + } else { + withoutVirt = append(withoutVirt, conds[i]) } } - return conds, filterConds + return withoutVirt, withVirt } func matchIndicesProp(idxCols []*expression.Column, colLens []int, propItems []property.SortItem) bool { diff --git a/planner/core/physical_plan_test.go b/planner/core/physical_plan_test.go index 1b3d13c02cc03..bc145e4a14ad7 100644 --- a/planner/core/physical_plan_test.go +++ b/planner/core/physical_plan_test.go @@ -2016,3 +2016,26 @@ func (s *testPlanSuite) TestIssue28316(c *C) { tk.MustQuery("explain format='brief' " + ts).Check(testkit.Rows(output[i].Plan...)) } } + +func (s *testPlanSuite) TestIssue30965(c *C) { + store, dom, err := newStoreWithBootstrap() + c.Assert(err, IsNil) + defer func() { + dom.Close() + store.Close() + }() + tk := testkit.NewTestKit(c, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t30965") + tk.MustExec("CREATE TABLE `t30965` ( `a` int(11) DEFAULT NULL, `b` int(11) DEFAULT NULL, `c` int(11) DEFAULT NULL, `d` int(11) GENERATED ALWAYS AS (`a` + 1) VIRTUAL, KEY `ib` (`b`));") + tk.MustExec("insert into t30965 (a,b,c) value(3,4,5);") + tk.MustQuery("select count(*) from t30965 where d = 2 and b = 4 and a = 3 and c = 5;").Check(testkit.Rows("0")) + tk.MustQuery("explain format = 'brief' select count(*) from t30965 where d = 2 and b = 4 and a = 3 and c = 5;").Check( + testkit.Rows( + "StreamAgg 1.00 root funcs:count(1)->Column#6", + "└─Selection 0.00 root eq(test.t30965.d, 2)", + " └─IndexLookUp 0.00 root ", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t30965, index:ib(b) range:[4,4], keep order:false, stats:pseudo", + " └─Selection(Probe) 0.00 cop[tikv] eq(test.t30965.a, 3), eq(test.t30965.c, 5)", + " └─TableRowIDScan 10.00 cop[tikv] table:t30965 keep order:false, stats:pseudo")) +} From 4dc385ce8b7a3b194462e76670f89483de5884b7 Mon Sep 17 00:00:00 2001 From: tangenta Date: Thu, 23 Dec 2021 20:01:47 +0800 Subject: [PATCH 36/44] test: merge serial tests in ddl, infoschema, session, store, table, telemetry and types (#30874) --- ddl/db_cache_serial_test.go | 97 ----- ddl/db_cache_test.go | 70 ++++ ...m_serial_test.go => ddl_algorithm_test.go} | 0 ...fail_db_serial_test.go => fail_db_test.go} | 0 .../{syncer_serial_test.go => syncer_test.go} | 0 ..._serial_test.go => cluster_tables_test.go} | 0 infoschema/perfschema/tables_serial_test.go | 159 -------- infoschema/perfschema/tables_test.go | 136 +++++++ .../{tables_serial_test.go => tables_test.go} | 0 ...strap_serial_test.go => bootstrap_test.go} | 0 session/clustered_index_serial_test.go | 341 ------------------ session/clustered_index_test.go | 318 ++++++++++++++++ ...test.go => index_usage_sync_lease_test.go} | 0 ..._serial_test.go => schema_amender_test.go} | 0 ...rial_test.go => batch_coprocessor_test.go} | 0 .../{config_serial_test.go => config_test.go} | 0 ...erial_test.go => snap_interceptor_test.go} | 0 ...l_fail_serial_test.go => sql_fail_test.go} | 0 .../{txn_serial_test.go => txn_test.go} | 0 ...orker_serial_test.go => gc_worker_test.go} | 0 table/tables/index_serial_test.go | 324 ----------------- table/tables/index_test.go | 295 ++++++++++++++- ...dow_serial_test.go => data_window_test.go} | 0 telemetry/telemetry_serial_test.go | 54 --- telemetry/telemetry_test.go | 26 ++ types/{enum_serial_test.go => enum_test.go} | 0 types/mydecimal_serial_test.go | 199 ---------- types/mydecimal_test.go | 178 +++++++++ types/{set_serial_test.go => set_test.go} | 0 29 files changed, 1022 insertions(+), 1175 deletions(-) delete mode 100644 ddl/db_cache_serial_test.go rename ddl/{ddl_algorithm_serial_test.go => ddl_algorithm_test.go} (100%) rename ddl/failtest/{fail_db_serial_test.go => fail_db_test.go} (100%) rename ddl/util/{syncer_serial_test.go => syncer_test.go} (100%) rename infoschema/{cluster_tables_serial_test.go => cluster_tables_test.go} (100%) delete mode 100644 infoschema/perfschema/tables_serial_test.go rename infoschema/{tables_serial_test.go => tables_test.go} (100%) rename session/{bootstrap_serial_test.go => bootstrap_test.go} (100%) delete mode 100644 session/clustered_index_serial_test.go rename session/{index_usage_sync_lease_serial_test.go => index_usage_sync_lease_test.go} (100%) rename session/{schema_amender_serial_test.go => schema_amender_test.go} (100%) rename store/{batch_coprocessor_serial_test.go => batch_coprocessor_test.go} (100%) rename store/driver/{config_serial_test.go => config_test.go} (100%) rename store/driver/{snap_interceptor_serial_test.go => snap_interceptor_test.go} (100%) rename store/driver/{sql_fail_serial_test.go => sql_fail_test.go} (100%) rename store/driver/{txn_serial_test.go => txn_test.go} (100%) rename store/gcworker/{gc_worker_serial_test.go => gc_worker_test.go} (100%) delete mode 100644 table/tables/index_serial_test.go rename telemetry/{data_window_serial_test.go => data_window_test.go} (100%) delete mode 100644 telemetry/telemetry_serial_test.go rename types/{enum_serial_test.go => enum_test.go} (100%) delete mode 100644 types/mydecimal_serial_test.go rename types/{set_serial_test.go => set_test.go} (100%) diff --git a/ddl/db_cache_serial_test.go b/ddl/db_cache_serial_test.go deleted file mode 100644 index e024cef522682..0000000000000 --- a/ddl/db_cache_serial_test.go +++ /dev/null @@ -1,97 +0,0 @@ -// Copyright 2021 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package ddl_test - -import ( - "testing" - "time" - - "github.com/pingcap/tidb/ddl" - "github.com/pingcap/tidb/domain" - "github.com/pingcap/tidb/errno" - "github.com/pingcap/tidb/parser/model" - "github.com/pingcap/tidb/parser/terror" - "github.com/pingcap/tidb/session" - "github.com/pingcap/tidb/store/mockstore" - "github.com/pingcap/tidb/testkit" - "github.com/stretchr/testify/require" -) - -func TestAlterTableCache(t *testing.T) { - store, err := mockstore.NewMockStore() - require.NoError(t, err) - session.SetSchemaLease(600 * time.Millisecond) - session.DisableStats4Test() - dom, err := session.BootstrapSession(store) - require.NoError(t, err) - - dom.SetStatsUpdating(true) - - clean := func() { - dom.Close() - err := store.Close() - require.NoError(t, err) - } - defer clean() - tk := testkit.NewTestKit(t, store) - tk2 := testkit.NewTestKit(t, store) - - tk.MustExec("use test") - tk.MustExec("drop table if exists t1") - tk2.MustExec("use test") - /* Test of cache table */ - tk.MustExec("create table t1 ( n int auto_increment primary key)") - tk.MustGetErrCode("alter table t1 ca", errno.ErrParse) - tk.MustGetErrCode("alter table t2 cache", errno.ErrNoSuchTable) - tk.MustExec("alter table t1 cache") - checkTableCacheStatus(t, tk.Session(), "test", "t1", model.TableCacheStatusEnable) - tk.MustExec("drop table if exists t1") - /*Test can't skip schema checker*/ - tk.MustExec("drop table if exists t1,t2") - tk.MustExec("CREATE TABLE t1 (a int)") - tk.MustExec("CREATE TABLE t2 (a int)") - tk.MustExec("begin") - tk.MustExec("insert into t1 set a=1;") - tk2.MustExec("alter table t1 cache;") - _, err = tk.Exec("commit") - require.True(t, terror.ErrorEqual(domain.ErrInfoSchemaChanged, err)) - /* Test can skip schema checker */ - tk.MustExec("begin") - tk.MustExec("drop table if exists t1") - tk.MustExec("CREATE TABLE t1 (a int)") - tk.MustExec("insert into t1 set a=2;") - tk2.MustExec("alter table t2 cache") - tk.MustExec("commit") - // Test if a table is not exists - tk.MustExec("drop table if exists t") - tk.MustGetErrCode("alter table t cache", errno.ErrNoSuchTable) - tk.MustExec("create table t (a int)") - tk.MustExec("alter table t cache") - // Multiple alter cache is okay - tk.MustExec("alter table t cache") - tk.MustExec("alter table t cache") - // Test a temporary table - tk.MustExec("drop table if exists t") - tk.MustExec("create temporary table t (id int primary key auto_increment, u int unique, v int)") - tk.MustExec("drop table if exists tmp1") - // local temporary table alter is not supported - tk.MustGetErrCode("alter table t cache", errno.ErrUnsupportedDDLOperation) - // test global temporary table - tk.MustExec("create global temporary table tmp1 " + - "(id int not null primary key, code int not null, value int default null, unique key code(code))" + - "on commit delete rows") - tk.MustGetErrMsg("alter table tmp1 cache", ddl.ErrOptOnTemporaryTable.GenWithStackByArgs("alter temporary table cache").Error()) - -} diff --git a/ddl/db_cache_test.go b/ddl/db_cache_test.go index 120c0c687e049..506b8d7089fe6 100644 --- a/ddl/db_cache_test.go +++ b/ddl/db_cache_test.go @@ -16,12 +16,16 @@ package ddl_test import ( "testing" + "time" + "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/parser/terror" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/testkit" "github.com/stretchr/testify/require" @@ -133,3 +137,69 @@ func TestIndexOnCacheTable(t *testing.T) { tk.MustExec("alter table cache_index_1 cache") tk.MustGetErrCode("alter table cache_index_1 drop index i1, drop index i2;", errno.ErrOptOnCacheTable) } + +func TestAlterTableCache(t *testing.T) { + store, err := mockstore.NewMockStore() + require.NoError(t, err) + session.SetSchemaLease(600 * time.Millisecond) + session.DisableStats4Test() + dom, err := session.BootstrapSession(store) + require.NoError(t, err) + + dom.SetStatsUpdating(true) + + clean := func() { + dom.Close() + err := store.Close() + require.NoError(t, err) + } + defer clean() + tk := testkit.NewTestKit(t, store) + tk2 := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("drop table if exists t1") + tk2.MustExec("use test") + /* Test of cache table */ + tk.MustExec("create table t1 ( n int auto_increment primary key)") + tk.MustGetErrCode("alter table t1 ca", errno.ErrParse) + tk.MustGetErrCode("alter table t2 cache", errno.ErrNoSuchTable) + tk.MustExec("alter table t1 cache") + checkTableCacheStatus(t, tk.Session(), "test", "t1", model.TableCacheStatusEnable) + tk.MustExec("drop table if exists t1") + /*Test can't skip schema checker*/ + tk.MustExec("drop table if exists t1,t2") + tk.MustExec("CREATE TABLE t1 (a int)") + tk.MustExec("CREATE TABLE t2 (a int)") + tk.MustExec("begin") + tk.MustExec("insert into t1 set a=1;") + tk2.MustExec("alter table t1 cache;") + _, err = tk.Exec("commit") + require.True(t, terror.ErrorEqual(domain.ErrInfoSchemaChanged, err)) + /* Test can skip schema checker */ + tk.MustExec("begin") + tk.MustExec("drop table if exists t1") + tk.MustExec("CREATE TABLE t1 (a int)") + tk.MustExec("insert into t1 set a=2;") + tk2.MustExec("alter table t2 cache") + tk.MustExec("commit") + // Test if a table is not exists + tk.MustExec("drop table if exists t") + tk.MustGetErrCode("alter table t cache", errno.ErrNoSuchTable) + tk.MustExec("create table t (a int)") + tk.MustExec("alter table t cache") + // Multiple alter cache is okay + tk.MustExec("alter table t cache") + tk.MustExec("alter table t cache") + // Test a temporary table + tk.MustExec("drop table if exists t") + tk.MustExec("create temporary table t (id int primary key auto_increment, u int unique, v int)") + tk.MustExec("drop table if exists tmp1") + // local temporary table alter is not supported + tk.MustGetErrCode("alter table t cache", errno.ErrUnsupportedDDLOperation) + // test global temporary table + tk.MustExec("create global temporary table tmp1 " + + "(id int not null primary key, code int not null, value int default null, unique key code(code))" + + "on commit delete rows") + tk.MustGetErrMsg("alter table tmp1 cache", ddl.ErrOptOnTemporaryTable.GenWithStackByArgs("alter temporary table cache").Error()) +} diff --git a/ddl/ddl_algorithm_serial_test.go b/ddl/ddl_algorithm_test.go similarity index 100% rename from ddl/ddl_algorithm_serial_test.go rename to ddl/ddl_algorithm_test.go diff --git a/ddl/failtest/fail_db_serial_test.go b/ddl/failtest/fail_db_test.go similarity index 100% rename from ddl/failtest/fail_db_serial_test.go rename to ddl/failtest/fail_db_test.go diff --git a/ddl/util/syncer_serial_test.go b/ddl/util/syncer_test.go similarity index 100% rename from ddl/util/syncer_serial_test.go rename to ddl/util/syncer_test.go diff --git a/infoschema/cluster_tables_serial_test.go b/infoschema/cluster_tables_test.go similarity index 100% rename from infoschema/cluster_tables_serial_test.go rename to infoschema/cluster_tables_test.go diff --git a/infoschema/perfschema/tables_serial_test.go b/infoschema/perfschema/tables_serial_test.go deleted file mode 100644 index f8126a57b2b7c..0000000000000 --- a/infoschema/perfschema/tables_serial_test.go +++ /dev/null @@ -1,159 +0,0 @@ -// Copyright 2021 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package perfschema_test - -import ( - "fmt" - "io" - "net/http" - "net/http/httptest" - "os" - "path/filepath" - "runtime/pprof" - "strings" - "testing" - - "github.com/pingcap/failpoint" - "github.com/pingcap/tidb/parser/terror" - "github.com/pingcap/tidb/testkit" - "github.com/stretchr/testify/require" -) - -func TestTiKVProfileCPU(t *testing.T) { - store, clean := newMockStore(t) - defer clean() - - router := http.NewServeMux() - mockServer := httptest.NewServer(router) - mockAddr := strings.TrimPrefix(mockServer.URL, "http://") - defer mockServer.Close() - - // mock tikv profile - copyHandler := func(filename string) http.HandlerFunc { - return func(w http.ResponseWriter, _ *http.Request) { - file, err := os.Open(filepath.Join(currentSourceDir(), filename)) - if err != nil { - http.Error(w, err.Error(), http.StatusBadRequest) - return - } - defer func() { terror.Log(file.Close()) }() - _, err = io.Copy(w, file) - terror.Log(err) - } - } - router.HandleFunc("/debug/pprof/profile", copyHandler("testdata/tikv.cpu.profile")) - - // failpoint setting - servers := []string{ - strings.Join([]string{"tikv", mockAddr, mockAddr}, ","), - strings.Join([]string{"pd", mockAddr, mockAddr}, ","), - } - fpExpr := strings.Join(servers, ";") - fpName := "github.com/pingcap/tidb/infoschema/perfschema/mockRemoteNodeStatusAddress" - require.NoError(t, failpoint.Enable(fpName, fmt.Sprintf(`return("%s")`, fpExpr))) - defer func() { require.NoError(t, failpoint.Disable(fpName)) }() - - tk := testkit.NewTestKit(t, store) - - tk.MustExec("use performance_schema") - result := tk.MustQuery("select function, percent_abs, percent_rel from tikv_profile_cpu where depth < 3") - - warnings := tk.Session().GetSessionVars().StmtCtx.GetWarnings() - require.Lenf(t, warnings, 0, "expect no warnings, but found: %+v", warnings) - - result.Check(testkit.Rows( - "root 100% 100%", - "├─tikv::server::load_statistics::linux::ThreadLoadStatistics::record::h59facb8d680e7794 75.00% 75.00%", - "│ └─procinfo::pid::stat::stat_task::h69e1aa2c331aebb6 75.00% 100%", - "├─nom::nom::digit::h905aaaeff7d8ec8e 16.07% 16.07%", - "│ ├─ as core::iter::traits::iterator::Iterator>::next::h16936f9061bb75e4 6.25% 38.89%", - "│ ├─Unknown 3.57% 22.22%", - "│ ├─<&u8 as nom::traits::AsChar>::is_dec_digit::he9eacc3fad26ab81 2.68% 16.67%", - "│ ├─<&[u8] as nom::traits::InputIter>::iter_indices::h6192338433683bff 1.79% 11.11%", - "│ └─<&[T] as nom::traits::Slice>>::slice::h38d31f11f84aa302 1.79% 11.11%", - "├─::realloc::h5199c50710ab6f9d 1.79% 1.79%", - "│ └─rallocx 1.79% 100%", - "├─::dealloc::hea83459aa98dd2dc 1.79% 1.79%", - "│ └─sdallocx 1.79% 100%", - "├─::alloc::hc7962e02169a5c56 0.89% 0.89%", - "│ └─mallocx 0.89% 100%", - "├─engine::rocks::util::engine_metrics::flush_engine_iostall_properties::h64a7661c95aa1db7 0.89% 0.89%", - "│ └─rocksdb::rocksdb::DB::get_map_property_cf::h9722f9040411af44 0.89% 100%", - "├─core::ptr::real_drop_in_place::h8def0d99e7136f33 0.89% 0.89%", - "│ └─ as core::ops::drop::Drop>::drop::h9b59b303bffde02c 0.89% 100%", - "├─tikv_util::metrics::threads_linux::ThreadInfoStatistics::record::ha8cc290b3f46af88 0.89% 0.89%", - "│ └─procinfo::pid::stat::stat_task::h69e1aa2c331aebb6 0.89% 100%", - "├─crossbeam_utils::backoff::Backoff::snooze::h5c121ef4ce616a3c 0.89% 0.89%", - "│ └─core::iter::range::>::next::hdb23ceb766e7a91f 0.89% 100%", - "└─::next::he129c78b3deb639d 0.89% 0.89%", - " └─Unknown 0.89% 100%")) - - // We can use current processe profile to mock profile of PD because the PD has the - // same way of retrieving profile with TiDB. And the purpose of this test case is used - // to make sure all profile HTTP API have been accessed. - accessed := map[string]struct{}{} - handlerFactory := func(name string, debug ...int) func(w http.ResponseWriter, _ *http.Request) { - debugLevel := 0 - if len(debug) > 0 { - debugLevel = debug[0] - } - return func(w http.ResponseWriter, _ *http.Request) { - profile := pprof.Lookup(name) - if profile == nil { - http.Error(w, fmt.Sprintf("profile %s not found", name), http.StatusBadRequest) - return - } - if err := profile.WriteTo(w, debugLevel); err != nil { - http.Error(w, err.Error(), http.StatusBadRequest) - return - } - accessed[name] = struct{}{} - } - } - - // mock PD profile - router.HandleFunc("/pd/api/v1/debug/pprof/profile", copyHandler("../../util/profile/testdata/test.pprof")) - router.HandleFunc("/pd/api/v1/debug/pprof/heap", handlerFactory("heap")) - router.HandleFunc("/pd/api/v1/debug/pprof/mutex", handlerFactory("mutex")) - router.HandleFunc("/pd/api/v1/debug/pprof/allocs", handlerFactory("allocs")) - router.HandleFunc("/pd/api/v1/debug/pprof/block", handlerFactory("block")) - router.HandleFunc("/pd/api/v1/debug/pprof/goroutine", handlerFactory("goroutine", 2)) - - tk.MustQuery("select * from pd_profile_cpu where depth < 3") - warnings = tk.Session().GetSessionVars().StmtCtx.GetWarnings() - require.Lenf(t, warnings, 0, "expect no warnings, but found: %+v", warnings) - - tk.MustQuery("select * from pd_profile_memory where depth < 3") - warnings = tk.Session().GetSessionVars().StmtCtx.GetWarnings() - require.Lenf(t, warnings, 0, "expect no warnings, but found: %+v", warnings) - - tk.MustQuery("select * from pd_profile_mutex where depth < 3") - warnings = tk.Session().GetSessionVars().StmtCtx.GetWarnings() - require.Lenf(t, warnings, 0, "expect no warnings, but found: %+v", warnings) - - tk.MustQuery("select * from pd_profile_allocs where depth < 3") - warnings = tk.Session().GetSessionVars().StmtCtx.GetWarnings() - require.Lenf(t, warnings, 0, "expect no warnings, but found: %+v", warnings) - - tk.MustQuery("select * from pd_profile_block where depth < 3") - warnings = tk.Session().GetSessionVars().StmtCtx.GetWarnings() - require.Lenf(t, warnings, 0, "expect no warnings, but found: %+v", warnings) - - tk.MustQuery("select * from pd_profile_goroutines") - warnings = tk.Session().GetSessionVars().StmtCtx.GetWarnings() - require.Lenf(t, warnings, 0, "expect no warnings, but found: %+v", warnings) - - require.Lenf(t, accessed, 5, "expect all HTTP API had been accessed, but found: %v", accessed) -} diff --git a/infoschema/perfschema/tables_test.go b/infoschema/perfschema/tables_test.go index be739a5b6af48..057ba404efec7 100644 --- a/infoschema/perfschema/tables_test.go +++ b/infoschema/perfschema/tables_test.go @@ -15,12 +15,21 @@ package perfschema_test import ( + "fmt" + "io" + "net/http" + "net/http/httptest" + "os" "path/filepath" "runtime" + "runtime/pprof" + "strings" "testing" + "github.com/pingcap/failpoint" "github.com/pingcap/tidb/infoschema/perfschema" "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/parser/terror" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/testkit" @@ -44,6 +53,133 @@ func TestPerfSchemaTables(t *testing.T) { tk.MustQuery("select * from events_stages_history_long").Check(testkit.Rows()) } +func TestTiKVProfileCPU(t *testing.T) { + store, clean := newMockStore(t) + defer clean() + + router := http.NewServeMux() + mockServer := httptest.NewServer(router) + mockAddr := strings.TrimPrefix(mockServer.URL, "http://") + defer mockServer.Close() + + // mock tikv profile + copyHandler := func(filename string) http.HandlerFunc { + return func(w http.ResponseWriter, _ *http.Request) { + file, err := os.Open(filepath.Join(currentSourceDir(), filename)) + if err != nil { + http.Error(w, err.Error(), http.StatusBadRequest) + return + } + defer func() { terror.Log(file.Close()) }() + _, err = io.Copy(w, file) + terror.Log(err) + } + } + router.HandleFunc("/debug/pprof/profile", copyHandler("testdata/tikv.cpu.profile")) + + // failpoint setting + servers := []string{ + strings.Join([]string{"tikv", mockAddr, mockAddr}, ","), + strings.Join([]string{"pd", mockAddr, mockAddr}, ","), + } + fpExpr := strings.Join(servers, ";") + fpName := "github.com/pingcap/tidb/infoschema/perfschema/mockRemoteNodeStatusAddress" + require.NoError(t, failpoint.Enable(fpName, fmt.Sprintf(`return("%s")`, fpExpr))) + defer func() { require.NoError(t, failpoint.Disable(fpName)) }() + + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use performance_schema") + result := tk.MustQuery("select function, percent_abs, percent_rel from tikv_profile_cpu where depth < 3") + + warnings := tk.Session().GetSessionVars().StmtCtx.GetWarnings() + require.Lenf(t, warnings, 0, "expect no warnings, but found: %+v", warnings) + + result.Check(testkit.Rows( + "root 100% 100%", + "├─tikv::server::load_statistics::linux::ThreadLoadStatistics::record::h59facb8d680e7794 75.00% 75.00%", + "│ └─procinfo::pid::stat::stat_task::h69e1aa2c331aebb6 75.00% 100%", + "├─nom::nom::digit::h905aaaeff7d8ec8e 16.07% 16.07%", + "│ ├─ as core::iter::traits::iterator::Iterator>::next::h16936f9061bb75e4 6.25% 38.89%", + "│ ├─Unknown 3.57% 22.22%", + "│ ├─<&u8 as nom::traits::AsChar>::is_dec_digit::he9eacc3fad26ab81 2.68% 16.67%", + "│ ├─<&[u8] as nom::traits::InputIter>::iter_indices::h6192338433683bff 1.79% 11.11%", + "│ └─<&[T] as nom::traits::Slice>>::slice::h38d31f11f84aa302 1.79% 11.11%", + "├─::realloc::h5199c50710ab6f9d 1.79% 1.79%", + "│ └─rallocx 1.79% 100%", + "├─::dealloc::hea83459aa98dd2dc 1.79% 1.79%", + "│ └─sdallocx 1.79% 100%", + "├─::alloc::hc7962e02169a5c56 0.89% 0.89%", + "│ └─mallocx 0.89% 100%", + "├─engine::rocks::util::engine_metrics::flush_engine_iostall_properties::h64a7661c95aa1db7 0.89% 0.89%", + "│ └─rocksdb::rocksdb::DB::get_map_property_cf::h9722f9040411af44 0.89% 100%", + "├─core::ptr::real_drop_in_place::h8def0d99e7136f33 0.89% 0.89%", + "│ └─ as core::ops::drop::Drop>::drop::h9b59b303bffde02c 0.89% 100%", + "├─tikv_util::metrics::threads_linux::ThreadInfoStatistics::record::ha8cc290b3f46af88 0.89% 0.89%", + "│ └─procinfo::pid::stat::stat_task::h69e1aa2c331aebb6 0.89% 100%", + "├─crossbeam_utils::backoff::Backoff::snooze::h5c121ef4ce616a3c 0.89% 0.89%", + "│ └─core::iter::range::>::next::hdb23ceb766e7a91f 0.89% 100%", + "└─::next::he129c78b3deb639d 0.89% 0.89%", + " └─Unknown 0.89% 100%")) + + // We can use current processe profile to mock profile of PD because the PD has the + // same way of retrieving profile with TiDB. And the purpose of this test case is used + // to make sure all profile HTTP API have been accessed. + accessed := map[string]struct{}{} + handlerFactory := func(name string, debug ...int) func(w http.ResponseWriter, _ *http.Request) { + debugLevel := 0 + if len(debug) > 0 { + debugLevel = debug[0] + } + return func(w http.ResponseWriter, _ *http.Request) { + profile := pprof.Lookup(name) + if profile == nil { + http.Error(w, fmt.Sprintf("profile %s not found", name), http.StatusBadRequest) + return + } + if err := profile.WriteTo(w, debugLevel); err != nil { + http.Error(w, err.Error(), http.StatusBadRequest) + return + } + accessed[name] = struct{}{} + } + } + + // mock PD profile + router.HandleFunc("/pd/api/v1/debug/pprof/profile", copyHandler("../../util/profile/testdata/test.pprof")) + router.HandleFunc("/pd/api/v1/debug/pprof/heap", handlerFactory("heap")) + router.HandleFunc("/pd/api/v1/debug/pprof/mutex", handlerFactory("mutex")) + router.HandleFunc("/pd/api/v1/debug/pprof/allocs", handlerFactory("allocs")) + router.HandleFunc("/pd/api/v1/debug/pprof/block", handlerFactory("block")) + router.HandleFunc("/pd/api/v1/debug/pprof/goroutine", handlerFactory("goroutine", 2)) + + tk.MustQuery("select * from pd_profile_cpu where depth < 3") + warnings = tk.Session().GetSessionVars().StmtCtx.GetWarnings() + require.Lenf(t, warnings, 0, "expect no warnings, but found: %+v", warnings) + + tk.MustQuery("select * from pd_profile_memory where depth < 3") + warnings = tk.Session().GetSessionVars().StmtCtx.GetWarnings() + require.Lenf(t, warnings, 0, "expect no warnings, but found: %+v", warnings) + + tk.MustQuery("select * from pd_profile_mutex where depth < 3") + warnings = tk.Session().GetSessionVars().StmtCtx.GetWarnings() + require.Lenf(t, warnings, 0, "expect no warnings, but found: %+v", warnings) + + tk.MustQuery("select * from pd_profile_allocs where depth < 3") + warnings = tk.Session().GetSessionVars().StmtCtx.GetWarnings() + require.Lenf(t, warnings, 0, "expect no warnings, but found: %+v", warnings) + + tk.MustQuery("select * from pd_profile_block where depth < 3") + warnings = tk.Session().GetSessionVars().StmtCtx.GetWarnings() + require.Lenf(t, warnings, 0, "expect no warnings, but found: %+v", warnings) + + tk.MustQuery("select * from pd_profile_goroutines") + warnings = tk.Session().GetSessionVars().StmtCtx.GetWarnings() + require.Lenf(t, warnings, 0, "expect no warnings, but found: %+v", warnings) + + require.Lenf(t, accessed, 5, "expect all HTTP API had been accessed, but found: %v", accessed) +} + func newMockStore(t *testing.T) (store kv.Storage, clean func()) { var err error store, err = mockstore.NewMockStore() diff --git a/infoschema/tables_serial_test.go b/infoschema/tables_test.go similarity index 100% rename from infoschema/tables_serial_test.go rename to infoschema/tables_test.go diff --git a/session/bootstrap_serial_test.go b/session/bootstrap_test.go similarity index 100% rename from session/bootstrap_serial_test.go rename to session/bootstrap_test.go diff --git a/session/clustered_index_serial_test.go b/session/clustered_index_serial_test.go deleted file mode 100644 index 056c525944ac0..0000000000000 --- a/session/clustered_index_serial_test.go +++ /dev/null @@ -1,341 +0,0 @@ -// Copyright 2021 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package session_test - -import ( - "fmt" - "math/rand" - "strings" - "testing" - - "github.com/pingcap/tidb/config" - "github.com/pingcap/tidb/errno" - "github.com/pingcap/tidb/sessionctx/variable" - "github.com/pingcap/tidb/testkit" - "github.com/pingcap/tidb/util/collate" - "github.com/pingcap/tidb/util/israce" -) - -func TestCreateClusteredTable(t *testing.T) { - store, clean := testkit.CreateMockStore(t) - defer clean() - - tk := createTestKit(t, store) - tk.MustExec("set @@tidb_enable_clustered_index = 'int_only';") - tk.MustExec("drop table if exists t1, t2, t3, t4, t5, t6, t7, t8") - tk.MustExec("create table t1(id int primary key, v int)") - tk.MustExec("create table t2(id varchar(10) primary key, v int)") - tk.MustExec("create table t3(id int primary key clustered, v int)") - tk.MustExec("create table t4(id varchar(10) primary key clustered, v int)") - tk.MustExec("create table t5(id int primary key nonclustered, v int)") - tk.MustExec("create table t6(id varchar(10) primary key nonclustered, v int)") - tk.MustExec("create table t7(id varchar(10), v int, primary key (id) /*T![clustered_index] CLUSTERED */)") - tk.MustExec("create table t8(id varchar(10), v int, primary key (id) /*T![clustered_index] NONCLUSTERED */)") - tk.MustQuery("show index from t1").Check(testkit.Rows("t1 0 PRIMARY 1 id A 0 BTREE YES YES")) - tk.MustQuery("show index from t2").Check(testkit.Rows("t2 0 PRIMARY 1 id A 0 BTREE YES NO")) - tk.MustQuery("show index from t3").Check(testkit.Rows("t3 0 PRIMARY 1 id A 0 BTREE YES YES")) - tk.MustQuery("show index from t4").Check(testkit.Rows("t4 0 PRIMARY 1 id A 0 BTREE YES YES")) - tk.MustQuery("show index from t5").Check(testkit.Rows("t5 0 PRIMARY 1 id A 0 BTREE YES NO")) - tk.MustQuery("show index from t6").Check(testkit.Rows("t6 0 PRIMARY 1 id A 0 BTREE YES NO")) - tk.MustQuery("show index from t7").Check(testkit.Rows("t7 0 PRIMARY 1 id A 0 BTREE YES YES")) - tk.MustQuery("show index from t8").Check(testkit.Rows("t8 0 PRIMARY 1 id A 0 BTREE YES NO")) - - tk.MustExec("set @@tidb_enable_clustered_index = 'off';") - tk.MustExec("drop table if exists t1, t2, t3, t4, t5, t6, t7, t8") - tk.MustExec("create table t1(id int primary key, v int)") - tk.MustExec("create table t2(id varchar(10) primary key, v int)") - tk.MustExec("create table t3(id int primary key clustered, v int)") - tk.MustExec("create table t4(id varchar(10) primary key clustered, v int)") - tk.MustExec("create table t5(id int primary key nonclustered, v int)") - tk.MustExec("create table t6(id varchar(10) primary key nonclustered, v int)") - tk.MustExec("create table t7(id varchar(10), v int, primary key (id) /*T![clustered_index] CLUSTERED */)") - tk.MustExec("create table t8(id varchar(10), v int, primary key (id) /*T![clustered_index] NONCLUSTERED */)") - tk.MustQuery("show index from t1").Check(testkit.Rows("t1 0 PRIMARY 1 id A 0 BTREE YES NO")) - tk.MustQuery("show index from t2").Check(testkit.Rows("t2 0 PRIMARY 1 id A 0 BTREE YES NO")) - tk.MustQuery("show index from t3").Check(testkit.Rows("t3 0 PRIMARY 1 id A 0 BTREE YES YES")) - tk.MustQuery("show index from t4").Check(testkit.Rows("t4 0 PRIMARY 1 id A 0 BTREE YES YES")) - tk.MustQuery("show index from t5").Check(testkit.Rows("t5 0 PRIMARY 1 id A 0 BTREE YES NO")) - tk.MustQuery("show index from t6").Check(testkit.Rows("t6 0 PRIMARY 1 id A 0 BTREE YES NO")) - tk.MustQuery("show index from t7").Check(testkit.Rows("t7 0 PRIMARY 1 id A 0 BTREE YES YES")) - tk.MustQuery("show index from t8").Check(testkit.Rows("t8 0 PRIMARY 1 id A 0 BTREE YES NO")) - - tk.MustExec("set @@tidb_enable_clustered_index = 'on';") - tk.MustExec("drop table if exists t1, t2, t3, t4, t5, t6, t7, t8") - tk.MustExec("create table t1(id int primary key, v int)") - tk.MustExec("create table t2(id varchar(10) primary key, v int)") - tk.MustExec("create table t3(id int primary key clustered, v int)") - tk.MustExec("create table t4(id varchar(10) primary key clustered, v int)") - tk.MustExec("create table t5(id int primary key nonclustered, v int)") - tk.MustExec("create table t6(id varchar(10) primary key nonclustered, v int)") - tk.MustExec("create table t7(id varchar(10), v int, primary key (id) /*T![clustered_index] CLUSTERED */)") - tk.MustExec("create table t8(id varchar(10), v int, primary key (id) /*T![clustered_index] NONCLUSTERED */)") - tk.MustQuery("show index from t1").Check(testkit.Rows("t1 0 PRIMARY 1 id A 0 BTREE YES YES")) - tk.MustQuery("show index from t2").Check(testkit.Rows("t2 0 PRIMARY 1 id A 0 BTREE YES YES")) - tk.MustQuery("show index from t3").Check(testkit.Rows("t3 0 PRIMARY 1 id A 0 BTREE YES YES")) - tk.MustQuery("show index from t4").Check(testkit.Rows("t4 0 PRIMARY 1 id A 0 BTREE YES YES")) - tk.MustQuery("show index from t5").Check(testkit.Rows("t5 0 PRIMARY 1 id A 0 BTREE YES NO")) - tk.MustQuery("show index from t6").Check(testkit.Rows("t6 0 PRIMARY 1 id A 0 BTREE YES NO")) - tk.MustQuery("show index from t7").Check(testkit.Rows("t7 0 PRIMARY 1 id A 0 BTREE YES YES")) - tk.MustQuery("show index from t8").Check(testkit.Rows("t8 0 PRIMARY 1 id A 0 BTREE YES NO")) - - tk.MustExec("set @@tidb_enable_clustered_index = 'int_only';") - defer config.RestoreFunc()() - config.UpdateGlobal(func(conf *config.Config) { - conf.AlterPrimaryKey = true - }) - tk.MustExec("drop table if exists t1, t2, t3, t4, t5, t6, t7, t8") - tk.MustExec("create table t1(id int primary key, v int)") - tk.MustExec("create table t2(id varchar(10) primary key, v int)") - tk.MustExec("create table t3(id int primary key clustered, v int)") - tk.MustExec("create table t4(id varchar(10) primary key clustered, v int)") - tk.MustExec("create table t5(id int primary key nonclustered, v int)") - tk.MustExec("create table t6(id varchar(10) primary key nonclustered, v int)") - tk.MustExec("create table t7(id varchar(10), v int, primary key (id) /*T![clustered_index] CLUSTERED */)") - tk.MustExec("create table t8(id varchar(10), v int, primary key (id) /*T![clustered_index] NONCLUSTERED */)") - tk.MustQuery("show index from t1").Check(testkit.Rows("t1 0 PRIMARY 1 id A 0 BTREE YES NO")) - tk.MustQuery("show index from t2").Check(testkit.Rows("t2 0 PRIMARY 1 id A 0 BTREE YES NO")) - tk.MustQuery("show index from t3").Check(testkit.Rows("t3 0 PRIMARY 1 id A 0 BTREE YES YES")) - tk.MustQuery("show index from t4").Check(testkit.Rows("t4 0 PRIMARY 1 id A 0 BTREE YES YES")) - tk.MustQuery("show index from t5").Check(testkit.Rows("t5 0 PRIMARY 1 id A 0 BTREE YES NO")) - tk.MustQuery("show index from t6").Check(testkit.Rows("t6 0 PRIMARY 1 id A 0 BTREE YES NO")) - tk.MustQuery("show index from t7").Check(testkit.Rows("t7 0 PRIMARY 1 id A 0 BTREE YES YES")) - tk.MustQuery("show index from t8").Check(testkit.Rows("t8 0 PRIMARY 1 id A 0 BTREE YES NO")) -} - -// Test for union scan in prefixed clustered index table. -// See https://github.com/pingcap/tidb/issues/22069. -func TestClusteredUnionScanOnPrefixingPrimaryKey(t *testing.T) { - originCollate := collate.NewCollationEnabled() - collate.SetNewCollationEnabledForTest(false) - defer collate.SetNewCollationEnabledForTest(originCollate) - store, clean := testkit.CreateMockStore(t) - defer clean() - tk := createTestKit(t, store) - tk.MustExec("drop table if exists t;") - tk.MustExec("create table t (col_1 varchar(255), col_2 tinyint, primary key idx_1 (col_1(1)));") - tk.MustExec("insert into t values ('aaaaa', -38);") - tk.MustExec("insert into t values ('bbbbb', -48);") - - tk.MustExec("begin PESSIMISTIC;") - tk.MustExec("update t set col_2 = 47 where col_1 in ('aaaaa') order by col_1,col_2;") - tk.MustQuery("select * from t;").Check(testkit.Rows("aaaaa 47", "bbbbb -48")) - tk.MustGetErrCode("insert into t values ('bb', 0);", errno.ErrDupEntry) - tk.MustGetErrCode("insert into t values ('aa', 0);", errno.ErrDupEntry) - tk.MustExec("commit;") - tk.MustQuery("select * from t;").Check(testkit.Rows("aaaaa 47", "bbbbb -48")) - tk.MustExec("admin check table t;") -} - -// https://github.com/pingcap/tidb/issues/22453 -func TestClusteredIndexSplitAndAddIndex2(t *testing.T) { - store, clean := testkit.CreateMockStore(t) - defer clean() - - tk := createTestKit(t, store) - tk.MustExec("drop table if exists t;") - tk.MustExec("create table t (a int, b enum('Alice'), c int, primary key (c, b));") - tk.MustExec("insert into t values (-1,'Alice',100);") - tk.MustExec("insert into t values (-1,'Alice',7000);") - tk.MustQuery("split table t between (0,'Alice') and (10000,'Alice') regions 2;").Check(testkit.Rows("1 1")) - tk.MustExec("set @@global.tidb_ddl_error_count_limit = 3;") - tk.MustExec("alter table t add index idx (c);") - tk.MustExec("admin check table t;") -} - -func TestClusteredIndexSyntax(t *testing.T) { - store, clean := testkit.CreateMockStore(t) - defer clean() - - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - const showPKType = `select tidb_pk_type from information_schema.tables where table_schema = 'test' and table_name = 't';` - const nonClustered, clustered = `NONCLUSTERED`, `CLUSTERED` - assertPkType := func(sql string, pkType string) { - tk.MustExec("drop table if exists t;") - tk.MustExec(sql) - tk.MustQuery(showPKType).Check(testkit.Rows(pkType)) - } - - // Test single integer column as the primary key. - clusteredDefault := clustered - assertPkType("create table t (a int primary key, b int);", clusteredDefault) - assertPkType("create table t (a int, b int, primary key(a) clustered);", clustered) - assertPkType("create table t (a int, b int, primary key(a) /*T![clustered_index] clustered */);", clustered) - assertPkType("create table t (a int, b int, primary key(a) nonclustered);", nonClustered) - assertPkType("create table t (a int, b int, primary key(a) /*T![clustered_index] nonclustered */);", nonClustered) - - // Test for clustered index. - tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly - assertPkType("create table t (a int, b varchar(255), primary key(b, a));", nonClustered) - assertPkType("create table t (a int, b varchar(255), primary key(b, a) nonclustered);", nonClustered) - assertPkType("create table t (a int, b varchar(255), primary key(b, a) clustered);", clustered) - tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn - assertPkType("create table t (a int, b varchar(255), primary key(b, a));", clusteredDefault) - assertPkType("create table t (a int, b varchar(255), primary key(b, a) nonclustered);", nonClustered) - assertPkType("create table t (a int, b varchar(255), primary key(b, a) /*T![clustered_index] nonclustered */);", nonClustered) - assertPkType("create table t (a int, b varchar(255), primary key(b, a) clustered);", clustered) - assertPkType("create table t (a int, b varchar(255), primary key(b, a) /*T![clustered_index] clustered */);", clustered) - - tk.MustGetErrCode("create table t (a varchar(255) unique key clustered);", errno.ErrParse) - tk.MustGetErrCode("create table t (a varchar(255), foreign key (a) reference t1(a) clustered);", errno.ErrParse) - tk.MustGetErrCode("create table t (a varchar(255), foreign key (a) clustered reference t1(a));", errno.ErrParse) - tk.MustGetErrCode("create table t (a varchar(255) clustered);", errno.ErrParse) - - errMsg := "[ddl:8200]CLUSTERED/NONCLUSTERED keyword is only supported for primary key" - tk.MustGetErrMsg("create table t (a varchar(255), unique key(a) clustered);", errMsg) - tk.MustGetErrMsg("create table t (a varchar(255), unique key(a) nonclustered);", errMsg) - tk.MustGetErrMsg("create table t (a varchar(255), unique index(a) clustered);", errMsg) - tk.MustGetErrMsg("create table t (a varchar(255), unique index(a) nonclustered);", errMsg) - tk.MustGetErrMsg("create table t (a varchar(255), key(a) clustered);", errMsg) - tk.MustGetErrMsg("create table t (a varchar(255), key(a) nonclustered);", errMsg) - tk.MustGetErrMsg("create table t (a varchar(255), index(a) clustered);", errMsg) - tk.MustGetErrMsg("create table t (a varchar(255), index(a) nonclustered);", errMsg) - tk.MustGetErrMsg("create table t (a varchar(255), b decimal(5, 4), primary key (a, b) clustered, key (b) clustered)", errMsg) - tk.MustGetErrMsg("create table t (a varchar(255), b decimal(5, 4), primary key (a, b) clustered, key (b) nonclustered)", errMsg) -} - -func TestPrefixClusteredIndexAddIndexAndRecover(t *testing.T) { - store, clean := testkit.CreateMockStore(t) - defer clean() - - tk1 := testkit.NewTestKit(t, store) - tk1.MustExec("use test;") - tk1.MustExec("drop table if exists t;") - defer func() { - tk1.MustExec("drop table if exists t;") - }() - - tk1.MustExec("create table t(a char(3), b char(3), primary key(a(1)) clustered)") - tk1.MustExec("insert into t values ('aaa', 'bbb')") - tk1.MustExec("alter table t add index idx(b)") - tk1.MustQuery("select * from t use index(idx)").Check(testkit.Rows("aaa bbb")) - tk1.MustExec("admin check table t") - tk1.MustExec("admin recover index t idx") - tk1.MustQuery("select * from t use index(idx)").Check(testkit.Rows("aaa bbb")) - tk1.MustExec("admin check table t") -} - -func TestPartitionTable(t *testing.T) { - if israce.RaceEnabled { - t.Skip("exhaustive types test, skip race test") - } - - store, clean := testkit.CreateMockStore(t) - defer clean() - - tk := testkit.NewTestKit(t, store) - tk.MustExec("create database test_view") - tk.MustExec("use test_view") - tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") - - tk.MustExec(`create table thash (a int, b int, c varchar(32), primary key(a, b) clustered) partition by hash(a) partitions 4`) - tk.MustExec(`create table trange (a int, b int, c varchar(32), primary key(a, b) clustered) partition by range columns(a) ( - partition p0 values less than (3000), - partition p1 values less than (6000), - partition p2 values less than (9000), - partition p3 values less than (10000))`) - tk.MustExec(`create table tnormal (a int, b int, c varchar(32), primary key(a, b))`) - - vals := make([]string, 0, 4000) - existedPK := make(map[string]struct{}, 4000) - for i := 0; i < 4000; { - a := rand.Intn(10000) - b := rand.Intn(10000) - pk := fmt.Sprintf("%v, %v", a, b) - if _, ok := existedPK[pk]; ok { - continue - } - existedPK[pk] = struct{}{} - i++ - vals = append(vals, fmt.Sprintf(`(%v, %v, '%v')`, a, b, rand.Intn(10000))) - } - - tk.MustExec("insert into thash values " + strings.Join(vals, ", ")) - tk.MustExec("insert into trange values " + strings.Join(vals, ", ")) - tk.MustExec("insert into tnormal values " + strings.Join(vals, ", ")) - - for i := 0; i < 200; i++ { - cond := fmt.Sprintf("where a in (%v, %v, %v) and b < %v", rand.Intn(10000), rand.Intn(10000), rand.Intn(10000), rand.Intn(10000)) - result := tk.MustQuery("select * from tnormal " + cond).Sort().Rows() - tk.MustQuery("select * from thash use index(primary) " + cond).Sort().Check(result) - tk.MustQuery("select * from trange use index(primary) " + cond).Sort().Check(result) - } -} - -// https://github.com/pingcap/tidb/issues/23106 -func TestClusteredIndexDecodeRestoredDataV5(t *testing.T) { - defer collate.SetNewCollationEnabledForTest(false) - collate.SetNewCollationEnabledForTest(true) - - store, clean := testkit.CreateMockStore(t) - defer clean() - - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn - tk.MustExec("drop table if exists t;") - tk.MustExec("create table t (id1 int, id2 varchar(10), a1 int, primary key(id1, id2) clustered) collate utf8mb4_general_ci;") - tk.MustExec("insert into t values (1, 'asd', 1), (1, 'dsa', 1);") - tk.MustGetErrCode("alter table t add unique index t_idx(id1, a1);", errno.ErrDupEntry) - - tk.MustExec("drop table if exists t;") - tk.MustExec("create table t (id1 int, id2 varchar(10), a1 int, primary key(id1, id2) clustered, unique key t_idx(id1, a1)) collate utf8mb4_general_ci;") - tk.MustExec("begin;") - tk.MustExec("insert into t values (1, 'asd', 1);") - tk.MustQuery("select * from t use index (t_idx);").Check(testkit.Rows("1 asd 1")) - tk.MustExec("commit;") - tk.MustExec("admin check table t;") - tk.MustExec("drop table t;") -} - -// https://github.com/pingcap/tidb/issues/23178 -func TestPrefixedClusteredIndexUniqueKeyWithNewCollation(t *testing.T) { - defer collate.SetNewCollationEnabledForTest(false) - collate.SetNewCollationEnabledForTest(true) - - store, clean := testkit.CreateMockStore(t) - defer clean() - - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test;") - tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn - tk.MustExec("create table t (a text collate utf8mb4_general_ci not null, b int(11) not null, " + - "primary key (a(10), b) clustered, key idx(a(2)) ) default charset=utf8mb4 collate=utf8mb4_bin;") - tk.MustExec("insert into t values ('aaa', 2);") - // Key-value content: sk = sortKey, p = prefixed - // row record: sk(aaa), 2 -> aaa - // index record: sk(p(aa)), {sk(aaa), 2} -> restore data(aaa) - tk.MustExec("admin check table t;") - tk.MustExec("drop table t;") -} - -func TestClusteredIndexNewCollationWithOldRowFormat(t *testing.T) { - // This case maybe not useful, because newCollation isn't convenience to run on TiKV(it's required serialSuit) - // but unistore doesn't support old row format. - defer collate.SetNewCollationEnabledForTest(false) - collate.SetNewCollationEnabledForTest(true) - - store, clean := testkit.CreateMockStore(t) - defer clean() - - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test;") - tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn - tk.Session().GetSessionVars().RowEncoder.Enable = false - tk.MustExec("drop table if exists t2") - tk.MustExec("create table t2(col_1 varchar(132) CHARACTER SET utf8 COLLATE utf8_unicode_ci, primary key(col_1) clustered)") - tk.MustExec("insert into t2 select 'aBc'") - tk.MustQuery("select col_1 from t2 where col_1 = 'aBc'").Check(testkit.Rows("aBc")) -} diff --git a/session/clustered_index_test.go b/session/clustered_index_test.go index b993bd3405a6d..32de5a4348632 100644 --- a/session/clustered_index_test.go +++ b/session/clustered_index_test.go @@ -15,14 +15,20 @@ package session_test import ( + "fmt" + "math/rand" + "strings" "testing" + "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/testkit/testdata" + "github.com/pingcap/tidb/util/collate" + "github.com/pingcap/tidb/util/israce" "github.com/stretchr/testify/require" ) @@ -421,3 +427,315 @@ func TestClusteredIndexSelectWhereInNull(t *testing.T) { tk.MustExec("create table t (a datetime, b bigint, primary key (a));") tk.MustQuery("select * from t where a in (null);").Check(testkit.Rows( /* empty result */ )) } + +func TestCreateClusteredTable(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := createTestKit(t, store) + tk.MustExec("set @@tidb_enable_clustered_index = 'int_only';") + tk.MustExec("drop table if exists t1, t2, t3, t4, t5, t6, t7, t8") + tk.MustExec("create table t1(id int primary key, v int)") + tk.MustExec("create table t2(id varchar(10) primary key, v int)") + tk.MustExec("create table t3(id int primary key clustered, v int)") + tk.MustExec("create table t4(id varchar(10) primary key clustered, v int)") + tk.MustExec("create table t5(id int primary key nonclustered, v int)") + tk.MustExec("create table t6(id varchar(10) primary key nonclustered, v int)") + tk.MustExec("create table t7(id varchar(10), v int, primary key (id) /*T![clustered_index] CLUSTERED */)") + tk.MustExec("create table t8(id varchar(10), v int, primary key (id) /*T![clustered_index] NONCLUSTERED */)") + tk.MustQuery("show index from t1").Check(testkit.Rows("t1 0 PRIMARY 1 id A 0 BTREE YES YES")) + tk.MustQuery("show index from t2").Check(testkit.Rows("t2 0 PRIMARY 1 id A 0 BTREE YES NO")) + tk.MustQuery("show index from t3").Check(testkit.Rows("t3 0 PRIMARY 1 id A 0 BTREE YES YES")) + tk.MustQuery("show index from t4").Check(testkit.Rows("t4 0 PRIMARY 1 id A 0 BTREE YES YES")) + tk.MustQuery("show index from t5").Check(testkit.Rows("t5 0 PRIMARY 1 id A 0 BTREE YES NO")) + tk.MustQuery("show index from t6").Check(testkit.Rows("t6 0 PRIMARY 1 id A 0 BTREE YES NO")) + tk.MustQuery("show index from t7").Check(testkit.Rows("t7 0 PRIMARY 1 id A 0 BTREE YES YES")) + tk.MustQuery("show index from t8").Check(testkit.Rows("t8 0 PRIMARY 1 id A 0 BTREE YES NO")) + + tk.MustExec("set @@tidb_enable_clustered_index = 'off';") + tk.MustExec("drop table if exists t1, t2, t3, t4, t5, t6, t7, t8") + tk.MustExec("create table t1(id int primary key, v int)") + tk.MustExec("create table t2(id varchar(10) primary key, v int)") + tk.MustExec("create table t3(id int primary key clustered, v int)") + tk.MustExec("create table t4(id varchar(10) primary key clustered, v int)") + tk.MustExec("create table t5(id int primary key nonclustered, v int)") + tk.MustExec("create table t6(id varchar(10) primary key nonclustered, v int)") + tk.MustExec("create table t7(id varchar(10), v int, primary key (id) /*T![clustered_index] CLUSTERED */)") + tk.MustExec("create table t8(id varchar(10), v int, primary key (id) /*T![clustered_index] NONCLUSTERED */)") + tk.MustQuery("show index from t1").Check(testkit.Rows("t1 0 PRIMARY 1 id A 0 BTREE YES NO")) + tk.MustQuery("show index from t2").Check(testkit.Rows("t2 0 PRIMARY 1 id A 0 BTREE YES NO")) + tk.MustQuery("show index from t3").Check(testkit.Rows("t3 0 PRIMARY 1 id A 0 BTREE YES YES")) + tk.MustQuery("show index from t4").Check(testkit.Rows("t4 0 PRIMARY 1 id A 0 BTREE YES YES")) + tk.MustQuery("show index from t5").Check(testkit.Rows("t5 0 PRIMARY 1 id A 0 BTREE YES NO")) + tk.MustQuery("show index from t6").Check(testkit.Rows("t6 0 PRIMARY 1 id A 0 BTREE YES NO")) + tk.MustQuery("show index from t7").Check(testkit.Rows("t7 0 PRIMARY 1 id A 0 BTREE YES YES")) + tk.MustQuery("show index from t8").Check(testkit.Rows("t8 0 PRIMARY 1 id A 0 BTREE YES NO")) + + tk.MustExec("set @@tidb_enable_clustered_index = 'on';") + tk.MustExec("drop table if exists t1, t2, t3, t4, t5, t6, t7, t8") + tk.MustExec("create table t1(id int primary key, v int)") + tk.MustExec("create table t2(id varchar(10) primary key, v int)") + tk.MustExec("create table t3(id int primary key clustered, v int)") + tk.MustExec("create table t4(id varchar(10) primary key clustered, v int)") + tk.MustExec("create table t5(id int primary key nonclustered, v int)") + tk.MustExec("create table t6(id varchar(10) primary key nonclustered, v int)") + tk.MustExec("create table t7(id varchar(10), v int, primary key (id) /*T![clustered_index] CLUSTERED */)") + tk.MustExec("create table t8(id varchar(10), v int, primary key (id) /*T![clustered_index] NONCLUSTERED */)") + tk.MustQuery("show index from t1").Check(testkit.Rows("t1 0 PRIMARY 1 id A 0 BTREE YES YES")) + tk.MustQuery("show index from t2").Check(testkit.Rows("t2 0 PRIMARY 1 id A 0 BTREE YES YES")) + tk.MustQuery("show index from t3").Check(testkit.Rows("t3 0 PRIMARY 1 id A 0 BTREE YES YES")) + tk.MustQuery("show index from t4").Check(testkit.Rows("t4 0 PRIMARY 1 id A 0 BTREE YES YES")) + tk.MustQuery("show index from t5").Check(testkit.Rows("t5 0 PRIMARY 1 id A 0 BTREE YES NO")) + tk.MustQuery("show index from t6").Check(testkit.Rows("t6 0 PRIMARY 1 id A 0 BTREE YES NO")) + tk.MustQuery("show index from t7").Check(testkit.Rows("t7 0 PRIMARY 1 id A 0 BTREE YES YES")) + tk.MustQuery("show index from t8").Check(testkit.Rows("t8 0 PRIMARY 1 id A 0 BTREE YES NO")) + + tk.MustExec("set @@tidb_enable_clustered_index = 'int_only';") + defer config.RestoreFunc()() + config.UpdateGlobal(func(conf *config.Config) { + conf.AlterPrimaryKey = true + }) + tk.MustExec("drop table if exists t1, t2, t3, t4, t5, t6, t7, t8") + tk.MustExec("create table t1(id int primary key, v int)") + tk.MustExec("create table t2(id varchar(10) primary key, v int)") + tk.MustExec("create table t3(id int primary key clustered, v int)") + tk.MustExec("create table t4(id varchar(10) primary key clustered, v int)") + tk.MustExec("create table t5(id int primary key nonclustered, v int)") + tk.MustExec("create table t6(id varchar(10) primary key nonclustered, v int)") + tk.MustExec("create table t7(id varchar(10), v int, primary key (id) /*T![clustered_index] CLUSTERED */)") + tk.MustExec("create table t8(id varchar(10), v int, primary key (id) /*T![clustered_index] NONCLUSTERED */)") + tk.MustQuery("show index from t1").Check(testkit.Rows("t1 0 PRIMARY 1 id A 0 BTREE YES NO")) + tk.MustQuery("show index from t2").Check(testkit.Rows("t2 0 PRIMARY 1 id A 0 BTREE YES NO")) + tk.MustQuery("show index from t3").Check(testkit.Rows("t3 0 PRIMARY 1 id A 0 BTREE YES YES")) + tk.MustQuery("show index from t4").Check(testkit.Rows("t4 0 PRIMARY 1 id A 0 BTREE YES YES")) + tk.MustQuery("show index from t5").Check(testkit.Rows("t5 0 PRIMARY 1 id A 0 BTREE YES NO")) + tk.MustQuery("show index from t6").Check(testkit.Rows("t6 0 PRIMARY 1 id A 0 BTREE YES NO")) + tk.MustQuery("show index from t7").Check(testkit.Rows("t7 0 PRIMARY 1 id A 0 BTREE YES YES")) + tk.MustQuery("show index from t8").Check(testkit.Rows("t8 0 PRIMARY 1 id A 0 BTREE YES NO")) +} + +// Test for union scan in prefixed clustered index table. +// See https://github.com/pingcap/tidb/issues/22069. +func TestClusteredUnionScanOnPrefixingPrimaryKey(t *testing.T) { + originCollate := collate.NewCollationEnabled() + collate.SetNewCollationEnabledForTest(false) + defer collate.SetNewCollationEnabledForTest(originCollate) + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := createTestKit(t, store) + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t (col_1 varchar(255), col_2 tinyint, primary key idx_1 (col_1(1)));") + tk.MustExec("insert into t values ('aaaaa', -38);") + tk.MustExec("insert into t values ('bbbbb', -48);") + + tk.MustExec("begin PESSIMISTIC;") + tk.MustExec("update t set col_2 = 47 where col_1 in ('aaaaa') order by col_1,col_2;") + tk.MustQuery("select * from t;").Check(testkit.Rows("aaaaa 47", "bbbbb -48")) + tk.MustGetErrCode("insert into t values ('bb', 0);", errno.ErrDupEntry) + tk.MustGetErrCode("insert into t values ('aa', 0);", errno.ErrDupEntry) + tk.MustExec("commit;") + tk.MustQuery("select * from t;").Check(testkit.Rows("aaaaa 47", "bbbbb -48")) + tk.MustExec("admin check table t;") +} + +// https://github.com/pingcap/tidb/issues/22453 +func TestClusteredIndexSplitAndAddIndex2(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := createTestKit(t, store) + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t (a int, b enum('Alice'), c int, primary key (c, b));") + tk.MustExec("insert into t values (-1,'Alice',100);") + tk.MustExec("insert into t values (-1,'Alice',7000);") + tk.MustQuery("split table t between (0,'Alice') and (10000,'Alice') regions 2;").Check(testkit.Rows("1 1")) + tk.MustExec("set @@global.tidb_ddl_error_count_limit = 3;") + tk.MustExec("alter table t add index idx (c);") + tk.MustExec("admin check table t;") +} + +func TestClusteredIndexSyntax(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + const showPKType = `select tidb_pk_type from information_schema.tables where table_schema = 'test' and table_name = 't';` + const nonClustered, clustered = `NONCLUSTERED`, `CLUSTERED` + assertPkType := func(sql string, pkType string) { + tk.MustExec("drop table if exists t;") + tk.MustExec(sql) + tk.MustQuery(showPKType).Check(testkit.Rows(pkType)) + } + + // Test single integer column as the primary key. + clusteredDefault := clustered + assertPkType("create table t (a int primary key, b int);", clusteredDefault) + assertPkType("create table t (a int, b int, primary key(a) clustered);", clustered) + assertPkType("create table t (a int, b int, primary key(a) /*T![clustered_index] clustered */);", clustered) + assertPkType("create table t (a int, b int, primary key(a) nonclustered);", nonClustered) + assertPkType("create table t (a int, b int, primary key(a) /*T![clustered_index] nonclustered */);", nonClustered) + + // Test for clustered index. + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly + assertPkType("create table t (a int, b varchar(255), primary key(b, a));", nonClustered) + assertPkType("create table t (a int, b varchar(255), primary key(b, a) nonclustered);", nonClustered) + assertPkType("create table t (a int, b varchar(255), primary key(b, a) clustered);", clustered) + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + assertPkType("create table t (a int, b varchar(255), primary key(b, a));", clusteredDefault) + assertPkType("create table t (a int, b varchar(255), primary key(b, a) nonclustered);", nonClustered) + assertPkType("create table t (a int, b varchar(255), primary key(b, a) /*T![clustered_index] nonclustered */);", nonClustered) + assertPkType("create table t (a int, b varchar(255), primary key(b, a) clustered);", clustered) + assertPkType("create table t (a int, b varchar(255), primary key(b, a) /*T![clustered_index] clustered */);", clustered) + + tk.MustGetErrCode("create table t (a varchar(255) unique key clustered);", errno.ErrParse) + tk.MustGetErrCode("create table t (a varchar(255), foreign key (a) reference t1(a) clustered);", errno.ErrParse) + tk.MustGetErrCode("create table t (a varchar(255), foreign key (a) clustered reference t1(a));", errno.ErrParse) + tk.MustGetErrCode("create table t (a varchar(255) clustered);", errno.ErrParse) + + errMsg := "[ddl:8200]CLUSTERED/NONCLUSTERED keyword is only supported for primary key" + tk.MustGetErrMsg("create table t (a varchar(255), unique key(a) clustered);", errMsg) + tk.MustGetErrMsg("create table t (a varchar(255), unique key(a) nonclustered);", errMsg) + tk.MustGetErrMsg("create table t (a varchar(255), unique index(a) clustered);", errMsg) + tk.MustGetErrMsg("create table t (a varchar(255), unique index(a) nonclustered);", errMsg) + tk.MustGetErrMsg("create table t (a varchar(255), key(a) clustered);", errMsg) + tk.MustGetErrMsg("create table t (a varchar(255), key(a) nonclustered);", errMsg) + tk.MustGetErrMsg("create table t (a varchar(255), index(a) clustered);", errMsg) + tk.MustGetErrMsg("create table t (a varchar(255), index(a) nonclustered);", errMsg) + tk.MustGetErrMsg("create table t (a varchar(255), b decimal(5, 4), primary key (a, b) clustered, key (b) clustered)", errMsg) + tk.MustGetErrMsg("create table t (a varchar(255), b decimal(5, 4), primary key (a, b) clustered, key (b) nonclustered)", errMsg) +} + +func TestPrefixClusteredIndexAddIndexAndRecover(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk1 := testkit.NewTestKit(t, store) + tk1.MustExec("use test;") + tk1.MustExec("drop table if exists t;") + defer func() { + tk1.MustExec("drop table if exists t;") + }() + + tk1.MustExec("create table t(a char(3), b char(3), primary key(a(1)) clustered)") + tk1.MustExec("insert into t values ('aaa', 'bbb')") + tk1.MustExec("alter table t add index idx(b)") + tk1.MustQuery("select * from t use index(idx)").Check(testkit.Rows("aaa bbb")) + tk1.MustExec("admin check table t") + tk1.MustExec("admin recover index t idx") + tk1.MustQuery("select * from t use index(idx)").Check(testkit.Rows("aaa bbb")) + tk1.MustExec("admin check table t") +} + +func TestPartitionTable(t *testing.T) { + if israce.RaceEnabled { + t.Skip("exhaustive types test, skip race test") + } + + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("create database test_view") + tk.MustExec("use test_view") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + + tk.MustExec(`create table thash (a int, b int, c varchar(32), primary key(a, b) clustered) partition by hash(a) partitions 4`) + tk.MustExec(`create table trange (a int, b int, c varchar(32), primary key(a, b) clustered) partition by range columns(a) ( + partition p0 values less than (3000), + partition p1 values less than (6000), + partition p2 values less than (9000), + partition p3 values less than (10000))`) + tk.MustExec(`create table tnormal (a int, b int, c varchar(32), primary key(a, b))`) + + vals := make([]string, 0, 4000) + existedPK := make(map[string]struct{}, 4000) + for i := 0; i < 4000; { + a := rand.Intn(10000) + b := rand.Intn(10000) + pk := fmt.Sprintf("%v, %v", a, b) + if _, ok := existedPK[pk]; ok { + continue + } + existedPK[pk] = struct{}{} + i++ + vals = append(vals, fmt.Sprintf(`(%v, %v, '%v')`, a, b, rand.Intn(10000))) + } + + tk.MustExec("insert into thash values " + strings.Join(vals, ", ")) + tk.MustExec("insert into trange values " + strings.Join(vals, ", ")) + tk.MustExec("insert into tnormal values " + strings.Join(vals, ", ")) + + for i := 0; i < 200; i++ { + cond := fmt.Sprintf("where a in (%v, %v, %v) and b < %v", rand.Intn(10000), rand.Intn(10000), rand.Intn(10000), rand.Intn(10000)) + result := tk.MustQuery("select * from tnormal " + cond).Sort().Rows() + tk.MustQuery("select * from thash use index(primary) " + cond).Sort().Check(result) + tk.MustQuery("select * from trange use index(primary) " + cond).Sort().Check(result) + } +} + +// https://github.com/pingcap/tidb/issues/23106 +func TestClusteredIndexDecodeRestoredDataV5(t *testing.T) { + defer collate.SetNewCollationEnabledForTest(false) + collate.SetNewCollationEnabledForTest(true) + + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t (id1 int, id2 varchar(10), a1 int, primary key(id1, id2) clustered) collate utf8mb4_general_ci;") + tk.MustExec("insert into t values (1, 'asd', 1), (1, 'dsa', 1);") + tk.MustGetErrCode("alter table t add unique index t_idx(id1, a1);", errno.ErrDupEntry) + + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t (id1 int, id2 varchar(10), a1 int, primary key(id1, id2) clustered, unique key t_idx(id1, a1)) collate utf8mb4_general_ci;") + tk.MustExec("begin;") + tk.MustExec("insert into t values (1, 'asd', 1);") + tk.MustQuery("select * from t use index (t_idx);").Check(testkit.Rows("1 asd 1")) + tk.MustExec("commit;") + tk.MustExec("admin check table t;") + tk.MustExec("drop table t;") +} + +// https://github.com/pingcap/tidb/issues/23178 +func TestPrefixedClusteredIndexUniqueKeyWithNewCollation(t *testing.T) { + defer collate.SetNewCollationEnabledForTest(false) + collate.SetNewCollationEnabledForTest(true) + + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test;") + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + tk.MustExec("create table t (a text collate utf8mb4_general_ci not null, b int(11) not null, " + + "primary key (a(10), b) clustered, key idx(a(2)) ) default charset=utf8mb4 collate=utf8mb4_bin;") + tk.MustExec("insert into t values ('aaa', 2);") + // Key-value content: sk = sortKey, p = prefixed + // row record: sk(aaa), 2 -> aaa + // index record: sk(p(aa)), {sk(aaa), 2} -> restore data(aaa) + tk.MustExec("admin check table t;") + tk.MustExec("drop table t;") +} + +func TestClusteredIndexNewCollationWithOldRowFormat(t *testing.T) { + // This case maybe not useful, because newCollation isn't convenience to run on TiKV(it's required serialSuit) + // but unistore doesn't support old row format. + defer collate.SetNewCollationEnabledForTest(false) + collate.SetNewCollationEnabledForTest(true) + + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test;") + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + tk.Session().GetSessionVars().RowEncoder.Enable = false + tk.MustExec("drop table if exists t2") + tk.MustExec("create table t2(col_1 varchar(132) CHARACTER SET utf8 COLLATE utf8_unicode_ci, primary key(col_1) clustered)") + tk.MustExec("insert into t2 select 'aBc'") + tk.MustQuery("select col_1 from t2 where col_1 = 'aBc'").Check(testkit.Rows("aBc")) +} diff --git a/session/index_usage_sync_lease_serial_test.go b/session/index_usage_sync_lease_test.go similarity index 100% rename from session/index_usage_sync_lease_serial_test.go rename to session/index_usage_sync_lease_test.go diff --git a/session/schema_amender_serial_test.go b/session/schema_amender_test.go similarity index 100% rename from session/schema_amender_serial_test.go rename to session/schema_amender_test.go diff --git a/store/batch_coprocessor_serial_test.go b/store/batch_coprocessor_test.go similarity index 100% rename from store/batch_coprocessor_serial_test.go rename to store/batch_coprocessor_test.go diff --git a/store/driver/config_serial_test.go b/store/driver/config_test.go similarity index 100% rename from store/driver/config_serial_test.go rename to store/driver/config_test.go diff --git a/store/driver/snap_interceptor_serial_test.go b/store/driver/snap_interceptor_test.go similarity index 100% rename from store/driver/snap_interceptor_serial_test.go rename to store/driver/snap_interceptor_test.go diff --git a/store/driver/sql_fail_serial_test.go b/store/driver/sql_fail_test.go similarity index 100% rename from store/driver/sql_fail_serial_test.go rename to store/driver/sql_fail_test.go diff --git a/store/driver/txn_serial_test.go b/store/driver/txn_test.go similarity index 100% rename from store/driver/txn_serial_test.go rename to store/driver/txn_test.go diff --git a/store/gcworker/gc_worker_serial_test.go b/store/gcworker/gc_worker_test.go similarity index 100% rename from store/gcworker/gc_worker_serial_test.go rename to store/gcworker/gc_worker_test.go diff --git a/table/tables/index_serial_test.go b/table/tables/index_serial_test.go deleted file mode 100644 index 16784c16d0c7e..0000000000000 --- a/table/tables/index_serial_test.go +++ /dev/null @@ -1,324 +0,0 @@ -// Copyright 2021 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package tables_test - -import ( - "context" - "io" - "testing" - "time" - - "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/parser/model" - "github.com/pingcap/tidb/parser/mysql" - "github.com/pingcap/tidb/parser/terror" - "github.com/pingcap/tidb/sessionctx/stmtctx" - "github.com/pingcap/tidb/table" - "github.com/pingcap/tidb/table/tables" - "github.com/pingcap/tidb/tablecodec" - "github.com/pingcap/tidb/testkit" - "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/codec" - "github.com/pingcap/tidb/util/collate" - "github.com/pingcap/tidb/util/mock" - "github.com/pingcap/tidb/util/rowcodec" - "github.com/stretchr/testify/require" -) - -func TestIndex(t *testing.T) { - tblInfo := &model.TableInfo{ - ID: 1, - Indices: []*model.IndexInfo{ - { - ID: 2, - Name: model.NewCIStr("test"), - Columns: []*model.IndexColumn{ - {Offset: 0}, - {Offset: 1}, - }, - }, - }, - Columns: []*model.ColumnInfo{ - {ID: 1, Name: model.NewCIStr("c2"), State: model.StatePublic, Offset: 0, FieldType: *types.NewFieldType(mysql.TypeVarchar)}, - {ID: 2, Name: model.NewCIStr("c2"), State: model.StatePublic, Offset: 1, FieldType: *types.NewFieldType(mysql.TypeString)}, - }, - } - index := tables.NewIndex(tblInfo.ID, tblInfo, tblInfo.Indices[0]) - - // Test ununiq index. - store, clean := testkit.CreateMockStore(t) - defer clean() - txn, err := store.Begin() - require.NoError(t, err) - - values := types.MakeDatums(1, 2) - mockCtx := mock.NewContext() - _, err = index.Create(mockCtx, txn, values, kv.IntHandle(1), nil) - require.NoError(t, err) - - it, err := index.SeekFirst(txn) - require.NoError(t, err) - - getValues, h, err := it.Next() - require.NoError(t, err) - require.Len(t, getValues, 2) - require.Equal(t, int64(1), getValues[0].GetInt64()) - require.Equal(t, int64(2), getValues[1].GetInt64()) - require.Equal(t, int64(1), h.IntValue()) - it.Close() - sc := &stmtctx.StatementContext{TimeZone: time.Local} - exist, _, err := index.Exist(sc, txn, values, kv.IntHandle(100)) - require.NoError(t, err) - require.False(t, exist) - - exist, _, err = index.Exist(sc, txn, values, kv.IntHandle(1)) - require.NoError(t, err) - require.True(t, exist) - - err = index.Delete(sc, txn, values, kv.IntHandle(1)) - require.NoError(t, err) - - it, err = index.SeekFirst(txn) - require.NoError(t, err) - - _, _, err = it.Next() - require.Truef(t, terror.ErrorEqual(err, io.EOF), "err %v", err) - it.Close() - - _, err = index.Create(mockCtx, txn, values, kv.IntHandle(0), nil) - require.NoError(t, err) - - _, err = index.SeekFirst(txn) - require.NoError(t, err) - - _, hit, err := index.Seek(sc, txn, values) - require.NoError(t, err) - require.False(t, hit) - - err = index.Drop(txn) - require.NoError(t, err) - - it, hit, err = index.Seek(sc, txn, values) - require.NoError(t, err) - require.False(t, hit) - - _, _, err = it.Next() - require.Truef(t, terror.ErrorEqual(err, io.EOF), "err %v", err) - it.Close() - - it, err = index.SeekFirst(txn) - require.NoError(t, err) - - _, _, err = it.Next() - require.Truef(t, terror.ErrorEqual(err, io.EOF), "err %v", err) - it.Close() - - err = txn.Commit(context.Background()) - require.NoError(t, err) - - tblInfo = &model.TableInfo{ - ID: 2, - Indices: []*model.IndexInfo{ - { - ID: 3, - Name: model.NewCIStr("test"), - Unique: true, - Columns: []*model.IndexColumn{ - {Offset: 0}, - {Offset: 1}, - }, - }, - }, - Columns: []*model.ColumnInfo{ - {ID: 1, Name: model.NewCIStr("c2"), State: model.StatePublic, Offset: 0, FieldType: *types.NewFieldType(mysql.TypeVarchar)}, - {ID: 2, Name: model.NewCIStr("c2"), State: model.StatePublic, Offset: 1, FieldType: *types.NewFieldType(mysql.TypeString)}, - }, - } - index = tables.NewIndex(tblInfo.ID, tblInfo, tblInfo.Indices[0]) - - // Test uniq index. - txn, err = store.Begin() - require.NoError(t, err) - - _, err = index.Create(mockCtx, txn, values, kv.IntHandle(1), nil) - require.NoError(t, err) - - _, err = index.Create(mockCtx, txn, values, kv.IntHandle(2), nil) - require.NotNil(t, err) - - it, err = index.SeekFirst(txn) - require.NoError(t, err) - - getValues, h, err = it.Next() - require.NoError(t, err) - require.Len(t, getValues, 2) - require.Equal(t, int64(1), getValues[0].GetInt64()) - require.Equal(t, int64(2), getValues[1].GetInt64()) - require.Equal(t, int64(1), h.IntValue()) - it.Close() - - exist, h, err = index.Exist(sc, txn, values, kv.IntHandle(1)) - require.NoError(t, err) - require.Equal(t, int64(1), h.IntValue()) - require.True(t, exist) - - exist, h, err = index.Exist(sc, txn, values, kv.IntHandle(2)) - require.NotNil(t, err) - require.Equal(t, int64(1), h.IntValue()) - require.True(t, exist) - - err = txn.Commit(context.Background()) - require.NoError(t, err) - - _, err = index.FetchValues(make([]types.Datum, 0), nil) - require.NotNil(t, err) - - txn, err = store.Begin() - require.NoError(t, err) - - // Test the function of Next when the value of unique key is nil. - values2 := types.MakeDatums(nil, nil) - _, err = index.Create(mockCtx, txn, values2, kv.IntHandle(2), nil) - require.NoError(t, err) - it, err = index.SeekFirst(txn) - require.NoError(t, err) - getValues, h, err = it.Next() - require.NoError(t, err) - require.Len(t, getValues, 2) - require.Equal(t, nil, getValues[0].GetInterface()) - require.Equal(t, nil, getValues[1].GetInterface()) - require.Equal(t, int64(2), h.IntValue()) - it.Close() - - err = txn.Commit(context.Background()) - require.NoError(t, err) -} - -func TestCombineIndexSeek(t *testing.T) { - tblInfo := &model.TableInfo{ - ID: 1, - Indices: []*model.IndexInfo{ - { - ID: 2, - Name: model.NewCIStr("test"), - Columns: []*model.IndexColumn{ - {Offset: 1}, - {Offset: 2}, - }, - }, - }, - Columns: []*model.ColumnInfo{ - {Offset: 0}, - {Offset: 1}, - {Offset: 2}, - }, - } - index := tables.NewIndex(tblInfo.ID, tblInfo, tblInfo.Indices[0]) - - store, clean := testkit.CreateMockStore(t) - defer clean() - txn, err := store.Begin() - require.NoError(t, err) - - mockCtx := mock.NewContext() - values := types.MakeDatums("abc", "def") - _, err = index.Create(mockCtx, txn, values, kv.IntHandle(1), nil) - require.NoError(t, err) - - index2 := tables.NewIndex(tblInfo.ID, tblInfo, tblInfo.Indices[0]) - sc := &stmtctx.StatementContext{TimeZone: time.Local} - iter, hit, err := index2.Seek(sc, txn, types.MakeDatums("abc", nil)) - require.NoError(t, err) - defer iter.Close() - require.False(t, hit) - _, h, err := iter.Next() - require.NoError(t, err) - require.Equal(t, int64(1), h.IntValue()) -} - -func TestMultiColumnCommonHandle(t *testing.T) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - tblInfo := buildTableInfo(t, "create table t (a int, b int, u varchar(64) unique, nu varchar(64), primary key (a, b), index nu (nu))") - var idxUnique, idxNonUnique table.Index - for _, idxInfo := range tblInfo.Indices { - idx := tables.NewIndex(tblInfo.ID, tblInfo, idxInfo) - if idxInfo.Name.L == "u" { - idxUnique = idx - } else if idxInfo.Name.L == "nu" { - idxNonUnique = idx - } - } - var a, b *model.ColumnInfo - for _, col := range tblInfo.Columns { - if col.Name.String() == "a" { - a = col - } else if col.Name.String() == "b" { - b = col - } - } - require.NotNil(t, a) - require.NotNil(t, b) - - store, clean := testkit.CreateMockStore(t) - defer clean() - txn, err := store.Begin() - require.NoError(t, err) - mockCtx := mock.NewContext() - sc := mockCtx.GetSessionVars().StmtCtx - // create index for "insert t values (3, 2, "abc", "abc") - idxColVals := types.MakeDatums("abc") - handleColVals := types.MakeDatums(3, 2) - encodedHandle, err := codec.EncodeKey(sc, nil, handleColVals...) - require.NoError(t, err) - commonHandle, err := kv.NewCommonHandle(encodedHandle) - require.NoError(t, err) - _ = idxNonUnique - for _, idx := range []table.Index{idxUnique, idxNonUnique} { - key, _, err := idx.GenIndexKey(sc, idxColVals, commonHandle, nil) - require.NoError(t, err) - _, err = idx.Create(mockCtx, txn, idxColVals, commonHandle, nil) - require.NoError(t, err) - val, err := txn.Get(context.Background(), key) - require.NoError(t, err) - colInfo := tables.BuildRowcodecColInfoForIndexColumns(idx.Meta(), tblInfo) - colInfo = append(colInfo, rowcodec.ColInfo{ - ID: a.ID, - IsPKHandle: false, - Ft: rowcodec.FieldTypeFromModelColumn(a), - }) - colInfo = append(colInfo, rowcodec.ColInfo{ - ID: b.ID, - IsPKHandle: false, - Ft: rowcodec.FieldTypeFromModelColumn(b), - }) - colVals, err := tablecodec.DecodeIndexKV(key, val, 1, tablecodec.HandleDefault, colInfo) - require.NoError(t, err) - require.Len(t, colVals, 3) - _, d, err := codec.DecodeOne(colVals[0]) - require.NoError(t, err) - require.Equal(t, "abc", d.GetString()) - _, d, err = codec.DecodeOne(colVals[1]) - require.NoError(t, err) - require.Equal(t, int64(3), d.GetInt64()) - _, d, err = codec.DecodeOne(colVals[2]) - require.NoError(t, err) - require.Equal(t, int64(2), d.GetInt64()) - handle, err := tablecodec.DecodeIndexHandle(key, val, 1) - require.NoError(t, err) - require.False(t, handle.IsInt()) - require.Equal(t, commonHandle.Encoded(), handle.Encoded()) - } -} diff --git a/table/tables/index_test.go b/table/tables/index_test.go index 5678ce8b39b18..3e70c85365bd8 100644 --- a/table/tables/index_test.go +++ b/table/tables/index_test.go @@ -1,4 +1,4 @@ -// Copyright 2016 PingCAP, Inc. +// Copyright 2021 PingCAP, Inc. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -16,23 +16,316 @@ package tables_test import ( "context" + "io" "testing" + "time" "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tidb/parser/terror" + "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/codec" + "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/mock" + "github.com/pingcap/tidb/util/rowcodec" "github.com/stretchr/testify/require" ) +func TestIndex(t *testing.T) { + tblInfo := &model.TableInfo{ + ID: 1, + Indices: []*model.IndexInfo{ + { + ID: 2, + Name: model.NewCIStr("test"), + Columns: []*model.IndexColumn{ + {Offset: 0}, + {Offset: 1}, + }, + }, + }, + Columns: []*model.ColumnInfo{ + {ID: 1, Name: model.NewCIStr("c2"), State: model.StatePublic, Offset: 0, FieldType: *types.NewFieldType(mysql.TypeVarchar)}, + {ID: 2, Name: model.NewCIStr("c2"), State: model.StatePublic, Offset: 1, FieldType: *types.NewFieldType(mysql.TypeString)}, + }, + } + index := tables.NewIndex(tblInfo.ID, tblInfo, tblInfo.Indices[0]) + + // Test ununiq index. + store, clean := testkit.CreateMockStore(t) + defer clean() + txn, err := store.Begin() + require.NoError(t, err) + + values := types.MakeDatums(1, 2) + mockCtx := mock.NewContext() + _, err = index.Create(mockCtx, txn, values, kv.IntHandle(1), nil) + require.NoError(t, err) + + it, err := index.SeekFirst(txn) + require.NoError(t, err) + + getValues, h, err := it.Next() + require.NoError(t, err) + require.Len(t, getValues, 2) + require.Equal(t, int64(1), getValues[0].GetInt64()) + require.Equal(t, int64(2), getValues[1].GetInt64()) + require.Equal(t, int64(1), h.IntValue()) + it.Close() + sc := &stmtctx.StatementContext{TimeZone: time.Local} + exist, _, err := index.Exist(sc, txn, values, kv.IntHandle(100)) + require.NoError(t, err) + require.False(t, exist) + + exist, _, err = index.Exist(sc, txn, values, kv.IntHandle(1)) + require.NoError(t, err) + require.True(t, exist) + + err = index.Delete(sc, txn, values, kv.IntHandle(1)) + require.NoError(t, err) + + it, err = index.SeekFirst(txn) + require.NoError(t, err) + + _, _, err = it.Next() + require.Truef(t, terror.ErrorEqual(err, io.EOF), "err %v", err) + it.Close() + + _, err = index.Create(mockCtx, txn, values, kv.IntHandle(0), nil) + require.NoError(t, err) + + _, err = index.SeekFirst(txn) + require.NoError(t, err) + + _, hit, err := index.Seek(sc, txn, values) + require.NoError(t, err) + require.False(t, hit) + + err = index.Drop(txn) + require.NoError(t, err) + + it, hit, err = index.Seek(sc, txn, values) + require.NoError(t, err) + require.False(t, hit) + + _, _, err = it.Next() + require.Truef(t, terror.ErrorEqual(err, io.EOF), "err %v", err) + it.Close() + + it, err = index.SeekFirst(txn) + require.NoError(t, err) + + _, _, err = it.Next() + require.Truef(t, terror.ErrorEqual(err, io.EOF), "err %v", err) + it.Close() + + err = txn.Commit(context.Background()) + require.NoError(t, err) + + tblInfo = &model.TableInfo{ + ID: 2, + Indices: []*model.IndexInfo{ + { + ID: 3, + Name: model.NewCIStr("test"), + Unique: true, + Columns: []*model.IndexColumn{ + {Offset: 0}, + {Offset: 1}, + }, + }, + }, + Columns: []*model.ColumnInfo{ + {ID: 1, Name: model.NewCIStr("c2"), State: model.StatePublic, Offset: 0, FieldType: *types.NewFieldType(mysql.TypeVarchar)}, + {ID: 2, Name: model.NewCIStr("c2"), State: model.StatePublic, Offset: 1, FieldType: *types.NewFieldType(mysql.TypeString)}, + }, + } + index = tables.NewIndex(tblInfo.ID, tblInfo, tblInfo.Indices[0]) + + // Test uniq index. + txn, err = store.Begin() + require.NoError(t, err) + + _, err = index.Create(mockCtx, txn, values, kv.IntHandle(1), nil) + require.NoError(t, err) + + _, err = index.Create(mockCtx, txn, values, kv.IntHandle(2), nil) + require.NotNil(t, err) + + it, err = index.SeekFirst(txn) + require.NoError(t, err) + + getValues, h, err = it.Next() + require.NoError(t, err) + require.Len(t, getValues, 2) + require.Equal(t, int64(1), getValues[0].GetInt64()) + require.Equal(t, int64(2), getValues[1].GetInt64()) + require.Equal(t, int64(1), h.IntValue()) + it.Close() + + exist, h, err = index.Exist(sc, txn, values, kv.IntHandle(1)) + require.NoError(t, err) + require.Equal(t, int64(1), h.IntValue()) + require.True(t, exist) + + exist, h, err = index.Exist(sc, txn, values, kv.IntHandle(2)) + require.NotNil(t, err) + require.Equal(t, int64(1), h.IntValue()) + require.True(t, exist) + + err = txn.Commit(context.Background()) + require.NoError(t, err) + + _, err = index.FetchValues(make([]types.Datum, 0), nil) + require.NotNil(t, err) + + txn, err = store.Begin() + require.NoError(t, err) + + // Test the function of Next when the value of unique key is nil. + values2 := types.MakeDatums(nil, nil) + _, err = index.Create(mockCtx, txn, values2, kv.IntHandle(2), nil) + require.NoError(t, err) + it, err = index.SeekFirst(txn) + require.NoError(t, err) + getValues, h, err = it.Next() + require.NoError(t, err) + require.Len(t, getValues, 2) + require.Equal(t, nil, getValues[0].GetInterface()) + require.Equal(t, nil, getValues[1].GetInterface()) + require.Equal(t, int64(2), h.IntValue()) + it.Close() + + err = txn.Commit(context.Background()) + require.NoError(t, err) +} + +func TestCombineIndexSeek(t *testing.T) { + tblInfo := &model.TableInfo{ + ID: 1, + Indices: []*model.IndexInfo{ + { + ID: 2, + Name: model.NewCIStr("test"), + Columns: []*model.IndexColumn{ + {Offset: 1}, + {Offset: 2}, + }, + }, + }, + Columns: []*model.ColumnInfo{ + {Offset: 0}, + {Offset: 1}, + {Offset: 2}, + }, + } + index := tables.NewIndex(tblInfo.ID, tblInfo, tblInfo.Indices[0]) + + store, clean := testkit.CreateMockStore(t) + defer clean() + txn, err := store.Begin() + require.NoError(t, err) + + mockCtx := mock.NewContext() + values := types.MakeDatums("abc", "def") + _, err = index.Create(mockCtx, txn, values, kv.IntHandle(1), nil) + require.NoError(t, err) + + index2 := tables.NewIndex(tblInfo.ID, tblInfo, tblInfo.Indices[0]) + sc := &stmtctx.StatementContext{TimeZone: time.Local} + iter, hit, err := index2.Seek(sc, txn, types.MakeDatums("abc", nil)) + require.NoError(t, err) + defer iter.Close() + require.False(t, hit) + _, h, err := iter.Next() + require.NoError(t, err) + require.Equal(t, int64(1), h.IntValue()) +} + +func TestMultiColumnCommonHandle(t *testing.T) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + tblInfo := buildTableInfo(t, "create table t (a int, b int, u varchar(64) unique, nu varchar(64), primary key (a, b), index nu (nu))") + var idxUnique, idxNonUnique table.Index + for _, idxInfo := range tblInfo.Indices { + idx := tables.NewIndex(tblInfo.ID, tblInfo, idxInfo) + if idxInfo.Name.L == "u" { + idxUnique = idx + } else if idxInfo.Name.L == "nu" { + idxNonUnique = idx + } + } + var a, b *model.ColumnInfo + for _, col := range tblInfo.Columns { + if col.Name.String() == "a" { + a = col + } else if col.Name.String() == "b" { + b = col + } + } + require.NotNil(t, a) + require.NotNil(t, b) + + store, clean := testkit.CreateMockStore(t) + defer clean() + txn, err := store.Begin() + require.NoError(t, err) + mockCtx := mock.NewContext() + sc := mockCtx.GetSessionVars().StmtCtx + // create index for "insert t values (3, 2, "abc", "abc") + idxColVals := types.MakeDatums("abc") + handleColVals := types.MakeDatums(3, 2) + encodedHandle, err := codec.EncodeKey(sc, nil, handleColVals...) + require.NoError(t, err) + commonHandle, err := kv.NewCommonHandle(encodedHandle) + require.NoError(t, err) + _ = idxNonUnique + for _, idx := range []table.Index{idxUnique, idxNonUnique} { + key, _, err := idx.GenIndexKey(sc, idxColVals, commonHandle, nil) + require.NoError(t, err) + _, err = idx.Create(mockCtx, txn, idxColVals, commonHandle, nil) + require.NoError(t, err) + val, err := txn.Get(context.Background(), key) + require.NoError(t, err) + colInfo := tables.BuildRowcodecColInfoForIndexColumns(idx.Meta(), tblInfo) + colInfo = append(colInfo, rowcodec.ColInfo{ + ID: a.ID, + IsPKHandle: false, + Ft: rowcodec.FieldTypeFromModelColumn(a), + }) + colInfo = append(colInfo, rowcodec.ColInfo{ + ID: b.ID, + IsPKHandle: false, + Ft: rowcodec.FieldTypeFromModelColumn(b), + }) + colVals, err := tablecodec.DecodeIndexKV(key, val, 1, tablecodec.HandleDefault, colInfo) + require.NoError(t, err) + require.Len(t, colVals, 3) + _, d, err := codec.DecodeOne(colVals[0]) + require.NoError(t, err) + require.Equal(t, "abc", d.GetString()) + _, d, err = codec.DecodeOne(colVals[1]) + require.NoError(t, err) + require.Equal(t, int64(3), d.GetInt64()) + _, d, err = codec.DecodeOne(colVals[2]) + require.NoError(t, err) + require.Equal(t, int64(2), d.GetInt64()) + handle, err := tablecodec.DecodeIndexHandle(key, val, 1) + require.NoError(t, err) + require.False(t, handle.IsInt()) + require.Equal(t, commonHandle.Encoded(), handle.Encoded()) + } +} + func TestSingleColumnCommonHandle(t *testing.T) { tblInfo := buildTableInfo(t, "create table t (a varchar(255) primary key, u int unique, nu int, index nu (nu))") var idxUnique, idxNonUnique table.Index diff --git a/telemetry/data_window_serial_test.go b/telemetry/data_window_test.go similarity index 100% rename from telemetry/data_window_serial_test.go rename to telemetry/data_window_test.go diff --git a/telemetry/telemetry_serial_test.go b/telemetry/telemetry_serial_test.go deleted file mode 100644 index c464d46c4c6de..0000000000000 --- a/telemetry/telemetry_serial_test.go +++ /dev/null @@ -1,54 +0,0 @@ -// Copyright 2021 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package telemetry_test - -import ( - "runtime" - "testing" - - "github.com/Jeffail/gabs/v2" - "github.com/pingcap/tidb/config" - "github.com/pingcap/tidb/session" - "github.com/pingcap/tidb/telemetry" - "github.com/pingcap/tidb/testkit" - "github.com/stretchr/testify/require" - "go.etcd.io/etcd/integration" -) - -func TestReport(t *testing.T) { - if runtime.GOOS == "windows" { - t.Skip("integration.NewClusterV3 will create file contains a colon which is not allowed on Windows") - } - - etcdCluster := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) - defer etcdCluster.Terminate(t) - store, clean := testkit.CreateMockStore(t) - defer clean() - se, err := session.CreateSession4Test(store) - require.NoError(t, err) - defer se.Close() - - config.GetGlobalConfig().EnableTelemetry = false - require.NoError(t, telemetry.ReportUsageData(se, etcdCluster.RandClient())) - - status, err := telemetry.GetTelemetryStatus(etcdCluster.RandClient()) - require.NoError(t, err) - - jsonParsed, err := gabs.ParseJSON([]byte(status)) - require.NoError(t, err) - require.True(t, jsonParsed.Path("is_error").Data().(bool)) - require.Equal(t, "telemetry is disabled", jsonParsed.Path("error_msg").Data().(string)) - require.False(t, jsonParsed.Path("is_request_sent").Data().(bool)) -} diff --git a/telemetry/telemetry_test.go b/telemetry/telemetry_test.go index c7f3cef8d92a8..56fc2da76852a 100644 --- a/telemetry/telemetry_test.go +++ b/telemetry/telemetry_test.go @@ -101,3 +101,29 @@ func TestPreview(t *testing.T) { require.NoError(t, err) require.Equal(t, "", r) } + +func TestReport(t *testing.T) { + if runtime.GOOS == "windows" { + t.Skip("integration.NewClusterV3 will create file contains a colon which is not allowed on Windows") + } + + etcdCluster := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) + defer etcdCluster.Terminate(t) + store, clean := testkit.CreateMockStore(t) + defer clean() + se, err := session.CreateSession4Test(store) + require.NoError(t, err) + defer se.Close() + + config.GetGlobalConfig().EnableTelemetry = false + require.NoError(t, telemetry.ReportUsageData(se, etcdCluster.RandClient())) + + status, err := telemetry.GetTelemetryStatus(etcdCluster.RandClient()) + require.NoError(t, err) + + jsonParsed, err := gabs.ParseJSON([]byte(status)) + require.NoError(t, err) + require.True(t, jsonParsed.Path("is_error").Data().(bool)) + require.Equal(t, "telemetry is disabled", jsonParsed.Path("error_msg").Data().(string)) + require.False(t, jsonParsed.Path("is_request_sent").Data().(bool)) +} diff --git a/types/enum_serial_test.go b/types/enum_test.go similarity index 100% rename from types/enum_serial_test.go rename to types/enum_test.go diff --git a/types/mydecimal_serial_test.go b/types/mydecimal_serial_test.go deleted file mode 100644 index cd72ef7ce131a..0000000000000 --- a/types/mydecimal_serial_test.go +++ /dev/null @@ -1,199 +0,0 @@ -// Copyright 2021 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package types - -import ( - "fmt" - "testing" - - "github.com/stretchr/testify/require" -) - -// this test will change global variable `wordBufLen`, so it must run in serial -func TestShiftMyDecimal(t *testing.T) { - type tcase struct { - input string - shift int - output string - err error - } - - var dotest = func(tests []tcase) { - for _, test := range tests { - t.Run(fmt.Sprintf("%v (shift: %v, wordBufLen: %v)", test.input, test.shift, wordBufLen), func(t *testing.T) { - var dec MyDecimal - require.NoError(t, dec.FromString([]byte(test.input))) - require.Equal(t, test.err, dec.Shift(test.shift)) - require.Equal(t, test.output, string(dec.ToString())) - }) - } - } - - wordBufLen = maxWordBufLen - tests := []tcase{ - {"123.123", 1, "1231.23", nil}, - {"123457189.123123456789000", 1, "1234571891.23123456789", nil}, - {"123457189.123123456789000", 8, "12345718912312345.6789", nil}, - {"123457189.123123456789000", 9, "123457189123123456.789", nil}, - {"123457189.123123456789000", 10, "1234571891231234567.89", nil}, - {"123457189.123123456789000", 17, "12345718912312345678900000", nil}, - {"123457189.123123456789000", 18, "123457189123123456789000000", nil}, - {"123457189.123123456789000", 19, "1234571891231234567890000000", nil}, - {"123457189.123123456789000", 26, "12345718912312345678900000000000000", nil}, - {"123457189.123123456789000", 27, "123457189123123456789000000000000000", nil}, - {"123457189.123123456789000", 28, "1234571891231234567890000000000000000", nil}, - {"000000000000000000000000123457189.123123456789000", 26, "12345718912312345678900000000000000", nil}, - {"00000000123457189.123123456789000", 27, "123457189123123456789000000000000000", nil}, - {"00000000000000000123457189.123123456789000", 28, "1234571891231234567890000000000000000", nil}, - {"123", 1, "1230", nil}, - {"123", 10, "1230000000000", nil}, - {".123", 1, "1.23", nil}, - {".123", 10, "1230000000", nil}, - {".123", 14, "12300000000000", nil}, - {"000.000", 1000, "0", nil}, - {"000.", 1000, "0", nil}, - {".000", 1000, "0", nil}, - {"1", 1000, "1", ErrOverflow}, - {"123.123", -1, "12.3123", nil}, - {"123987654321.123456789000", -1, "12398765432.1123456789", nil}, - {"123987654321.123456789000", -2, "1239876543.21123456789", nil}, - {"123987654321.123456789000", -3, "123987654.321123456789", nil}, - {"123987654321.123456789000", -8, "1239.87654321123456789", nil}, - {"123987654321.123456789000", -9, "123.987654321123456789", nil}, - {"123987654321.123456789000", -10, "12.3987654321123456789", nil}, - {"123987654321.123456789000", -11, "1.23987654321123456789", nil}, - {"123987654321.123456789000", -12, "0.123987654321123456789", nil}, - {"123987654321.123456789000", -13, "0.0123987654321123456789", nil}, - {"123987654321.123456789000", -14, "0.00123987654321123456789", nil}, - {"00000087654321.123456789000", -14, "0.00000087654321123456789", nil}, - } - dotest(tests) - - wordBufLen = 2 - tests = []tcase{ - {"123.123", -2, "1.23123", nil}, - {"123.123", -3, "0.123123", nil}, - {"123.123", -6, "0.000123123", nil}, - {"123.123", -7, "0.0000123123", nil}, - {"123.123", -15, "0.000000000000123123", nil}, - {"123.123", -16, "0.000000000000012312", ErrTruncated}, - {"123.123", -17, "0.000000000000001231", ErrTruncated}, - {"123.123", -18, "0.000000000000000123", ErrTruncated}, - {"123.123", -19, "0.000000000000000012", ErrTruncated}, - {"123.123", -20, "0.000000000000000001", ErrTruncated}, - {"123.123", -21, "0", ErrTruncated}, - {".000000000123", -1, "0.0000000000123", nil}, - {".000000000123", -6, "0.000000000000000123", nil}, - {".000000000123", -7, "0.000000000000000012", ErrTruncated}, - {".000000000123", -8, "0.000000000000000001", ErrTruncated}, - {".000000000123", -9, "0", ErrTruncated}, - {".000000000123", 1, "0.00000000123", nil}, - {".000000000123", 8, "0.0123", nil}, - {".000000000123", 9, "0.123", nil}, - {".000000000123", 10, "1.23", nil}, - {".000000000123", 17, "12300000", nil}, - {".000000000123", 18, "123000000", nil}, - {".000000000123", 19, "1230000000", nil}, - {".000000000123", 20, "12300000000", nil}, - {".000000000123", 21, "123000000000", nil}, - {".000000000123", 22, "1230000000000", nil}, - {".000000000123", 23, "12300000000000", nil}, - {".000000000123", 24, "123000000000000", nil}, - {".000000000123", 25, "1230000000000000", nil}, - {".000000000123", 26, "12300000000000000", nil}, - {".000000000123", 27, "123000000000000000", nil}, - {".000000000123", 28, "0.000000000123", ErrOverflow}, - {"123456789.987654321", -1, "12345678.998765432", ErrTruncated}, - {"123456789.987654321", -2, "1234567.899876543", ErrTruncated}, - {"123456789.987654321", -8, "1.234567900", ErrTruncated}, - {"123456789.987654321", -9, "0.123456789987654321", nil}, - {"123456789.987654321", -10, "0.012345678998765432", ErrTruncated}, - {"123456789.987654321", -17, "0.000000001234567900", ErrTruncated}, - {"123456789.987654321", -18, "0.000000000123456790", ErrTruncated}, - {"123456789.987654321", -19, "0.000000000012345679", ErrTruncated}, - {"123456789.987654321", -26, "0.000000000000000001", ErrTruncated}, - {"123456789.987654321", -27, "0", ErrTruncated}, - {"123456789.987654321", 1, "1234567900", ErrTruncated}, - {"123456789.987654321", 2, "12345678999", ErrTruncated}, - {"123456789.987654321", 4, "1234567899877", ErrTruncated}, - {"123456789.987654321", 8, "12345678998765432", ErrTruncated}, - {"123456789.987654321", 9, "123456789987654321", nil}, - {"123456789.987654321", 10, "123456789.987654321", ErrOverflow}, - {"123456789.987654321", 0, "123456789.987654321", nil}, - } - dotest(tests) - - // reset - wordBufLen = maxWordBufLen -} - -// this test will change global variable `wordBufLen`, so it must run in serial -func TestFromStringMyDecimal(t *testing.T) { - type tcase struct { - input string - output string - err error - } - - var dotest = func(tests []tcase) { - for _, test := range tests { - t.Run(fmt.Sprintf("%v (wordBufLen: %v)", test.input, wordBufLen), func(t *testing.T) { - var dec MyDecimal - require.Equal(t, test.err, dec.FromString([]byte(test.input))) - require.Equal(t, test.output, string(dec.ToString())) - }) - } - } - - wordBufLen = maxWordBufLen - tests := []tcase{ - {"12345", "12345", nil}, - {"12345.", "12345", nil}, - {"123.45.", "123.45", ErrTruncated}, - {"-123.45.", "-123.45", ErrTruncated}, - {".00012345000098765", "0.00012345000098765", nil}, - {".12345000098765", "0.12345000098765", nil}, - {"-.000000012345000098765", "-0.000000012345000098765", nil}, - {"1234500009876.5", "1234500009876.5", nil}, - {"123E5", "12300000", nil}, - {"123E-2", "1.23", nil}, - {"1e1073741823", "999999999999999999999999999999999999999999999999999999999999999999999999999999999", ErrOverflow}, - {"-1e1073741823", "-999999999999999999999999999999999999999999999999999999999999999999999999999999999", ErrOverflow}, - {"1e18446744073709551620", "0", ErrBadNumber}, - {"1e", "1", ErrTruncated}, - {"1e001", "10", nil}, - {"1e00", "1", nil}, - {"1eabc", "1", ErrTruncated}, - {"1e 1dddd ", "10", ErrTruncated}, - {"1e - 1", "1", ErrTruncated}, - {"1e -1", "0.1", nil}, - {"0.00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000", "0.000000000000000000000000000000000000000000000000000000000000000000000000", ErrTruncated}, - {"1asf", "1", ErrTruncated}, - {"1.1.1.1.1", "1.1", ErrTruncated}, - {"1 1", "1", ErrTruncated}, - {"1 ", "1", nil}, - } - dotest(tests) - - wordBufLen = 1 - tests = []tcase{ - {"123450000098765", "98765", ErrOverflow}, - {"123450.000098765", "123450", ErrTruncated}, - } - dotest(tests) - - // reset - wordBufLen = maxWordBufLen -} diff --git a/types/mydecimal_test.go b/types/mydecimal_test.go index 61e86ffb1dc7a..13da9284ad395 100644 --- a/types/mydecimal_test.go +++ b/types/mydecimal_test.go @@ -15,6 +15,7 @@ package types import ( + "fmt" "strconv" "strings" "testing" @@ -814,3 +815,180 @@ func TestReset(t *testing.T) { require.NoError(t, DecimalAdd(&x2, &y2, &z1)) require.Equal(t, z2, z1) } + +// this test will change global variable `wordBufLen`, so it must run in serial +func TestShiftMyDecimal(t *testing.T) { + type tcase struct { + input string + shift int + output string + err error + } + + var dotest = func(tests []tcase) { + for _, test := range tests { + t.Run(fmt.Sprintf("%v (shift: %v, wordBufLen: %v)", test.input, test.shift, wordBufLen), func(t *testing.T) { + var dec MyDecimal + require.NoError(t, dec.FromString([]byte(test.input))) + require.Equal(t, test.err, dec.Shift(test.shift)) + require.Equal(t, test.output, string(dec.ToString())) + }) + } + } + + wordBufLen = maxWordBufLen + tests := []tcase{ + {"123.123", 1, "1231.23", nil}, + {"123457189.123123456789000", 1, "1234571891.23123456789", nil}, + {"123457189.123123456789000", 8, "12345718912312345.6789", nil}, + {"123457189.123123456789000", 9, "123457189123123456.789", nil}, + {"123457189.123123456789000", 10, "1234571891231234567.89", nil}, + {"123457189.123123456789000", 17, "12345718912312345678900000", nil}, + {"123457189.123123456789000", 18, "123457189123123456789000000", nil}, + {"123457189.123123456789000", 19, "1234571891231234567890000000", nil}, + {"123457189.123123456789000", 26, "12345718912312345678900000000000000", nil}, + {"123457189.123123456789000", 27, "123457189123123456789000000000000000", nil}, + {"123457189.123123456789000", 28, "1234571891231234567890000000000000000", nil}, + {"000000000000000000000000123457189.123123456789000", 26, "12345718912312345678900000000000000", nil}, + {"00000000123457189.123123456789000", 27, "123457189123123456789000000000000000", nil}, + {"00000000000000000123457189.123123456789000", 28, "1234571891231234567890000000000000000", nil}, + {"123", 1, "1230", nil}, + {"123", 10, "1230000000000", nil}, + {".123", 1, "1.23", nil}, + {".123", 10, "1230000000", nil}, + {".123", 14, "12300000000000", nil}, + {"000.000", 1000, "0", nil}, + {"000.", 1000, "0", nil}, + {".000", 1000, "0", nil}, + {"1", 1000, "1", ErrOverflow}, + {"123.123", -1, "12.3123", nil}, + {"123987654321.123456789000", -1, "12398765432.1123456789", nil}, + {"123987654321.123456789000", -2, "1239876543.21123456789", nil}, + {"123987654321.123456789000", -3, "123987654.321123456789", nil}, + {"123987654321.123456789000", -8, "1239.87654321123456789", nil}, + {"123987654321.123456789000", -9, "123.987654321123456789", nil}, + {"123987654321.123456789000", -10, "12.3987654321123456789", nil}, + {"123987654321.123456789000", -11, "1.23987654321123456789", nil}, + {"123987654321.123456789000", -12, "0.123987654321123456789", nil}, + {"123987654321.123456789000", -13, "0.0123987654321123456789", nil}, + {"123987654321.123456789000", -14, "0.00123987654321123456789", nil}, + {"00000087654321.123456789000", -14, "0.00000087654321123456789", nil}, + } + dotest(tests) + + wordBufLen = 2 + tests = []tcase{ + {"123.123", -2, "1.23123", nil}, + {"123.123", -3, "0.123123", nil}, + {"123.123", -6, "0.000123123", nil}, + {"123.123", -7, "0.0000123123", nil}, + {"123.123", -15, "0.000000000000123123", nil}, + {"123.123", -16, "0.000000000000012312", ErrTruncated}, + {"123.123", -17, "0.000000000000001231", ErrTruncated}, + {"123.123", -18, "0.000000000000000123", ErrTruncated}, + {"123.123", -19, "0.000000000000000012", ErrTruncated}, + {"123.123", -20, "0.000000000000000001", ErrTruncated}, + {"123.123", -21, "0", ErrTruncated}, + {".000000000123", -1, "0.0000000000123", nil}, + {".000000000123", -6, "0.000000000000000123", nil}, + {".000000000123", -7, "0.000000000000000012", ErrTruncated}, + {".000000000123", -8, "0.000000000000000001", ErrTruncated}, + {".000000000123", -9, "0", ErrTruncated}, + {".000000000123", 1, "0.00000000123", nil}, + {".000000000123", 8, "0.0123", nil}, + {".000000000123", 9, "0.123", nil}, + {".000000000123", 10, "1.23", nil}, + {".000000000123", 17, "12300000", nil}, + {".000000000123", 18, "123000000", nil}, + {".000000000123", 19, "1230000000", nil}, + {".000000000123", 20, "12300000000", nil}, + {".000000000123", 21, "123000000000", nil}, + {".000000000123", 22, "1230000000000", nil}, + {".000000000123", 23, "12300000000000", nil}, + {".000000000123", 24, "123000000000000", nil}, + {".000000000123", 25, "1230000000000000", nil}, + {".000000000123", 26, "12300000000000000", nil}, + {".000000000123", 27, "123000000000000000", nil}, + {".000000000123", 28, "0.000000000123", ErrOverflow}, + {"123456789.987654321", -1, "12345678.998765432", ErrTruncated}, + {"123456789.987654321", -2, "1234567.899876543", ErrTruncated}, + {"123456789.987654321", -8, "1.234567900", ErrTruncated}, + {"123456789.987654321", -9, "0.123456789987654321", nil}, + {"123456789.987654321", -10, "0.012345678998765432", ErrTruncated}, + {"123456789.987654321", -17, "0.000000001234567900", ErrTruncated}, + {"123456789.987654321", -18, "0.000000000123456790", ErrTruncated}, + {"123456789.987654321", -19, "0.000000000012345679", ErrTruncated}, + {"123456789.987654321", -26, "0.000000000000000001", ErrTruncated}, + {"123456789.987654321", -27, "0", ErrTruncated}, + {"123456789.987654321", 1, "1234567900", ErrTruncated}, + {"123456789.987654321", 2, "12345678999", ErrTruncated}, + {"123456789.987654321", 4, "1234567899877", ErrTruncated}, + {"123456789.987654321", 8, "12345678998765432", ErrTruncated}, + {"123456789.987654321", 9, "123456789987654321", nil}, + {"123456789.987654321", 10, "123456789.987654321", ErrOverflow}, + {"123456789.987654321", 0, "123456789.987654321", nil}, + } + dotest(tests) + + // reset + wordBufLen = maxWordBufLen +} + +// this test will change global variable `wordBufLen`, so it must run in serial +func TestFromStringMyDecimal(t *testing.T) { + type tcase struct { + input string + output string + err error + } + + var dotest = func(tests []tcase) { + for _, test := range tests { + t.Run(fmt.Sprintf("%v (wordBufLen: %v)", test.input, wordBufLen), func(t *testing.T) { + var dec MyDecimal + require.Equal(t, test.err, dec.FromString([]byte(test.input))) + require.Equal(t, test.output, string(dec.ToString())) + }) + } + } + + wordBufLen = maxWordBufLen + tests := []tcase{ + {"12345", "12345", nil}, + {"12345.", "12345", nil}, + {"123.45.", "123.45", ErrTruncated}, + {"-123.45.", "-123.45", ErrTruncated}, + {".00012345000098765", "0.00012345000098765", nil}, + {".12345000098765", "0.12345000098765", nil}, + {"-.000000012345000098765", "-0.000000012345000098765", nil}, + {"1234500009876.5", "1234500009876.5", nil}, + {"123E5", "12300000", nil}, + {"123E-2", "1.23", nil}, + {"1e1073741823", "999999999999999999999999999999999999999999999999999999999999999999999999999999999", ErrOverflow}, + {"-1e1073741823", "-999999999999999999999999999999999999999999999999999999999999999999999999999999999", ErrOverflow}, + {"1e18446744073709551620", "0", ErrBadNumber}, + {"1e", "1", ErrTruncated}, + {"1e001", "10", nil}, + {"1e00", "1", nil}, + {"1eabc", "1", ErrTruncated}, + {"1e 1dddd ", "10", ErrTruncated}, + {"1e - 1", "1", ErrTruncated}, + {"1e -1", "0.1", nil}, + {"0.00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000", "0.000000000000000000000000000000000000000000000000000000000000000000000000", ErrTruncated}, + {"1asf", "1", ErrTruncated}, + {"1.1.1.1.1", "1.1", ErrTruncated}, + {"1 1", "1", ErrTruncated}, + {"1 ", "1", nil}, + } + dotest(tests) + + wordBufLen = 1 + tests = []tcase{ + {"123450000098765", "98765", ErrOverflow}, + {"123450.000098765", "123450", ErrTruncated}, + } + dotest(tests) + + // reset + wordBufLen = maxWordBufLen +} diff --git a/types/set_serial_test.go b/types/set_test.go similarity index 100% rename from types/set_serial_test.go rename to types/set_test.go From 404895c04a5183d6bf759cc910e642475f4f9f2b Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Thu, 23 Dec 2021 21:57:47 +0800 Subject: [PATCH 37/44] executor: fix the returned field count of the prepare statement (#30981) close pingcap/tidb#30971 --- executor/executor_test.go | 26 ++++++++++++++++++++++++++ executor/prepared.go | 2 +- 2 files changed, 27 insertions(+), 1 deletion(-) diff --git a/executor/executor_test.go b/executor/executor_test.go index 02feb44abaa65..d4b11b0758d8b 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -9686,3 +9686,29 @@ func (s *testSerialSuite) TestUnreasonablyClose(c *C) { } c.Assert(opsAlreadyCoveredMask, Equals, opsNeedsCoveredMask, Commentf("these operators are not covered %s", commentBuf.String())) } + +func (s *testSerialSuite) TestIssue30971(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t1, t2") + tk.MustExec("create table t1 (id int);") + tk.MustExec("create table t2 (id int, c int);") + + testCases := []struct { + sql string + fields int + }{ + // Fix a bug that the column length field returned to client is incorrect using MySQL prepare protocol. + {"select * from t1 union select 1 from t1", 1}, + {"select c from t2 union select * from t1", 1}, + {"select * from t1", 1}, + {"select * from t2 where c in (select * from t1)", 2}, + {"insert into t1 values (?)", 0}, + {"update t1 set id = ?", 0}, + } + for _, test := range testCases { + _, _, fields, err := tk.Se.PrepareStmt(test.sql) + c.Assert(err, IsNil) + c.Assert(fields, HasLen, test.fields) + } +} diff --git a/executor/prepared.go b/executor/prepared.go index e18495d45c601..0676a77215f2e 100644 --- a/executor/prepared.go +++ b/executor/prepared.go @@ -220,7 +220,7 @@ func (e *PrepareExec) Next(ctx context.Context, req *chunk.Chunk) error { if err != nil { return err } - if _, ok := stmt.(*ast.SelectStmt); ok { + if p.Schema().Len() > 0 { e.Fields = colNames2ResultFields(p.Schema(), p.OutputNames(), vars.CurrentDB) } if e.ID == 0 { From 6e6db1fd356e872f71d724467e9ec02f7c6979d7 Mon Sep 17 00:00:00 2001 From: qupeng Date: Fri, 24 Dec 2021 12:55:47 +0800 Subject: [PATCH 38/44] binlog: allow multiple ddl targets (#30904) --- ddl/table.go | 7 +++-- ddl/table_test.go | 67 +++++++++++++++++++++++++++++++++++++++++++++ parser/model/ddl.go | 14 ++++++++++ 3 files changed, 85 insertions(+), 3 deletions(-) diff --git a/ddl/table.go b/ddl/table.go index 625b4f39df759..a60e5d9a76b20 100644 --- a/ddl/table.go +++ b/ddl/table.go @@ -798,21 +798,22 @@ func onRenameTables(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error return ver, errors.Trace(err) } - tblInfo := &model.TableInfo{} + var tblInfos = make([]*model.TableInfo, 0, len(tableNames)) var err error for i, oldSchemaID := range oldSchemaIDs { job.TableID = tableIDs[i] - ver, tblInfo, err = checkAndRenameTables(t, job, oldSchemaID, newSchemaIDs[i], oldSchemaNames[i], tableNames[i]) + ver, tblInfo, err := checkAndRenameTables(t, job, oldSchemaID, newSchemaIDs[i], oldSchemaNames[i], tableNames[i]) if err != nil { return ver, errors.Trace(err) } + tblInfos = append(tblInfos, tblInfo) } ver, err = updateSchemaVersion(t, job) if err != nil { return ver, errors.Trace(err) } - job.FinishTableJob(model.JobStateDone, model.StatePublic, ver, tblInfo) + job.FinishMultipleTableJob(model.JobStateDone, model.StatePublic, ver, tblInfos) return ver, nil } diff --git a/ddl/table_test.go b/ddl/table_test.go index f366c090b1686..78be85a8dbd97 100644 --- a/ddl/table_test.go +++ b/ddl/table_test.go @@ -16,6 +16,7 @@ package ddl import ( "context" + "fmt" "testing" "github.com/pingcap/errors" @@ -66,6 +67,24 @@ func testRenameTable(t *testing.T, ctx sessionctx.Context, d *ddl, newSchemaID, return job } +func testRenameTables( + t *testing.T, ctx sessionctx.Context, d *ddl, + oldSchemaIDs, newSchemaIDs []int64, newTableNames []*model.CIStr, + oldTableIDs []int64, oldSchemaNames []*model.CIStr, +) *model.Job { + job := &model.Job{ + Type: model.ActionRenameTables, + BinlogInfo: &model.HistoryInfo{}, + Args: []interface{}{oldSchemaIDs, newSchemaIDs, newTableNames, oldTableIDs, oldSchemaNames}, + } + err := d.doDDLJob(ctx, job) + require.NoError(t, err) + + v := getSchemaVerT(t, ctx) + checkHistoryJobArgsT(t, ctx, job.ID, &historyJobArgs{ver: v, tbl: nil}) + return job +} + func testLockTable(t *testing.T, ctx sessionctx.Context, d *ddl, newSchemaID int64, tblInfo *model.TableInfo, lockTp model.TableLockType) *model.Job { arg := &lockTablesArg{ LockTables: []model.TableLockTpInfo{{SchemaID: newSchemaID, TableID: tblInfo.ID, Tp: lockTp}}, @@ -326,3 +345,51 @@ func testAlterNoCacheTable(t *testing.T, ctx sessionctx.Context, d *ddl, newSche checkHistoryJobArgsT(t, ctx, job.ID, &historyJobArgs{ver: v}) return job } + +func TestRenameTables(t *testing.T) { + store, err := mockstore.NewMockStore() + require.NoError(t, err) + ddl, err := testNewDDLAndStart( + context.Background(), + WithStore(store), + WithLease(testLease), + ) + require.NoError(t, err) + + dbInfo, err := testSchemaInfo(ddl, "test_table") + require.NoError(t, err) + testCreateSchemaT(t, testNewContext(ddl), ddl, dbInfo) + + ctx := testNewContext(ddl) + var tblInfos = make([]*model.TableInfo, 0, 2) + var newTblInfos = make([]*model.TableInfo, 0, 2) + for i := 1; i < 3; i++ { + tableName := fmt.Sprintf("t%d", i) + tblInfo, err := testTableInfo(ddl, tableName, 3) + require.NoError(t, err) + job := testCreateTableT(t, ctx, ddl, dbInfo, tblInfo) + testCheckTableStateT(t, ddl, dbInfo, tblInfo, model.StatePublic) + testCheckJobDoneT(t, ddl, job, true) + tblInfos = append(tblInfos, tblInfo) + + newTableName := fmt.Sprintf("tt%d", i) + tblInfo, err = testTableInfo(ddl, newTableName, 3) + require.NoError(t, err) + newTblInfos = append(newTblInfos, tblInfo) + } + + job := testRenameTables( + t, ctx, ddl, + []int64{dbInfo.ID, dbInfo.ID}, + []int64{dbInfo.ID, dbInfo.ID}, + []*model.CIStr{&newTblInfos[0].Name, &newTblInfos[1].Name}, + []int64{tblInfos[0].ID, tblInfos[1].ID}, + []*model.CIStr{&dbInfo.Name, &dbInfo.Name}, + ) + + txn, _ := ctx.Txn(true) + historyJob, _ := meta.NewMeta(txn).GetHistoryDDLJob(job.ID) + wantTblInfos := historyJob.BinlogInfo.MultipleTableInfos + require.Equal(t, wantTblInfos[0].Name.L, "tt1") + require.Equal(t, wantTblInfos[1].Name.L, "tt2") +} diff --git a/parser/model/ddl.go b/parser/model/ddl.go index c61372b55e263..9716cea38cd23 100644 --- a/parser/model/ddl.go +++ b/parser/model/ddl.go @@ -175,6 +175,9 @@ type HistoryInfo struct { DBInfo *DBInfo TableInfo *TableInfo FinishedTS uint64 + + // MultipleTableInfos is like TableInfo but only for operations updating multiple tables. + MultipleTableInfos []*TableInfo } // AddDBInfo adds schema version and schema information that are used for binlog. @@ -196,6 +199,7 @@ func (h *HistoryInfo) Clean() { h.SchemaVersion = 0 h.DBInfo = nil h.TableInfo = nil + h.MultipleTableInfos = nil } // DDLReorgMeta is meta info of DDL reorganization. @@ -279,6 +283,16 @@ func (job *Job) FinishTableJob(jobState JobState, schemaState SchemaState, ver i job.BinlogInfo.AddTableInfo(ver, tblInfo) } +// FinishMultipleTableJob is called when a job is finished. +// It updates the job's state information and adds tblInfos to the binlog. +func (job *Job) FinishMultipleTableJob(jobState JobState, schemaState SchemaState, ver int64, tblInfos []*TableInfo) { + job.State = jobState + job.SchemaState = schemaState + job.BinlogInfo.SchemaVersion = ver + job.BinlogInfo.MultipleTableInfos = tblInfos + job.BinlogInfo.TableInfo = tblInfos[len(tblInfos)-1] +} + // FinishDBJob is called when a job is finished. // It updates the job's state information and adds dbInfo the binlog. func (job *Job) FinishDBJob(jobState JobState, schemaState SchemaState, ver int64, dbInfo *DBInfo) { From 30c5f5ba59ebdab6cbc5a20940ed541953c929f7 Mon Sep 17 00:00:00 2001 From: Song Gao Date: Fri, 24 Dec 2021 13:15:47 +0800 Subject: [PATCH 39/44] planner: trace predicate push down (#30902) ref pingcap/tidb#29661 --- planner/core/logical_plan_trace_test.go | 38 ++++++ planner/core/plan.go | 2 +- planner/core/rule_predicate_push_down.go | 149 ++++++++++++++++++----- 3 files changed, 156 insertions(+), 33 deletions(-) diff --git a/planner/core/logical_plan_trace_test.go b/planner/core/logical_plan_trace_test.go index c09ffb1f19b6c..a315e9a23ea3f 100644 --- a/planner/core/logical_plan_trace_test.go +++ b/planner/core/logical_plan_trace_test.go @@ -86,6 +86,44 @@ func (s *testPlanSuite) TestSingleRuleTraceStep(c *C) { assertRuleName string assertRuleSteps []assertTraceStep }{ + { + sql: "select * from t as t1 join t as t2 on t1.a = t2.a where t1.a < 1;", + flags: []uint64{flagPredicatePushDown, flagBuildKeyInfo, flagPrunColumns}, + assertRuleName: "predicate_push_down", + assertRuleSteps: []assertTraceStep{ + { + assertReason: "", + assertAction: "The conditions[lt(test.t.a, 1)] are pushed down across DataSource_1", + }, + { + assertReason: "", + assertAction: "The conditions[lt(test.t.a, 1)] are pushed down across DataSource_2", + }, + { + assertAction: "Selection_4 is removed", + assertReason: "The conditions[eq(test.t.a, test.t.a)] in Selection_4 are pushed down", + }, + { + assertAction: "Selection_5 is removed", + assertReason: "The conditions[lt(test.t.a, 1)] in Selection_5 are pushed down", + }, + }, + }, + { + sql: "select * from t where a < 1;", + flags: []uint64{flagPredicatePushDown, flagBuildKeyInfo, flagPrunColumns}, + assertRuleName: "predicate_push_down", + assertRuleSteps: []assertTraceStep{ + { + assertReason: "", + assertAction: "The conditions[lt(test.t.a, 1)] are pushed down across DataSource_1", + }, + { + assertReason: "The conditions[lt(test.t.a, 1)] in Selection_2 are pushed down", + assertAction: "Selection_2 is removed", + }, + }, + }, { sql: "select * from t as t1 left join t as t2 on t1.a = t2.a order by t1.a limit 10;", flags: []uint64{flagPrunColumns, flagBuildKeyInfo, flagPushDownTopN}, diff --git a/planner/core/plan.go b/planner/core/plan.go index 4c2c0c6ba93c5..fed517996af1a 100644 --- a/planner/core/plan.go +++ b/planner/core/plan.go @@ -237,7 +237,7 @@ type LogicalPlan interface { // PredicatePushDown pushes down the predicates in the where/on/having clauses as deeply as possible. // It will accept a predicate that is an expression slice, and return the expressions that can't be pushed. // Because it might change the root if the having clause exists, we need to return a plan that represents a new root. - PredicatePushDown([]expression.Expression) ([]expression.Expression, LogicalPlan) + PredicatePushDown([]expression.Expression, *logicalOptimizeOp) ([]expression.Expression, LogicalPlan) // PruneColumns prunes the unused columns. PruneColumns([]*expression.Column) error diff --git a/planner/core/rule_predicate_push_down.go b/planner/core/rule_predicate_push_down.go index f59fedc25ba28..feed34d7ee567 100644 --- a/planner/core/rule_predicate_push_down.go +++ b/planner/core/rule_predicate_push_down.go @@ -15,7 +15,9 @@ package core import ( + "bytes" "context" + "fmt" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/kv" @@ -28,11 +30,11 @@ import ( type ppdSolver struct{} func (s *ppdSolver) optimize(ctx context.Context, lp LogicalPlan, opt *logicalOptimizeOp) (LogicalPlan, error) { - _, p := lp.PredicatePushDown(nil) + _, p := lp.PredicatePushDown(nil, opt) return p, nil } -func addSelection(p LogicalPlan, child LogicalPlan, conditions []expression.Expression, chIdx int) { +func addSelection(p LogicalPlan, child LogicalPlan, conditions []expression.Expression, chIdx int, opt *logicalOptimizeOp) { if len(conditions) == 0 { p.Children()[chIdx] = child return @@ -42,6 +44,7 @@ func addSelection(p LogicalPlan, child LogicalPlan, conditions []expression.Expr dual := Conds2TableDual(child, conditions) if dual != nil { p.Children()[chIdx] = dual + appendTableDualTraceStep(child, dual, conditions, opt) return } @@ -53,16 +56,17 @@ func addSelection(p LogicalPlan, child LogicalPlan, conditions []expression.Expr selection := LogicalSelection{Conditions: conditions}.Init(p.SCtx(), p.SelectBlockOffset()) selection.SetChildren(child) p.Children()[chIdx] = selection + appendAddSelectionTraceStep(p, child, selection, opt) } // PredicatePushDown implements LogicalPlan interface. -func (p *baseLogicalPlan) PredicatePushDown(predicates []expression.Expression) ([]expression.Expression, LogicalPlan) { +func (p *baseLogicalPlan) PredicatePushDown(predicates []expression.Expression, opt *logicalOptimizeOp) ([]expression.Expression, LogicalPlan) { if len(p.children) == 0 { return predicates, p.self } child := p.children[0] - rest, newChild := child.PredicatePushDown(predicates) - addSelection(p.self, newChild, rest, 0) + rest, newChild := child.PredicatePushDown(predicates, opt) + addSelection(p.self, newChild, rest, 0, opt) return nil, p.self } @@ -80,17 +84,19 @@ func splitSetGetVarFunc(filters []expression.Expression) ([]expression.Expressio } // PredicatePushDown implements LogicalPlan PredicatePushDown interface. -func (p *LogicalSelection) PredicatePushDown(predicates []expression.Expression) ([]expression.Expression, LogicalPlan) { +func (p *LogicalSelection) PredicatePushDown(predicates []expression.Expression, opt *logicalOptimizeOp) ([]expression.Expression, LogicalPlan) { predicates = DeleteTrueExprs(p, predicates) p.Conditions = DeleteTrueExprs(p, p.Conditions) var child LogicalPlan var retConditions []expression.Expression + var originConditions []expression.Expression if p.buildByHaving { - retConditions, child = p.children[0].PredicatePushDown(predicates) + retConditions, child = p.children[0].PredicatePushDown(predicates, opt) retConditions = append(retConditions, p.Conditions...) } else { canBePushDown, canNotBePushDown := splitSetGetVarFunc(p.Conditions) - retConditions, child = p.children[0].PredicatePushDown(append(canBePushDown, predicates...)) + originConditions = canBePushDown + retConditions, child = p.children[0].PredicatePushDown(append(canBePushDown, predicates...), opt) retConditions = append(retConditions, canNotBePushDown...) } if len(retConditions) > 0 { @@ -98,16 +104,18 @@ func (p *LogicalSelection) PredicatePushDown(predicates []expression.Expression) // Return table dual when filter is constant false or null. dual := Conds2TableDual(p, p.Conditions) if dual != nil { + appendTableDualTraceStep(p, dual, p.Conditions, opt) return nil, dual } return nil, p } + appendSelectionPredicatePushDownTraceStep(p, originConditions, opt) return nil, child } // PredicatePushDown implements LogicalPlan PredicatePushDown interface. -func (p *LogicalUnionScan) PredicatePushDown(predicates []expression.Expression) ([]expression.Expression, LogicalPlan) { - retainedPredicates, _ := p.children[0].PredicatePushDown(predicates) +func (p *LogicalUnionScan) PredicatePushDown(predicates []expression.Expression, opt *logicalOptimizeOp) ([]expression.Expression, LogicalPlan) { + retainedPredicates, _ := p.children[0].PredicatePushDown(predicates, opt) p.conditions = make([]expression.Expression, 0, len(predicates)) p.conditions = append(p.conditions, predicates...) // The conditions in UnionScan is only used for added rows, so parent Selection should not be removed. @@ -115,21 +123,22 @@ func (p *LogicalUnionScan) PredicatePushDown(predicates []expression.Expression) } // PredicatePushDown implements LogicalPlan PredicatePushDown interface. -func (ds *DataSource) PredicatePushDown(predicates []expression.Expression) ([]expression.Expression, LogicalPlan) { +func (ds *DataSource) PredicatePushDown(predicates []expression.Expression, opt *logicalOptimizeOp) ([]expression.Expression, LogicalPlan) { predicates = expression.PropagateConstant(ds.ctx, predicates) predicates = DeleteTrueExprs(ds, predicates) ds.allConds = predicates ds.pushedDownConds, predicates = expression.PushDownExprs(ds.ctx.GetSessionVars().StmtCtx, predicates, ds.ctx.GetClient(), kv.UnSpecified) + appendDataSourcePredicatePushDownTraceStep(ds, opt) return predicates, ds } // PredicatePushDown implements LogicalPlan PredicatePushDown interface. -func (p *LogicalTableDual) PredicatePushDown(predicates []expression.Expression) ([]expression.Expression, LogicalPlan) { +func (p *LogicalTableDual) PredicatePushDown(predicates []expression.Expression, opt *logicalOptimizeOp) ([]expression.Expression, LogicalPlan) { return predicates, p } // PredicatePushDown implements LogicalPlan PredicatePushDown interface. -func (p *LogicalJoin) PredicatePushDown(predicates []expression.Expression) (ret []expression.Expression, retPlan LogicalPlan) { +func (p *LogicalJoin) PredicatePushDown(predicates []expression.Expression, opt *logicalOptimizeOp) (ret []expression.Expression, retPlan LogicalPlan) { simplifyOuterJoin(p, predicates) var equalCond []*expression.ScalarFunction var leftPushCond, rightPushCond, otherCond, leftCond, rightCond []expression.Expression @@ -138,6 +147,7 @@ func (p *LogicalJoin) PredicatePushDown(predicates []expression.Expression) (ret predicates = p.outerJoinPropConst(predicates) dual := Conds2TableDual(p, predicates) if dual != nil { + appendTableDualTraceStep(p, dual, predicates, opt) return ret, dual } // Handle where conditions @@ -156,6 +166,7 @@ func (p *LogicalJoin) PredicatePushDown(predicates []expression.Expression) (ret predicates = p.outerJoinPropConst(predicates) dual := Conds2TableDual(p, predicates) if dual != nil { + appendTableDualTraceStep(p, dual, predicates, opt) return ret, dual } // Handle where conditions @@ -182,6 +193,7 @@ func (p *LogicalJoin) PredicatePushDown(predicates []expression.Expression) (ret // Return table dual when filter is constant false or null. dual := Conds2TableDual(p, tempCond) if dual != nil { + appendTableDualTraceStep(p, dual, tempCond, opt) return ret, dual } equalCond, leftPushCond, rightPushCond, otherCond = p.extractOnCondition(tempCond, true, true) @@ -196,6 +208,7 @@ func (p *LogicalJoin) PredicatePushDown(predicates []expression.Expression) (ret // Return table dual when filter is constant false or null. dual := Conds2TableDual(p, predicates) if dual != nil { + appendTableDualTraceStep(p, dual, predicates, opt) return ret, dual } // `predicates` should only contain left conditions or constant filters. @@ -212,10 +225,10 @@ func (p *LogicalJoin) PredicatePushDown(predicates []expression.Expression) (ret } leftCond = expression.RemoveDupExprs(p.ctx, leftCond) rightCond = expression.RemoveDupExprs(p.ctx, rightCond) - leftRet, lCh := p.children[0].PredicatePushDown(leftCond) - rightRet, rCh := p.children[1].PredicatePushDown(rightCond) - addSelection(p, lCh, leftRet, 0) - addSelection(p, rCh, rightRet, 1) + leftRet, lCh := p.children[0].PredicatePushDown(leftCond, opt) + rightRet, rCh := p.children[1].PredicatePushDown(rightCond, opt) + addSelection(p, lCh, leftRet, 0, opt) + addSelection(p, rCh, rightRet, 1, opt) p.updateEQCond() buildKeyInfo(p) return ret, p.self @@ -380,12 +393,12 @@ func isNullRejected(ctx sessionctx.Context, schema *expression.Schema, expr expr } // PredicatePushDown implements LogicalPlan PredicatePushDown interface. -func (p *LogicalProjection) PredicatePushDown(predicates []expression.Expression) (ret []expression.Expression, retPlan LogicalPlan) { +func (p *LogicalProjection) PredicatePushDown(predicates []expression.Expression, opt *logicalOptimizeOp) (ret []expression.Expression, retPlan LogicalPlan) { canBePushed := make([]expression.Expression, 0, len(predicates)) canNotBePushed := make([]expression.Expression, 0, len(predicates)) for _, expr := range p.Exprs { if expression.HasAssignSetVarFunc(expr) { - _, child := p.baseLogicalPlan.PredicatePushDown(nil) + _, child := p.baseLogicalPlan.PredicatePushDown(nil, opt) return predicates, child } } @@ -397,23 +410,23 @@ func (p *LogicalProjection) PredicatePushDown(predicates []expression.Expression canNotBePushed = append(canNotBePushed, cond) } } - remained, child := p.baseLogicalPlan.PredicatePushDown(canBePushed) + remained, child := p.baseLogicalPlan.PredicatePushDown(canBePushed, opt) return append(remained, canNotBePushed...), child } // PredicatePushDown implements LogicalPlan PredicatePushDown interface. -func (p *LogicalUnionAll) PredicatePushDown(predicates []expression.Expression) (ret []expression.Expression, retPlan LogicalPlan) { +func (p *LogicalUnionAll) PredicatePushDown(predicates []expression.Expression, opt *logicalOptimizeOp) (ret []expression.Expression, retPlan LogicalPlan) { for i, proj := range p.children { newExprs := make([]expression.Expression, 0, len(predicates)) newExprs = append(newExprs, predicates...) - retCond, newChild := proj.PredicatePushDown(newExprs) - addSelection(p, newChild, retCond, i) + retCond, newChild := proj.PredicatePushDown(newExprs, opt) + addSelection(p, newChild, retCond, i, opt) } return nil, p } // PredicatePushDown implements LogicalPlan PredicatePushDown interface. -func (la *LogicalAggregation) PredicatePushDown(predicates []expression.Expression) (ret []expression.Expression, retPlan LogicalPlan) { +func (la *LogicalAggregation) PredicatePushDown(predicates []expression.Expression, opt *logicalOptimizeOp) (ret []expression.Expression, retPlan LogicalPlan) { var condsToPush []expression.Expression exprsOriginal := make([]expression.Expression, 0, len(la.AggFuncs)) for _, fun := range la.AggFuncs { @@ -447,21 +460,21 @@ func (la *LogicalAggregation) PredicatePushDown(predicates []expression.Expressi ret = append(ret, cond) } } - la.baseLogicalPlan.PredicatePushDown(condsToPush) + la.baseLogicalPlan.PredicatePushDown(condsToPush, opt) return ret, la } // PredicatePushDown implements LogicalPlan PredicatePushDown interface. -func (p *LogicalLimit) PredicatePushDown(predicates []expression.Expression) ([]expression.Expression, LogicalPlan) { +func (p *LogicalLimit) PredicatePushDown(predicates []expression.Expression, opt *logicalOptimizeOp) ([]expression.Expression, LogicalPlan) { // Limit forbids any condition to push down. - p.baseLogicalPlan.PredicatePushDown(nil) + p.baseLogicalPlan.PredicatePushDown(nil, opt) return predicates, p } // PredicatePushDown implements LogicalPlan PredicatePushDown interface. -func (p *LogicalMaxOneRow) PredicatePushDown(predicates []expression.Expression) ([]expression.Expression, LogicalPlan) { +func (p *LogicalMaxOneRow) PredicatePushDown(predicates []expression.Expression, opt *logicalOptimizeOp) ([]expression.Expression, LogicalPlan) { // MaxOneRow forbids any condition to push down. - p.baseLogicalPlan.PredicatePushDown(nil) + p.baseLogicalPlan.PredicatePushDown(nil, opt) return predicates, p } @@ -610,7 +623,7 @@ func (p *LogicalWindow) GetPartitionByCols() []*expression.Column { } // PredicatePushDown implements LogicalPlan PredicatePushDown interface. -func (p *LogicalWindow) PredicatePushDown(predicates []expression.Expression) ([]expression.Expression, LogicalPlan) { +func (p *LogicalWindow) PredicatePushDown(predicates []expression.Expression, opt *logicalOptimizeOp) ([]expression.Expression, LogicalPlan) { canBePushed := make([]expression.Expression, 0, len(predicates)) canNotBePushed := make([]expression.Expression, 0, len(predicates)) partitionCols := expression.NewSchema(p.GetPartitionByCols()...) @@ -623,12 +636,12 @@ func (p *LogicalWindow) PredicatePushDown(predicates []expression.Expression) ([ canNotBePushed = append(canNotBePushed, cond) } } - p.baseLogicalPlan.PredicatePushDown(canBePushed) + p.baseLogicalPlan.PredicatePushDown(canBePushed, opt) return canNotBePushed, p } // PredicatePushDown implements LogicalPlan PredicatePushDown interface. -func (p *LogicalMemTable) PredicatePushDown(predicates []expression.Expression) ([]expression.Expression, LogicalPlan) { +func (p *LogicalMemTable) PredicatePushDown(predicates []expression.Expression, opt *logicalOptimizeOp) ([]expression.Expression, LogicalPlan) { if p.Extractor != nil { predicates = p.Extractor.Extract(p.ctx, p.schema, p.names, predicates) } @@ -638,3 +651,75 @@ func (p *LogicalMemTable) PredicatePushDown(predicates []expression.Expression) func (*ppdSolver) name() string { return "predicate_push_down" } + +func appendTableDualTraceStep(replaced LogicalPlan, dual LogicalPlan, conditions []expression.Expression, opt *logicalOptimizeOp) { + action := func() string { + return fmt.Sprintf("%v_%v is replaced by %v_%v", replaced.TP(), replaced.ID(), dual.TP(), dual.ID()) + } + reason := func() string { + buffer := bytes.NewBufferString("The conditions[") + for i, cond := range conditions { + if i > 0 { + buffer.WriteString(",") + } + buffer.WriteString(cond.String()) + } + buffer.WriteString("] are constant false or null") + return buffer.String() + } + opt.appendStepToCurrent(dual.ID(), dual.TP(), reason, action) +} + +func appendSelectionPredicatePushDownTraceStep(p *LogicalSelection, conditions []expression.Expression, opt *logicalOptimizeOp) { + action := func() string { + return fmt.Sprintf("%v_%v is removed", p.TP(), p.ID()) + } + reason := func() string { + return "" + } + if len(conditions) > 0 && !p.buildByHaving { + reason = func() string { + buffer := bytes.NewBufferString("The conditions[") + for i, cond := range conditions { + if i > 0 { + buffer.WriteString(",") + } + buffer.WriteString(cond.String()) + } + buffer.WriteString(fmt.Sprintf("] in %v_%v are pushed down", p.TP(), p.ID())) + return buffer.String() + } + } + opt.appendStepToCurrent(p.ID(), p.TP(), reason, action) +} + +func appendDataSourcePredicatePushDownTraceStep(ds *DataSource, opt *logicalOptimizeOp) { + if len(ds.pushedDownConds) < 1 { + return + } + reason := func() string { + return "" + } + action := func() string { + buffer := bytes.NewBufferString("The conditions[") + for i, cond := range ds.pushedDownConds { + if i > 0 { + buffer.WriteString(",") + } + buffer.WriteString(cond.String()) + } + buffer.WriteString(fmt.Sprintf("] are pushed down across %v_%v", ds.TP(), ds.ID())) + return buffer.String() + } + opt.appendStepToCurrent(ds.ID(), ds.TP(), reason, action) +} + +func appendAddSelectionTraceStep(p LogicalPlan, child LogicalPlan, sel *LogicalSelection, opt *logicalOptimizeOp) { + reason := func() string { + return "" + } + action := func() string { + return fmt.Sprintf("add %v_%v to connect %v_%v and %v_%v", sel.TP(), sel.ID(), p.TP(), p.ID(), child.TP(), child.ID()) + } + opt.appendStepToCurrent(sel.ID(), sel.TP(), reason, action) +} From 93d28b9d3ffe2726640a72342a3c517f88bbd37d Mon Sep 17 00:00:00 2001 From: xhe Date: Fri, 24 Dec 2021 14:55:47 +0800 Subject: [PATCH 40/44] placement: give default 2 followers for non-sugar syntax (#31000) --- ddl/placement/bundle.go | 36 +++++++++++++----------------------- ddl/placement/bundle_test.go | 5 ++++- parser/parser.go | 7 ++++++- parser/parser.y | 7 ++++++- parser/parser_test.go | 6 ++++++ 5 files changed, 35 insertions(+), 26 deletions(-) diff --git a/ddl/placement/bundle.go b/ddl/placement/bundle.go index fe160c7653565..02e8bdcd5a4c9 100644 --- a/ddl/placement/bundle.go +++ b/ddl/placement/bundle.go @@ -85,33 +85,23 @@ func NewBundleFromConstraintsOptions(options *model.PlacementSettings) (*Bundle, return nil, fmt.Errorf("%w: LeaderConstraints conflicts with Constraints", err) } } - if len(LeaderConstraints) > 0 { - Rules = append(Rules, NewRule(Leader, 1, LeaderConstraints)) - } else if followerCount == 0 { - return nil, fmt.Errorf("%w: you must at least provide common/leader constraints, or set some followers", ErrInvalidPlacementOptions) - } - - if followerCount > 0 { - // if user did not specify leader, add one - if len(LeaderConstraints) == 0 { - Rules = append(Rules, NewRule(Leader, 1, NewConstraintsDirect())) - } + Rules = append(Rules, NewRule(Leader, 1, LeaderConstraints)) - FollowerRules, err := NewRules(Voter, followerCount, followerConstraints) - if err != nil { - return nil, fmt.Errorf("%w: invalid FollowerConstraints", err) - } - for _, rule := range FollowerRules { - for _, cnst := range CommonConstraints { - if err := rule.Constraints.Add(cnst); err != nil { - return nil, fmt.Errorf("%w: FollowerConstraints conflicts with Constraints", err) - } + if followerCount == 0 { + followerCount = 2 + } + FollowerRules, err := NewRules(Voter, followerCount, followerConstraints) + if err != nil { + return nil, fmt.Errorf("%w: invalid FollowerConstraints", err) + } + for _, rule := range FollowerRules { + for _, cnst := range CommonConstraints { + if err := rule.Constraints.Add(cnst); err != nil { + return nil, fmt.Errorf("%w: FollowerConstraints conflicts with Constraints", err) } } - Rules = append(Rules, FollowerRules...) - } else if followerConstraints != "" { - return nil, fmt.Errorf("%w: specify follower constraints without specify how many followers to be placed", ErrInvalidPlacementOptions) } + Rules = append(Rules, FollowerRules...) if learnerCount > 0 { LearnerRules, err := NewRules(Learner, learnerCount, learnerConstraints) diff --git a/ddl/placement/bundle_test.go b/ddl/placement/bundle_test.go index 3c11559f06304..c2a5b8c05dfad 100644 --- a/ddl/placement/bundle_test.go +++ b/ddl/placement/bundle_test.go @@ -572,7 +572,10 @@ func (s *testBundleSuite) TestNewBundleFromOptions(c *C) { LeaderConstraints: "[+region=as]", FollowerConstraints: "[-region=us]", }, - err: ErrInvalidPlacementOptions, + output: []*Rule{ + NewRule(Leader, 1, NewConstraintsDirect(NewConstraintDirect("region", In, "as"))), + NewRule(Voter, 2, NewConstraintsDirect(NewConstraintDirect("region", NotIn, "us"))), + }, }) tests = append(tests, TestCase{ diff --git a/parser/parser.go b/parser/parser.go index a4fa6c451eed0..c93d5f37e6ac7 100644 --- a/parser/parser.go +++ b/parser/parser.go @@ -11289,7 +11289,12 @@ yynewstate: } case 10: { - parser.yyVAL.item = &ast.PlacementOption{Tp: ast.PlacementOptionFollowerCount, UintValue: yyS[yypt-0].item.(uint64)} + cnt := yyS[yypt-0].item.(uint64) + if cnt == 0 { + yylex.AppendError(yylex.Errorf("FOLLOWERS must be positive")) + return 1 + } + parser.yyVAL.item = &ast.PlacementOption{Tp: ast.PlacementOptionFollowerCount, UintValue: cnt} } case 11: { diff --git a/parser/parser.y b/parser/parser.y index 91e3b05918fc9..b79e7bccd2591 100644 --- a/parser/parser.y +++ b/parser/parser.y @@ -1526,7 +1526,12 @@ DirectPlacementOption: } | "FOLLOWERS" EqOpt LengthNum { - $$ = &ast.PlacementOption{Tp: ast.PlacementOptionFollowerCount, UintValue: $3.(uint64)} + cnt := $3.(uint64) + if cnt == 0 { + yylex.AppendError(yylex.Errorf("FOLLOWERS must be positive")) + return 1 + } + $$ = &ast.PlacementOption{Tp: ast.PlacementOptionFollowerCount, UintValue: cnt} } | "VOTERS" EqOpt LengthNum { diff --git a/parser/parser_test.go b/parser/parser_test.go index 1c5ed6b4e9987..a9587bdba5724 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -2338,6 +2338,7 @@ func TestDDL(t *testing.T) { {`create table t (c int) regions="us,3";`, true, "CREATE TABLE `t` (`c` INT) REGIONS = 'us,3'"}, {`create table t (c int) followers="us,3";`, false, ""}, {`create table t (c int) followers=3;`, true, "CREATE TABLE `t` (`c` INT) FOLLOWERS = 3"}, + {`create table t (c int) followers=0;`, false, ""}, {`create table t (c int) voters="us,3";`, false, ""}, {`create table t (c int) voters=3;`, true, "CREATE TABLE `t` (`c` INT) VOTERS = 3"}, {`create table t (c int) learners="us,3";`, false, ""}, @@ -2353,6 +2354,7 @@ func TestDDL(t *testing.T) { {`create table t (c int) /*T![placement] regions="us,3" */;`, true, "CREATE TABLE `t` (`c` INT) REGIONS = 'us,3'"}, {`create table t (c int) /*T![placement] followers="us,3 */";`, false, ""}, {`create table t (c int) /*T![placement] followers=3 */;`, true, "CREATE TABLE `t` (`c` INT) FOLLOWERS = 3"}, + {`create table t (c int) /*T![placement] followers=0 */;`, false, ""}, {`create table t (c int) /*T![placement] voters="us,3" */;`, false, ""}, {`create table t (c int) /*T![placement] primary_region="us" regions="us,3" */;`, true, "CREATE TABLE `t` (`c` INT) PRIMARY_REGION = 'us' REGIONS = 'us,3'"}, {"create table t (c int) /*T![placement] placement policy=`x` */;", true, "CREATE TABLE `t` (`c` INT) PLACEMENT POLICY = `x`"}, @@ -2361,6 +2363,7 @@ func TestDDL(t *testing.T) { {`alter table t primary_region="us";`, true, "ALTER TABLE `t` PRIMARY_REGION = 'us'"}, {`alter table t regions="us,3";`, true, "ALTER TABLE `t` REGIONS = 'us,3'"}, {`alter table t followers=3;`, true, "ALTER TABLE `t` FOLLOWERS = 3"}, + {`alter table t followers=0;`, false, ""}, {`alter table t voters=3;`, true, "ALTER TABLE `t` VOTERS = 3"}, {`alter table t learners=3;`, true, "ALTER TABLE `t` LEARNERS = 3"}, {`alter table t schedule="even";`, true, "ALTER TABLE `t` SCHEDULE = 'even'"}, @@ -2375,6 +2378,7 @@ func TestDDL(t *testing.T) { {`create database t primary_region="us";`, true, "CREATE DATABASE `t` PRIMARY_REGION = 'us'"}, {`create database t regions="us,3";`, true, "CREATE DATABASE `t` REGIONS = 'us,3'"}, {`create database t followers=3;`, true, "CREATE DATABASE `t` FOLLOWERS = 3"}, + {`create database t followers=0;`, false, ""}, {`create database t voters=3;`, true, "CREATE DATABASE `t` VOTERS = 3"}, {`create database t learners=3;`, true, "CREATE DATABASE `t` LEARNERS = 3"}, {`create database t schedule="even";`, true, "CREATE DATABASE `t` SCHEDULE = 'even'"}, @@ -2390,6 +2394,7 @@ func TestDDL(t *testing.T) { {`alter database t primary_region="us";`, true, "ALTER DATABASE `t` PRIMARY_REGION = 'us'"}, {`alter database t regions="us,3";`, true, "ALTER DATABASE `t` REGIONS = 'us,3'"}, {`alter database t followers=3;`, true, "ALTER DATABASE `t` FOLLOWERS = 3"}, + {`alter database t followers=0;`, false, ""}, {`alter database t voters=3;`, true, "ALTER DATABASE `t` VOTERS = 3"}, {`alter database t learners=3;`, true, "ALTER DATABASE `t` LEARNERS = 3"}, {`alter database t schedule="even";`, true, "ALTER DATABASE `t` SCHEDULE = 'even'"}, @@ -3382,6 +3387,7 @@ func TestDDL(t *testing.T) { {"create placement policy x primary_region='us'", true, "CREATE PLACEMENT POLICY `x` PRIMARY_REGION = 'us'"}, {"create placement policy x region='us, 3'", false, ""}, {"create placement policy x followers=3", true, "CREATE PLACEMENT POLICY `x` FOLLOWERS = 3"}, + {"create placement policy x followers=0", false, ""}, {"create placement policy x voters=3", true, "CREATE PLACEMENT POLICY `x` VOTERS = 3"}, {"create placement policy x learners=3", true, "CREATE PLACEMENT POLICY `x` LEARNERS = 3"}, {"create placement policy x schedule='even'", true, "CREATE PLACEMENT POLICY `x` SCHEDULE = 'even'"}, From 42cf333da187329e8f13c4a8dacb7bc19b5236d7 Mon Sep 17 00:00:00 2001 From: Song Gao Date: Fri, 24 Dec 2021 16:48:45 +0800 Subject: [PATCH 41/44] flatten the json output (#30905) Signed-off-by: yisaer Co-authored-by: Ti Chi Robot --- util/tracing/opt_trace.go | 57 +++++++++++++++--- util/tracing/opt_trace_test.go | 105 +++++++++++++++++++++++++++++++++ 2 files changed, 154 insertions(+), 8 deletions(-) create mode 100644 util/tracing/opt_trace_test.go diff --git a/util/tracing/opt_trace.go b/util/tracing/opt_trace.go index e3d243b16ed33..506db98ee0d8d 100644 --- a/util/tracing/opt_trace.go +++ b/util/tracing/opt_trace.go @@ -16,17 +16,17 @@ package tracing // LogicalPlanTrace indicates for the LogicalPlan trace information type LogicalPlanTrace struct { - ID int `json:"id"` - TP string `json:"type"` - Children []*LogicalPlanTrace `json:"children"` + ID int + TP string + Children []*LogicalPlanTrace // ExplainInfo should be implemented by each implemented LogicalPlan - ExplainInfo string `json:"info"` + ExplainInfo string } // LogicalOptimizeTracer indicates the trace for the whole logicalOptimize processing type LogicalOptimizeTracer struct { - FinalLogicalPlan *LogicalPlanTrace `json:"final"` + FinalLogicalPlan []FlattenLogicalPlanTrace `json:"final"` Steps []*LogicalRuleOptimizeTracer `json:"steps"` // curRuleTracer indicates the current rule Tracer during optimize by rule curRuleTracer *LogicalRuleOptimizeTracer @@ -53,14 +53,14 @@ func (tracer *LogicalOptimizeTracer) AppendRuleTracerStepToCurrent(id int, tp, r // RecordFinalLogicalPlan add plan trace after logical optimize func (tracer *LogicalOptimizeTracer) RecordFinalLogicalPlan(final *LogicalPlanTrace) { - tracer.FinalLogicalPlan = final + tracer.FinalLogicalPlan = toFlattenLogicalPlanTrace(final) } // LogicalRuleOptimizeTracer indicates the trace for the LogicalPlan tree before and after // logical rule optimize type LogicalRuleOptimizeTracer struct { Index int `json:"index"` - Before *LogicalPlanTrace `json:"before"` + Before []FlattenLogicalPlanTrace `json:"before"` RuleName string `json:"name"` Steps []LogicalRuleOptimizeTraceStep `json:"steps"` } @@ -69,7 +69,7 @@ type LogicalRuleOptimizeTracer struct { func buildLogicalRuleOptimizeTracerBeforeOptimize(index int, name string, before *LogicalPlanTrace) *LogicalRuleOptimizeTracer { return &LogicalRuleOptimizeTracer{ Index: index, - Before: before, + Before: toFlattenLogicalPlanTrace(before), RuleName: name, Steps: make([]LogicalRuleOptimizeTraceStep, 0), } @@ -85,6 +85,47 @@ type LogicalRuleOptimizeTraceStep struct { Index int `json:"index"` } +// FlattenLogicalPlanTrace indicates the flatten LogicalPlanTrace +type FlattenLogicalPlanTrace struct { + ID int `json:"id"` + TP string `json:"type"` + Children []int `json:"children"` + + // ExplainInfo should be implemented by each implemented LogicalPlan + ExplainInfo string `json:"info"` +} + +// toFlattenLogicalPlanTrace transform LogicalPlanTrace into FlattenLogicalPlanTrace +func toFlattenLogicalPlanTrace(root *LogicalPlanTrace) []FlattenLogicalPlanTrace { + wrapper := &flattenWrapper{flatten: make([]FlattenLogicalPlanTrace, 0)} + flattenLogicalPlanTrace(root, wrapper) + return wrapper.flatten +} + +type flattenWrapper struct { + flatten []FlattenLogicalPlanTrace +} + +func flattenLogicalPlanTrace(node *LogicalPlanTrace, wrapper *flattenWrapper) { + flattenNode := FlattenLogicalPlanTrace{ + ID: node.ID, + TP: node.TP, + ExplainInfo: node.ExplainInfo, + Children: make([]int, 0), + } + if len(node.Children) < 1 { + wrapper.flatten = append(wrapper.flatten, flattenNode) + return + } + for _, child := range node.Children { + flattenNode.Children = append(flattenNode.Children, child.ID) + } + for _, child := range node.Children { + flattenLogicalPlanTrace(child, wrapper) + } + wrapper.flatten = append(wrapper.flatten, flattenNode) +} + // CETraceRecord records an expression and related cardinality estimation result. type CETraceRecord struct { TableID int64 `json:"-"` diff --git a/util/tracing/opt_trace_test.go b/util/tracing/opt_trace_test.go new file mode 100644 index 0000000000000..800749429ec0d --- /dev/null +++ b/util/tracing/opt_trace_test.go @@ -0,0 +1,105 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package tracing + +import ( + "testing" + + "github.com/stretchr/testify/require" +) + +func TestFlattenLogicalPlanTrace(t *testing.T) { + root1 := &LogicalPlanTrace{ + ID: 1, + TP: "foo1", + ExplainInfo: "bar1", + Children: []*LogicalPlanTrace{ + { + ID: 2, + TP: "foo2", + ExplainInfo: "bar2", + Children: nil, + }, + }, + } + root2 := &LogicalPlanTrace{ + ID: 1, + TP: "foo1", + ExplainInfo: "bar1", + Children: []*LogicalPlanTrace{ + { + ID: 2, + TP: "foo2", + ExplainInfo: "bar2", + Children: nil, + }, + { + ID: 3, + TP: "foo3", + ExplainInfo: "bar3", + Children: []*LogicalPlanTrace{ + { + ID: 4, + TP: "foo4", + ExplainInfo: "bar4", + Children: nil, + }, + }, + }, + }, + } + expect1 := []FlattenLogicalPlanTrace{ + { + ID: 2, + TP: "foo2", + ExplainInfo: "bar2", + Children: []int{}, + }, + { + ID: 1, + TP: "foo1", + ExplainInfo: "bar1", + Children: []int{2}, + }, + } + expect2 := []FlattenLogicalPlanTrace{ + { + ID: 2, + TP: "foo2", + ExplainInfo: "bar2", + Children: []int{}, + }, + { + ID: 4, + TP: "foo4", + ExplainInfo: "bar4", + Children: []int{}, + }, + { + ID: 3, + TP: "foo3", + ExplainInfo: "bar3", + Children: []int{4}, + }, + { + ID: 1, + TP: "foo1", + ExplainInfo: "bar1", + Children: []int{2, 3}, + }, + } + require.EqualValues(t, toFlattenLogicalPlanTrace(root1), expect1) + require.EqualValues(t, toFlattenLogicalPlanTrace(root2), expect2) +} From 3de13842b34ae56bec7ddff05ae3403a9c0883e1 Mon Sep 17 00:00:00 2001 From: tangenta Date: Fri, 24 Dec 2021 17:03:48 +0800 Subject: [PATCH 42/44] test: control log level with environment variables (#30871) --- Makefile | 33 ++++------------- Makefile.common | 3 -- bindinfo/main_test.go | 2 +- br/pkg/conn/main_test.go | 2 +- br/pkg/kv/main_test.go | 2 +- br/pkg/metautil/main_test.go | 2 +- br/pkg/pdutil/main_test.go | 2 +- br/pkg/rtree/main_test.go | 2 +- br/pkg/summary/main_test.go | 2 +- br/pkg/trace/main_test.go | 2 +- br/pkg/utils/main_test.go | 2 +- cmd/ddltest/main_test.go | 2 +- config/main_test.go | 2 +- ddl/failtest/main_test.go | 2 +- ddl/label/main_test.go | 2 +- ddl/util/main_test.go | 2 +- distsql/main_test.go | 2 +- domain/globalconfigsync/globalconfig_test.go | 2 +- domain/infosync/info_test.go | 2 +- domain/main_test.go | 2 +- errno/main_test.go | 2 +- executor/aggfuncs/main_test.go | 2 +- executor/main_test.go | 2 +- executor/oomtest/oom_test.go | 2 +- executor/seqtest/main_test.go | 2 +- expression/aggregation/main_test.go | 2 +- expression/main_test.go | 2 +- infoschema/main_test.go | 2 +- infoschema/perfschema/main_test.go | 2 +- kv/main_test.go | 2 +- meta/autoid/main_test.go | 2 +- meta/main_test.go | 2 +- metrics/main_test.go | 2 +- owner/main_test.go | 2 +- planner/cascades/main_test.go | 2 +- planner/core/main_test.go | 2 +- planner/implementation/main_test.go | 2 +- planner/memo/main_test.go | 2 +- planner/util/main_test.go | 2 +- plugin/conn_ip_example/main_test.go | 2 +- plugin/main_test.go | 2 +- privilege/privileges/main_test.go | 2 +- server/main_test.go | 2 +- session/main_test.go | 2 +- sessionctx/binloginfo/main_test.go | 2 +- sessionctx/main_test.go | 2 +- sessionctx/stmtctx/main_test.go | 2 +- sessionctx/variable/main_test.go | 2 +- sessiontxn/txn_context_serial_test.go | 2 +- statistics/handle/main_test.go | 2 +- statistics/main_test.go | 2 +- store/copr/main_test.go | 2 +- store/driver/error/error_test.go | 2 +- store/driver/main_test.go | 2 +- store/driver/txn/main_test.go | 2 +- store/gcworker/main_test.go | 2 +- store/helper/main_test.go | 2 +- store/main_test.go | 2 +- store/mockstore/main_test.go | 2 +- store/mockstore/mockcopr/main_test.go | 2 +- .../unistore/cophandler/main_test.go | 2 +- .../mockstore/unistore/lockstore/main_test.go | 2 +- store/mockstore/unistore/main_test.go | 2 +- store/mockstore/unistore/tikv/main_test.go | 2 +- .../unistore/util/lockwaiter/main_test.go | 2 +- structure/main_test.go | 2 +- table/main_test.go | 2 +- table/tables/main_test.go | 2 +- table/temptable/main_test.go | 2 +- tablecodec/main_test.go | 2 +- tablecodec/rowindexcodec/main_test.go | 2 +- telemetry/cte_test/cte_test.go | 2 +- telemetry/main_test.go | 2 +- tests/globalkilltest/main_test.go | 2 +- tests/graceshutdown/main_test.go | 2 +- tests/readonlytest/main_test.go | 2 +- tidb-server/main_test.go | 2 +- tools/check/go.mod | 2 - tools/check/go.sum | 4 -- types/json/main_test.go | 2 +- types/main_test.go | 2 +- types/parser_driver/main_test.go | 2 +- util/admin/main_test.go | 2 +- util/arena/main_test.go | 2 +- util/benchdaily/main_test.go | 2 +- util/bitmap/main_test.go | 2 +- util/checksum/main_test.go | 2 +- util/chunk/main_test.go | 2 +- util/codec/main_test.go | 2 +- util/collate/main_test.go | 2 +- util/cteutil/main_test.go | 2 +- util/dbterror/main_test.go | 2 +- util/deadlockhistory/main_test.go | 2 +- util/disjointset/main_test.go | 2 +- util/disk/main_test.go | 2 +- util/encrypt/main_test.go | 2 +- util/execdetails/main_test.go | 2 +- util/expensivequery/expensivequerey_test.go | 2 +- util/fastrand/main_test.go | 2 +- util/format/main_test.go | 2 +- util/generatedexpr/main_test.go | 2 +- util/hack/main_test.go | 2 +- util/keydecoder/main_test.go | 2 +- util/kvcache/main_test.go | 2 +- util/localpool/main_test.go | 2 +- util/logutil/main_test.go | 2 +- util/main_test.go | 2 +- util/math/main_test.go | 2 +- util/memory/main_test.go | 12 +----- util/mock/main_test.go | 2 +- util/mvmap/main_test.go | 2 +- util/parser/main_test.go | 2 +- util/plancodec/main_test.go | 2 +- util/printer/main_test.go | 2 +- util/profile/main_test.go | 2 +- util/ranger/main_test.go | 2 +- util/resourcegrouptag/main_test.go | 2 +- util/rowDecoder/main_test.go | 2 +- util/rowcodec/main_test.go | 2 +- util/selection/main_test.go | 2 +- util/sem/main_test.go | 2 +- util/set/main_test.go | 2 +- util/slice/main_test.go | 2 +- util/sqlexec/main_test.go | 2 +- util/stmtsummary/main_test.go | 2 +- util/stringutil/main_test.go | 2 +- util/sys/linux/main_test.go | 2 +- util/sys/storage/main_test.go | 2 +- util/systimemon/main_test.go | 2 +- util/testbridge/bridge.go | 37 +++++++++++++++++-- util/texttree/main_test.go | 2 +- util/timeutil/main_test.go | 2 +- util/topsql/main_test.go | 2 +- util/topsql/reporter/main_test.go | 2 +- util/topsql/stmtstats/main_test.go | 2 +- .../stmtstats/stmtstatstest/main_test.go | 2 +- util/topsql/tracecpu/main_test.go | 2 +- util/tracing/main_test.go | 2 +- util/vitess/main_test.go | 2 +- 139 files changed, 175 insertions(+), 182 deletions(-) diff --git a/Makefile b/Makefile index 30e01b2c1a1e1..e0ba263d192ab 100644 --- a/Makefile +++ b/Makefile @@ -136,24 +136,13 @@ gotest: failpoint-enable $(GOTEST) -ldflags '$(TEST_LDFLAGS)' $(EXTRA_TEST_ARGS) -timeout 20m -cover $(PACKAGES_TIDB_TESTS) -coverprofile=coverage.txt -check.p true > gotest.log || { $(FAILPOINT_DISABLE); cat 'gotest.log'; exit 1; } @$(FAILPOINT_DISABLE) -gotest_in_verify_ci_part_1: failpoint-enable tools/bin/gotestsum tools/bin/gocov tools/bin/gocov-xml - @echo "Running gotest_in_verify_ci_part_1." +gotest_in_verify_ci: failpoint-enable tools/bin/gotestsum + @echo "Running gotest_in_verify_ci" @mkdir -p $(TEST_COVERAGE_DIR) - @export log_level=info; export TZ='Asia/Shanghai'; \ - CGO_ENABLED=1 tools/bin/gotestsum --junitfile "$(TEST_COVERAGE_DIR)/tidb-junit-report.xml" -- -v -p $(P) --race \ - -ldflags '$(TEST_LDFLAGS)' $(EXTRA_TEST_ARGS) -coverprofile="$(TEST_COVERAGE_DIR)/tidb_cov.unit_test.out" \ - $(PACKAGES_TIDB_TESTS_EXPENSIVE) -check.p true || { $(FAILPOINT_DISABLE); exit 1; } - tools/bin/gocov convert "$(TEST_COVERAGE_DIR)/tidb_cov.unit_test.out" | tools/bin/gocov-xml > "$(TEST_COVERAGE_DIR)/tidb-coverage.xml" - @$(FAILPOINT_DISABLE) - -gotest_in_verify_ci_part_2: failpoint-enable tools/bin/gotestsum tools/bin/gocov tools/bin/gocov-xml - @echo "Running gotest_in_verify_ci_part_2." - @mkdir -p $(TEST_COVERAGE_DIR) - @export log_level=info; export TZ='Asia/Shanghai'; \ - CGO_ENABLED=1 tools/bin/gotestsum --junitfile "$(TEST_COVERAGE_DIR)/tidb-junit-report.xml" -- -v -p $(P) --race \ + @export TZ='Asia/Shanghai'; \ + CGO_ENABLED=1 tools/bin/gotestsum --junitfile "$(TEST_COVERAGE_DIR)/tidb-junit-report.xml" -- -v -p $(P) \ -ldflags '$(TEST_LDFLAGS)' $(EXTRA_TEST_ARGS) -coverprofile="$(TEST_COVERAGE_DIR)/tidb_cov.unit_test.out" \ - $(PACKAGES_TIDB_TESTS_OTHERS) -check.p true || { $(FAILPOINT_DISABLE); exit 1; } - tools/bin/gocov convert "$(TEST_COVERAGE_DIR)/tidb_cov.unit_test.out" | tools/bin/gocov-xml > "$(TEST_COVERAGE_DIR)/tidb-coverage.xml" + $(PACKAGES_TIDB_TESTS) -check.p true || { $(FAILPOINT_DISABLE); exit 1; } @$(FAILPOINT_DISABLE) race: failpoint-enable @@ -341,13 +330,12 @@ br_unit_test: $(GOTEST) $(RACE_FLAG) -ldflags '$(LDFLAGS)' -tags leak $(ARGS) -coverprofile=coverage.txt || ( make failpoint-disable && exit 1 ) @make failpoint-disable br_unit_test_in_verify_ci: export ARGS=$$($(BR_PACKAGES)) -br_unit_test_in_verify_ci: tools/bin/gotestsum tools/bin/gocov tools/bin/gocov-xml +br_unit_test_in_verify_ci: tools/bin/gotestsum @make failpoint-enable @export TZ='Asia/Shanghai'; @mkdir -p $(TEST_COVERAGE_DIR) CGO_ENABLED=1 tools/bin/gotestsum --junitfile "$(TEST_COVERAGE_DIR)/br-junit-report.xml" -- $(RACE_FLAG) -ldflags '$(LDFLAGS)' \ -tags leak $(ARGS) -coverprofile="$(TEST_COVERAGE_DIR)/br_cov.unit_test.out" || ( make failpoint-disable && exit 1 ) - tools/bin/gocov convert "$(TEST_COVERAGE_DIR)/br_cov.unit_test.out" | tools/bin/gocov-xml > "$(TEST_COVERAGE_DIR)/br-coverage.xml" @make failpoint-disable br_integration_test: br_bins build_br build_for_br_integration_test @@ -407,11 +395,10 @@ dumpling_unit_test: failpoint-enable $(DUMPLING_GOTEST) $(RACE_FLAG) -coverprofile=coverage.txt -covermode=atomic -tags leak $(DUMPLING_ARGS) || ( make failpoint-disable && exit 1 ) @make failpoint-disable dumpling_unit_test_in_verify_ci: export DUMPLING_ARGS=$$($(DUMPLING_PACKAGES)) -dumpling_unit_test_in_verify_ci: failpoint-enable tools/bin/gotestsum tools/bin/gocov tools/bin/gocov-xml +dumpling_unit_test_in_verify_ci: failpoint-enable tools/bin/gotestsum @mkdir -p $(TEST_COVERAGE_DIR) CGO_ENABLED=1 tools/bin/gotestsum --junitfile "$(TEST_COVERAGE_DIR)/dumpling-junit-report.xml" -- -tags leak $(DUMPLING_ARGS) \ $(RACE_FLAG) -coverprofile="$(TEST_COVERAGE_DIR)/dumpling_cov.unit_test.out" || ( make failpoint-disable && exit 1 ) - tools/bin/gocov convert "$(TEST_COVERAGE_DIR)/dumpling_cov.unit_test.out" | tools/bin/gocov-xml > "$(TEST_COVERAGE_DIR)/dumpling-coverage.xml" @make failpoint-disable dumpling_integration_test: dumpling_bins failpoint-enable build_dumpling @@ -435,9 +422,3 @@ dumpling_bins: tools/bin/gotestsum: tools/check/go.mod cd tools/check && $(GO) build -o ../bin/gotestsum gotest.tools/gotestsum - -tools/bin/gocov: tools/check/go.mod - cd tools/check && $(GO) build -o ../bin/gocov github.com/axw/gocov/gocov - -tools/bin/gocov-xml: tools/check/go.mod - cd tools/check && $(GO) build -o ../bin/gocov-xml github.com/AlekSi/gocov-xml diff --git a/Makefile.common b/Makefile.common index 2a8ea369521b1..8ea85d6a24694 100644 --- a/Makefile.common +++ b/Makefile.common @@ -47,11 +47,8 @@ MAC := "Darwin" PACKAGE_LIST := go list ./... PACKAGE_LIST_TIDB_TESTS := go list ./... | grep -vE "github.com\/pingcap\/tidb\/br|github.com\/pingcap\/tidb\/cmd|github.com\/pingcap\/tidb\/dumpling" -PACKAGE_LIST_TEST_OTHERS := go list ./... | grep -vE "github.com\/pingcap\/tidb\/br|github.com\/pingcap\/tidb\/cmd|github.com\/pingcap\/tidb\/dumpling|github.com\/pingcap\/tidb\/executor|github.com\/pingcap\/tidb\/cmd|github.com\/pingcap\/tidb\/ddl" PACKAGES ?= $$($(PACKAGE_LIST)) PACKAGES_TIDB_TESTS ?= $$($(PACKAGE_LIST_TIDB_TESTS)) -PACKAGES_TIDB_TESTS_EXPENSIVE ?= "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/ddl" -PACKAGES_TIDB_TESTS_OTHERS ?= $$($(PACKAGE_LIST_TEST_OTHERS)) PACKAGE_DIRECTORIES := $(PACKAGE_LIST) | sed 's|github.com/pingcap/$(PROJECT)/||' PACKAGE_DIRECTORIES_TIDB_TESTS := $(PACKAGE_LIST_TIDB_TESTS) | sed 's|github.com/pingcap/$(PROJECT)/||' FILES := $$(find $$($(PACKAGE_DIRECTORIES)) -name "*.go") diff --git a/bindinfo/main_test.go b/bindinfo/main_test.go index 85151366ea0ef..6104329761360 100644 --- a/bindinfo/main_test.go +++ b/bindinfo/main_test.go @@ -22,7 +22,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() opts := []goleak.Option{ goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), diff --git a/br/pkg/conn/main_test.go b/br/pkg/conn/main_test.go index b1299da7358de..7b46a892be79d 100644 --- a/br/pkg/conn/main_test.go +++ b/br/pkg/conn/main_test.go @@ -26,6 +26,6 @@ func TestMain(m *testing.M) { goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), } - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m, opts...) } diff --git a/br/pkg/kv/main_test.go b/br/pkg/kv/main_test.go index d7e28c807d792..72bbbcfbf4cb0 100644 --- a/br/pkg/kv/main_test.go +++ b/br/pkg/kv/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/br/pkg/metautil/main_test.go b/br/pkg/metautil/main_test.go index e73ef73e16f2d..44cb7f4a19097 100644 --- a/br/pkg/metautil/main_test.go +++ b/br/pkg/metautil/main_test.go @@ -25,6 +25,6 @@ func TestMain(m *testing.M) { opts := []goleak.Option{ goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), } - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m, opts...) } diff --git a/br/pkg/pdutil/main_test.go b/br/pkg/pdutil/main_test.go index 861c3921a3eb3..653d973bcd499 100644 --- a/br/pkg/pdutil/main_test.go +++ b/br/pkg/pdutil/main_test.go @@ -22,7 +22,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() opts := []goleak.Option{ goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), diff --git a/br/pkg/rtree/main_test.go b/br/pkg/rtree/main_test.go index 85dc99665acaf..8d4ae4216a894 100644 --- a/br/pkg/rtree/main_test.go +++ b/br/pkg/rtree/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/br/pkg/summary/main_test.go b/br/pkg/summary/main_test.go index e1b89ff3d0a0f..77dfc58c09d57 100644 --- a/br/pkg/summary/main_test.go +++ b/br/pkg/summary/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/br/pkg/trace/main_test.go b/br/pkg/trace/main_test.go index f253ad281ecdb..adbee42408e38 100644 --- a/br/pkg/trace/main_test.go +++ b/br/pkg/trace/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/br/pkg/utils/main_test.go b/br/pkg/utils/main_test.go index 47d7fd9b63b21..09976b09ac199 100644 --- a/br/pkg/utils/main_test.go +++ b/br/pkg/utils/main_test.go @@ -25,6 +25,6 @@ func TestMain(m *testing.M) { opts := []goleak.Option{ goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), } - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m, opts...) } diff --git a/cmd/ddltest/main_test.go b/cmd/ddltest/main_test.go index 890a52a7f4ab8..810ba80b9a974 100644 --- a/cmd/ddltest/main_test.go +++ b/cmd/ddltest/main_test.go @@ -26,7 +26,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() err := logutil.InitLogger(&logutil.LogConfig{Config: zaplog.Config{Level: *logLevel}}) if err != nil { fmt.Fprint(os.Stderr, err.Error()) diff --git a/config/main_test.go b/config/main_test.go index 0ef0a65458301..847d12a0a12ce 100644 --- a/config/main_test.go +++ b/config/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/ddl/failtest/main_test.go b/ddl/failtest/main_test.go index 7c9e3d4bae618..aea99a6687c23 100644 --- a/ddl/failtest/main_test.go +++ b/ddl/failtest/main_test.go @@ -25,7 +25,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() config.UpdateGlobal(func(conf *config.Config) { conf.TiKVClient.AsyncCommit.SafeWindow = 0 diff --git a/ddl/label/main_test.go b/ddl/label/main_test.go index 25784de1cd97e..577972a1f5a95 100644 --- a/ddl/label/main_test.go +++ b/ddl/label/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/ddl/util/main_test.go b/ddl/util/main_test.go index 3dc8c61daacda..9446758e36923 100644 --- a/ddl/util/main_test.go +++ b/ddl/util/main_test.go @@ -22,7 +22,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() opts := []goleak.Option{ goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), } diff --git a/distsql/main_test.go b/distsql/main_test.go index 1aaddbe954620..d000a32e9561f 100644 --- a/distsql/main_test.go +++ b/distsql/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/domain/globalconfigsync/globalconfig_test.go b/domain/globalconfigsync/globalconfig_test.go index c7be9137ddf68..a1705adffea1d 100644 --- a/domain/globalconfigsync/globalconfig_test.go +++ b/domain/globalconfigsync/globalconfig_test.go @@ -30,7 +30,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() opts := []goleak.Option{ goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), diff --git a/domain/infosync/info_test.go b/domain/infosync/info_test.go index bb0ded60e7e25..f8e0d5d728650 100644 --- a/domain/infosync/info_test.go +++ b/domain/infosync/info_test.go @@ -37,7 +37,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() opts := []goleak.Option{ goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), } diff --git a/domain/main_test.go b/domain/main_test.go index 069ed9590f222..969d50e4a0f67 100644 --- a/domain/main_test.go +++ b/domain/main_test.go @@ -23,7 +23,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() opts := []goleak.Option{ goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), diff --git a/errno/main_test.go b/errno/main_test.go index a38aae7f1cb39..f0064431962cc 100644 --- a/errno/main_test.go +++ b/errno/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() os.Exit(m.Run()) } diff --git a/executor/aggfuncs/main_test.go b/executor/aggfuncs/main_test.go index f46a63bba4d9a..a385a98c57215 100644 --- a/executor/aggfuncs/main_test.go +++ b/executor/aggfuncs/main_test.go @@ -22,7 +22,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() opts := []goleak.Option{ goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), diff --git a/executor/main_test.go b/executor/main_test.go index bff65b72d6a2d..88be5b84f0750 100644 --- a/executor/main_test.go +++ b/executor/main_test.go @@ -34,7 +34,7 @@ var prepareMergeSuiteData testdata.TestData var aggMergeSuiteData testdata.TestData func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() testDataMap.LoadTestSuiteData("testdata", "prepare_suite") testDataMap.LoadTestSuiteData("testdata", "agg_suite") diff --git a/executor/oomtest/oom_test.go b/executor/oomtest/oom_test.go index d592456bb61cd..b6a4d452bbe4a 100644 --- a/executor/oomtest/oom_test.go +++ b/executor/oomtest/oom_test.go @@ -34,7 +34,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() registerHook() domain.RunAutoAnalyze = false config.UpdateGlobal(func(conf *config.Config) { diff --git a/executor/seqtest/main_test.go b/executor/seqtest/main_test.go index 0a85b8ca61055..737a21822d083 100644 --- a/executor/seqtest/main_test.go +++ b/executor/seqtest/main_test.go @@ -23,7 +23,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() config.UpdateGlobal(func(conf *config.Config) { conf.TiKVClient.AsyncCommit.SafeWindow = 0 conf.TiKVClient.AsyncCommit.AllowedClockDrift = 0 diff --git a/expression/aggregation/main_test.go b/expression/aggregation/main_test.go index 4078dc6cf4b48..53126014ead5b 100644 --- a/expression/aggregation/main_test.go +++ b/expression/aggregation/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/expression/main_test.go b/expression/main_test.go index 9a1e170078f65..590c1c33ba72f 100644 --- a/expression/main_test.go +++ b/expression/main_test.go @@ -32,7 +32,7 @@ import ( var testDataMap = make(testdata.BookKeeper) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() testmain.ShortCircuitForBench(m) config.UpdateGlobal(func(conf *config.Config) { diff --git a/infoschema/main_test.go b/infoschema/main_test.go index 8d3c4dba9623e..8a878a925be01 100644 --- a/infoschema/main_test.go +++ b/infoschema/main_test.go @@ -22,7 +22,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() opts := []goleak.Option{ goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), diff --git a/infoschema/perfschema/main_test.go b/infoschema/perfschema/main_test.go index 068cebc3f3cb0..a13c7d1ddca79 100644 --- a/infoschema/perfschema/main_test.go +++ b/infoschema/perfschema/main_test.go @@ -22,7 +22,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() opts := []goleak.Option{ goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), diff --git a/kv/main_test.go b/kv/main_test.go index e87d9dd2a9916..6d1f289de7d9a 100644 --- a/kv/main_test.go +++ b/kv/main_test.go @@ -22,7 +22,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() opts := []goleak.Option{ goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), diff --git a/meta/autoid/main_test.go b/meta/autoid/main_test.go index a5e8e915db0db..c9ee29c2200f7 100644 --- a/meta/autoid/main_test.go +++ b/meta/autoid/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/meta/main_test.go b/meta/main_test.go index 858d4bbb6f6e6..7fd569a01a417 100644 --- a/meta/main_test.go +++ b/meta/main_test.go @@ -22,7 +22,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() opts := []goleak.Option{ goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), diff --git a/metrics/main_test.go b/metrics/main_test.go index 2e27c2eec0544..96050f378e142 100644 --- a/metrics/main_test.go +++ b/metrics/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/owner/main_test.go b/owner/main_test.go index ba940ab9a3d5d..ec1eeb75cf4a5 100644 --- a/owner/main_test.go +++ b/owner/main_test.go @@ -22,7 +22,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() opts := []goleak.Option{ goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), } diff --git a/planner/cascades/main_test.go b/planner/cascades/main_test.go index 590e1825b08bd..f1a3ceb81fe23 100644 --- a/planner/cascades/main_test.go +++ b/planner/cascades/main_test.go @@ -30,7 +30,7 @@ var stringerSuiteData testdata.TestData var transformationRulesSuiteData testdata.TestData func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() flag.Parse() diff --git a/planner/core/main_test.go b/planner/core/main_test.go index 236d154ff8934..c0fb896a5bfd2 100644 --- a/planner/core/main_test.go +++ b/planner/core/main_test.go @@ -28,7 +28,7 @@ var testDataMap = make(testdata.BookKeeper, 2) var indexMergeSuiteData testdata.TestData func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() flag.Parse() diff --git a/planner/implementation/main_test.go b/planner/implementation/main_test.go index b7a2088709314..2e4cb46332f0f 100644 --- a/planner/implementation/main_test.go +++ b/planner/implementation/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/planner/memo/main_test.go b/planner/memo/main_test.go index 784f011a9ddee..6072aab036844 100644 --- a/planner/memo/main_test.go +++ b/planner/memo/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/planner/util/main_test.go b/planner/util/main_test.go index 1b930670688d0..0a5c3ac1a47da 100644 --- a/planner/util/main_test.go +++ b/planner/util/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/plugin/conn_ip_example/main_test.go b/plugin/conn_ip_example/main_test.go index 640b8d3aa5108..24c8bc9dcec4e 100644 --- a/plugin/conn_ip_example/main_test.go +++ b/plugin/conn_ip_example/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/plugin/main_test.go b/plugin/main_test.go index 48633eff506ae..cd0e57872a41b 100644 --- a/plugin/main_test.go +++ b/plugin/main_test.go @@ -22,7 +22,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() opts := []goleak.Option{ goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), diff --git a/privilege/privileges/main_test.go b/privilege/privileges/main_test.go index 0d43ce5c21a4a..9819d73c47670 100644 --- a/privilege/privileges/main_test.go +++ b/privilege/privileges/main_test.go @@ -27,7 +27,7 @@ func TestMain(m *testing.M) { goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), } - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() session.SetSchemaLease(0) session.DisableStats4Test() diff --git a/server/main_test.go b/server/main_test.go index 155d9f9b7294e..1320e01c821bf 100644 --- a/server/main_test.go +++ b/server/main_test.go @@ -29,7 +29,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() runInGoTest = true // flag for NewServer to known it is running in test environment diff --git a/session/main_test.go b/session/main_test.go index ee79b1b30e967..fb75aaccf0f3a 100644 --- a/session/main_test.go +++ b/session/main_test.go @@ -42,7 +42,7 @@ var testDataMap = make(testdata.BookKeeper, 1) func TestMain(m *testing.M) { testmain.ShortCircuitForBench(m) - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() flag.Parse() testDataMap.LoadTestSuiteData("testdata", "clustered_index_suite") diff --git a/sessionctx/binloginfo/main_test.go b/sessionctx/binloginfo/main_test.go index 27abcdef3e02f..c7e1ff6d026be 100644 --- a/sessionctx/binloginfo/main_test.go +++ b/sessionctx/binloginfo/main_test.go @@ -22,7 +22,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() opts := []goleak.Option{ goleak.IgnoreTopFunction("google.golang.org/grpc.(*addrConn).resetTransport"), goleak.IgnoreTopFunction("google.golang.org/grpc.(*ccBalancerWrapper).watcher"), diff --git a/sessionctx/main_test.go b/sessionctx/main_test.go index 89eeb11becb85..ae77476986062 100644 --- a/sessionctx/main_test.go +++ b/sessionctx/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/sessionctx/stmtctx/main_test.go b/sessionctx/stmtctx/main_test.go index 5ccfada0cec6c..e97d9a07961a7 100644 --- a/sessionctx/stmtctx/main_test.go +++ b/sessionctx/stmtctx/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/sessionctx/variable/main_test.go b/sessionctx/variable/main_test.go index af5506566eac2..d8c4cc64199e8 100644 --- a/sessionctx/variable/main_test.go +++ b/sessionctx/variable/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/sessiontxn/txn_context_serial_test.go b/sessiontxn/txn_context_serial_test.go index 099af86a0e689..97144263590b5 100644 --- a/sessiontxn/txn_context_serial_test.go +++ b/sessiontxn/txn_context_serial_test.go @@ -32,7 +32,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() opts := []goleak.Option{ goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), diff --git a/statistics/handle/main_test.go b/statistics/handle/main_test.go index 5b8ea0ed4c0dc..658b377aba2b4 100644 --- a/statistics/handle/main_test.go +++ b/statistics/handle/main_test.go @@ -26,6 +26,6 @@ func TestMain(m *testing.M) { goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), } - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m, opts...) } diff --git a/statistics/main_test.go b/statistics/main_test.go index 7e40d650fe393..4b0960f69a735 100644 --- a/statistics/main_test.go +++ b/statistics/main_test.go @@ -32,7 +32,7 @@ import ( var testDataMap = make(testdata.BookKeeper, 3) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() if !flag.Parsed() { flag.Parse() diff --git a/store/copr/main_test.go b/store/copr/main_test.go index 411741f5e651e..e69b25bc2173e 100644 --- a/store/copr/main_test.go +++ b/store/copr/main_test.go @@ -34,6 +34,6 @@ func (m *main) Run() int { } func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(&main{m: m}) } diff --git a/store/driver/error/error_test.go b/store/driver/error/error_test.go index 3db2830179502..6d2222b29bc59 100644 --- a/store/driver/error/error_test.go +++ b/store/driver/error/error_test.go @@ -26,7 +26,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/store/driver/main_test.go b/store/driver/main_test.go index 1902174a16bf7..7e9de1553e8aa 100644 --- a/store/driver/main_test.go +++ b/store/driver/main_test.go @@ -37,7 +37,7 @@ var ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() tikv.EnableFailpoints() opts := []goleak.Option{ goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), diff --git a/store/driver/txn/main_test.go b/store/driver/txn/main_test.go index 3805c435df0e3..97fa3f9904ce4 100644 --- a/store/driver/txn/main_test.go +++ b/store/driver/txn/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/store/gcworker/main_test.go b/store/gcworker/main_test.go index 0a6be0a87548b..89909d438672a 100644 --- a/store/gcworker/main_test.go +++ b/store/gcworker/main_test.go @@ -25,7 +25,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() tikv.EnableFailpoints() opts := []goleak.Option{ goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), diff --git a/store/helper/main_test.go b/store/helper/main_test.go index ade457cc75183..52601cf5f3c6a 100644 --- a/store/helper/main_test.go +++ b/store/helper/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/store/main_test.go b/store/main_test.go index 40703aa3d2e51..b76d49ae6800c 100644 --- a/store/main_test.go +++ b/store/main_test.go @@ -22,7 +22,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() opts := []goleak.Option{ goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), diff --git a/store/mockstore/main_test.go b/store/mockstore/main_test.go index 0bb01f6c23182..dea595138efbf 100644 --- a/store/mockstore/main_test.go +++ b/store/mockstore/main_test.go @@ -24,7 +24,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() callback := func(i int) int { // wait for leveldb to close, leveldb will be closed in one second time.Sleep(time.Second) diff --git a/store/mockstore/mockcopr/main_test.go b/store/mockstore/mockcopr/main_test.go index 3f3036ba87541..91d38a4cff79c 100644 --- a/store/mockstore/mockcopr/main_test.go +++ b/store/mockstore/mockcopr/main_test.go @@ -24,7 +24,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() opts := []goleak.Option{ goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), diff --git a/store/mockstore/unistore/cophandler/main_test.go b/store/mockstore/unistore/cophandler/main_test.go index e740267de961d..cffc39b4fa1c0 100644 --- a/store/mockstore/unistore/cophandler/main_test.go +++ b/store/mockstore/unistore/cophandler/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/store/mockstore/unistore/lockstore/main_test.go b/store/mockstore/unistore/lockstore/main_test.go index 5674b940ca5e6..0fb96356bfc89 100644 --- a/store/mockstore/unistore/lockstore/main_test.go +++ b/store/mockstore/unistore/lockstore/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/store/mockstore/unistore/main_test.go b/store/mockstore/unistore/main_test.go index 9d44274a8c87d..8b40056fab6d9 100644 --- a/store/mockstore/unistore/main_test.go +++ b/store/mockstore/unistore/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/store/mockstore/unistore/tikv/main_test.go b/store/mockstore/unistore/tikv/main_test.go index 5bc57f47c8321..c74abb998b10a 100644 --- a/store/mockstore/unistore/tikv/main_test.go +++ b/store/mockstore/unistore/tikv/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/store/mockstore/unistore/util/lockwaiter/main_test.go b/store/mockstore/unistore/util/lockwaiter/main_test.go index 6bd0d063dae5c..4a0cf2fdd53de 100644 --- a/store/mockstore/unistore/util/lockwaiter/main_test.go +++ b/store/mockstore/unistore/util/lockwaiter/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/structure/main_test.go b/structure/main_test.go index 793fae6d90647..3874644b636a8 100644 --- a/structure/main_test.go +++ b/structure/main_test.go @@ -22,7 +22,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() opts := []goleak.Option{ goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), diff --git a/table/main_test.go b/table/main_test.go index 91463fb0b5f8c..8ff3f37c63060 100644 --- a/table/main_test.go +++ b/table/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/table/tables/main_test.go b/table/tables/main_test.go index ebfceb2bd3bca..9150776cd9404 100644 --- a/table/tables/main_test.go +++ b/table/tables/main_test.go @@ -22,7 +22,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() opts := []goleak.Option{ goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), diff --git a/table/temptable/main_test.go b/table/temptable/main_test.go index c7baa1e9f5208..2b312cecefc9b 100644 --- a/table/temptable/main_test.go +++ b/table/temptable/main_test.go @@ -38,7 +38,7 @@ func TestMain(m *testing.M) { goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), } - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m, opts...) } diff --git a/tablecodec/main_test.go b/tablecodec/main_test.go index 75ea2dc757133..9785957934b8b 100644 --- a/tablecodec/main_test.go +++ b/tablecodec/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/tablecodec/rowindexcodec/main_test.go b/tablecodec/rowindexcodec/main_test.go index 55b15ba96e15d..7a48bd9d289a1 100644 --- a/tablecodec/rowindexcodec/main_test.go +++ b/tablecodec/rowindexcodec/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/telemetry/cte_test/cte_test.go b/telemetry/cte_test/cte_test.go index 356b65eaad059..5849b285f97e6 100644 --- a/telemetry/cte_test/cte_test.go +++ b/telemetry/cte_test/cte_test.go @@ -33,7 +33,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() opts := []goleak.Option{ goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), diff --git a/telemetry/main_test.go b/telemetry/main_test.go index f498d16a2d564..feb7337e8fac3 100644 --- a/telemetry/main_test.go +++ b/telemetry/main_test.go @@ -27,7 +27,7 @@ var ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() opts := []goleak.Option{ goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), diff --git a/tests/globalkilltest/main_test.go b/tests/globalkilltest/main_test.go index ae4d8e2d63b02..71ee2c7e95780 100644 --- a/tests/globalkilltest/main_test.go +++ b/tests/globalkilltest/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() os.Exit(m.Run()) } diff --git a/tests/graceshutdown/main_test.go b/tests/graceshutdown/main_test.go index 65f1a0ac1d78e..87d7052c17762 100644 --- a/tests/graceshutdown/main_test.go +++ b/tests/graceshutdown/main_test.go @@ -22,7 +22,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() opts := []goleak.Option{ goleak.IgnoreTopFunction("syscall.syscall6"), } diff --git a/tests/readonlytest/main_test.go b/tests/readonlytest/main_test.go index 6d7f7491b6296..9e822e0c6ddb2 100644 --- a/tests/readonlytest/main_test.go +++ b/tests/readonlytest/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/tidb-server/main_test.go b/tidb-server/main_test.go index 84f16eb033561..77c042daf8eae 100644 --- a/tidb-server/main_test.go +++ b/tidb-server/main_test.go @@ -28,7 +28,7 @@ import ( var isCoverageServer string func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() opts := []goleak.Option{ goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), diff --git a/tools/check/go.mod b/tools/check/go.mod index 81ee48b2242cd..5db3382e2d8b9 100644 --- a/tools/check/go.mod +++ b/tools/check/go.mod @@ -1,10 +1,8 @@ module github.com/pingcap/tidb/_tools require ( - github.com/AlekSi/gocov-xml v0.0.0-20190121064608-3a14fb1c4737 github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc // indirect github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf // indirect - github.com/axw/gocov v1.0.0 github.com/chzchzchz/goword v0.0.0-20170907005317-a9744cb52b03 github.com/dnephin/govet v0.0.0-20171012192244-4a96d43e39d3 github.com/kisielk/errcheck v1.2.0 diff --git a/tools/check/go.sum b/tools/check/go.sum index 776ad3f913a32..53cc061d53eec 100644 --- a/tools/check/go.sum +++ b/tools/check/go.sum @@ -1,12 +1,8 @@ -github.com/AlekSi/gocov-xml v0.0.0-20190121064608-3a14fb1c4737 h1:JZHBkt0GhM+ARQykshqpI49yaWCHQbJonH3XpDTwMZQ= -github.com/AlekSi/gocov-xml v0.0.0-20190121064608-3a14fb1c4737/go.mod h1:w1KSuh2JgIL3nyRiZijboSUwbbxOrTzWwyWVFUHtXBQ= github.com/BurntSushi/toml v0.3.0 h1:e1/Ivsx3Z0FVTV0NSOv/aVgbUWyQuzj7DDnFblkRvsY= github.com/BurntSushi/toml v0.3.0/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf h1:qet1QNfXsQxTZqLG4oE62mJzwPIB8+Tee4RNCL9ulrY= github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= -github.com/axw/gocov v1.0.0 h1:YsqYR66hUmilVr23tu8USgnJIJvnwh3n7j5zRn7x4LU= -github.com/axw/gocov v1.0.0/go.mod h1:LvQpEYiwwIb2nYkXY2fDWhg9/AsYqkhmrCshjlUJECE= github.com/chzchzchz/goword v0.0.0-20170907005317-a9744cb52b03 h1:0wUHjDfbCAROEAZ96zAJGwcNMkPIheFaIjtQyv3QqfM= github.com/chzchzchz/goword v0.0.0-20170907005317-a9744cb52b03/go.mod h1:uFE9hX+zXEwvyUThZ4gDb9vkAwc5DoHUnRSEpH0VrOs= github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= diff --git a/types/json/main_test.go b/types/json/main_test.go index a8cf27b5e9331..85aa89330e238 100644 --- a/types/json/main_test.go +++ b/types/json/main_test.go @@ -24,6 +24,6 @@ import ( const benchStr = `{"a":[1,"2",{"aa":"bb"},4,null],"b":true,"c":null}` func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/types/main_test.go b/types/main_test.go index 26127901cab34..73d0d58061bda 100644 --- a/types/main_test.go +++ b/types/main_test.go @@ -22,7 +22,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() opts := []goleak.Option{ goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), diff --git a/types/parser_driver/main_test.go b/types/parser_driver/main_test.go index b1e6a35776689..d3be3b70a0a41 100644 --- a/types/parser_driver/main_test.go +++ b/types/parser_driver/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/admin/main_test.go b/util/admin/main_test.go index b4242e21591ee..e600c43eaefb8 100644 --- a/util/admin/main_test.go +++ b/util/admin/main_test.go @@ -23,7 +23,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() config.UpdateGlobal(func(conf *config.Config) { conf.TiKVClient.AsyncCommit.SafeWindow = 0 diff --git a/util/arena/main_test.go b/util/arena/main_test.go index 020d3424cc9b8..b0b84e0a0bf8a 100644 --- a/util/arena/main_test.go +++ b/util/arena/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/benchdaily/main_test.go b/util/benchdaily/main_test.go index de5653eaa3f98..aa5fe00c5fc55 100644 --- a/util/benchdaily/main_test.go +++ b/util/benchdaily/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/bitmap/main_test.go b/util/bitmap/main_test.go index c32a6a56b6c2a..65b018ed43e14 100644 --- a/util/bitmap/main_test.go +++ b/util/bitmap/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/checksum/main_test.go b/util/checksum/main_test.go index 4212ee3bc431c..4781cf2b6b147 100644 --- a/util/checksum/main_test.go +++ b/util/checksum/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/chunk/main_test.go b/util/chunk/main_test.go index bc9a0900ec180..d3f9eddaa54eb 100644 --- a/util/chunk/main_test.go +++ b/util/chunk/main_test.go @@ -24,7 +24,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() path, _ := os.MkdirTemp("", "oom-use-tmp-storage") config.UpdateGlobal(func(conf *config.Config) { diff --git a/util/codec/main_test.go b/util/codec/main_test.go index cdc978b14d253..b7219d2bfbff3 100644 --- a/util/codec/main_test.go +++ b/util/codec/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/collate/main_test.go b/util/collate/main_test.go index e82127b7013c9..048f02f25b627 100644 --- a/util/collate/main_test.go +++ b/util/collate/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/cteutil/main_test.go b/util/cteutil/main_test.go index f882348ac05cc..74244f4f0eec1 100644 --- a/util/cteutil/main_test.go +++ b/util/cteutil/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/dbterror/main_test.go b/util/dbterror/main_test.go index 7879e9d1a6904..c409af6eea5a7 100644 --- a/util/dbterror/main_test.go +++ b/util/dbterror/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/deadlockhistory/main_test.go b/util/deadlockhistory/main_test.go index f5d8af7b1f799..ea3e208d82692 100644 --- a/util/deadlockhistory/main_test.go +++ b/util/deadlockhistory/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/disjointset/main_test.go b/util/disjointset/main_test.go index 9ae4a8269842b..9145fc2f2d696 100644 --- a/util/disjointset/main_test.go +++ b/util/disjointset/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/disk/main_test.go b/util/disk/main_test.go index 966c9a8e86fa2..bef823c25baa9 100644 --- a/util/disk/main_test.go +++ b/util/disk/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/encrypt/main_test.go b/util/encrypt/main_test.go index 29fb8827d96e0..e6df43ef9941d 100644 --- a/util/encrypt/main_test.go +++ b/util/encrypt/main_test.go @@ -22,7 +22,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/execdetails/main_test.go b/util/execdetails/main_test.go index a55b06c98e775..0388a932da8c5 100644 --- a/util/execdetails/main_test.go +++ b/util/execdetails/main_test.go @@ -21,6 +21,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/expensivequery/expensivequerey_test.go b/util/expensivequery/expensivequerey_test.go index 8f5c60179694e..9852780a20b09 100644 --- a/util/expensivequery/expensivequerey_test.go +++ b/util/expensivequery/expensivequerey_test.go @@ -27,7 +27,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/fastrand/main_test.go b/util/fastrand/main_test.go index fb2255c79e252..a0ec62f151863 100644 --- a/util/fastrand/main_test.go +++ b/util/fastrand/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/format/main_test.go b/util/format/main_test.go index 66294a0dbadfb..e0269371cc883 100644 --- a/util/format/main_test.go +++ b/util/format/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/generatedexpr/main_test.go b/util/generatedexpr/main_test.go index d5418deebddcf..51ad2d1335ce0 100644 --- a/util/generatedexpr/main_test.go +++ b/util/generatedexpr/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/hack/main_test.go b/util/hack/main_test.go index aaa708c42a033..193c9492d3063 100644 --- a/util/hack/main_test.go +++ b/util/hack/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/keydecoder/main_test.go b/util/keydecoder/main_test.go index 6900f8e304ec0..1dd3d666b6a67 100644 --- a/util/keydecoder/main_test.go +++ b/util/keydecoder/main_test.go @@ -26,6 +26,6 @@ func TestMain(m *testing.M) { goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), } - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m, opts...) } diff --git a/util/kvcache/main_test.go b/util/kvcache/main_test.go index 90e09a3a45e77..9b4ead41b9f2d 100644 --- a/util/kvcache/main_test.go +++ b/util/kvcache/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/localpool/main_test.go b/util/localpool/main_test.go index fb21dcc42f4f5..35b66eda56f43 100644 --- a/util/localpool/main_test.go +++ b/util/localpool/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/logutil/main_test.go b/util/logutil/main_test.go index 7d833f0bd86ef..0113d6f75ee6c 100644 --- a/util/logutil/main_test.go +++ b/util/logutil/main_test.go @@ -34,7 +34,7 @@ var ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() opts := []goleak.Option{ goleak.IgnoreTopFunction("gopkg.in/natefinch/lumberjack%2ev2.(*Logger).millRun"), } diff --git a/util/main_test.go b/util/main_test.go index 1b930670688d0..0a5c3ac1a47da 100644 --- a/util/main_test.go +++ b/util/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/math/main_test.go b/util/math/main_test.go index c10b457e497a8..b315c0487ce89 100644 --- a/util/math/main_test.go +++ b/util/math/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/memory/main_test.go b/util/memory/main_test.go index 03d7f7d266e77..34ce2a33e66c3 100644 --- a/util/memory/main_test.go +++ b/util/memory/main_test.go @@ -15,22 +15,14 @@ package memory import ( - "fmt" - "os" "testing" - "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/testbridge" "go.uber.org/goleak" ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() - logLevel := os.Getenv("log_level") - err := logutil.InitLogger(logutil.NewLogConfig(logLevel, logutil.DefaultLogFormat, "", logutil.EmptyFileLogConfig, false)) - if err != nil { - fmt.Fprint(os.Stderr, err.Error()) - os.Exit(1) - } + testbridge.SetupForCommonTest() + goleak.VerifyTestMain(m) } diff --git a/util/mock/main_test.go b/util/mock/main_test.go index 11d845761a1ec..e855ad497df38 100644 --- a/util/mock/main_test.go +++ b/util/mock/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/mvmap/main_test.go b/util/mvmap/main_test.go index 86dd20958e3a6..39178d04f26fe 100644 --- a/util/mvmap/main_test.go +++ b/util/mvmap/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/parser/main_test.go b/util/parser/main_test.go index eef165cfd2fcf..f2cc66445b8e2 100644 --- a/util/parser/main_test.go +++ b/util/parser/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/plancodec/main_test.go b/util/plancodec/main_test.go index 31620cb293173..65fc11aefa4bd 100644 --- a/util/plancodec/main_test.go +++ b/util/plancodec/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/printer/main_test.go b/util/printer/main_test.go index ae6fbd3e45509..1f5f8dda193b5 100644 --- a/util/printer/main_test.go +++ b/util/printer/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/profile/main_test.go b/util/profile/main_test.go index cd5b368a787de..d3ddfb5ac3238 100644 --- a/util/profile/main_test.go +++ b/util/profile/main_test.go @@ -22,7 +22,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() opts := []goleak.Option{ goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), diff --git a/util/ranger/main_test.go b/util/ranger/main_test.go index f0f1652c18edd..410ac374b04d6 100644 --- a/util/ranger/main_test.go +++ b/util/ranger/main_test.go @@ -29,7 +29,7 @@ var testDataMap = make(testdata.BookKeeper, 1) var rangerSuiteData testdata.TestData func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() flag.Parse() diff --git a/util/resourcegrouptag/main_test.go b/util/resourcegrouptag/main_test.go index 9bd1c8df16cd4..0ab522e6ae8bb 100644 --- a/util/resourcegrouptag/main_test.go +++ b/util/resourcegrouptag/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/rowDecoder/main_test.go b/util/rowDecoder/main_test.go index 0e93bf3104a9b..1a843cbfe6b69 100644 --- a/util/rowDecoder/main_test.go +++ b/util/rowDecoder/main_test.go @@ -26,6 +26,6 @@ func TestMain(m *testing.M) { goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), } - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m, opts...) } diff --git a/util/rowcodec/main_test.go b/util/rowcodec/main_test.go index 7a54747ceaf72..b333276b3c672 100644 --- a/util/rowcodec/main_test.go +++ b/util/rowcodec/main_test.go @@ -25,7 +25,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/selection/main_test.go b/util/selection/main_test.go index 91e31ed6c5bb3..a35eb9a4bfe88 100644 --- a/util/selection/main_test.go +++ b/util/selection/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/sem/main_test.go b/util/sem/main_test.go index 81af71ff59d39..7c8abcb5b5049 100644 --- a/util/sem/main_test.go +++ b/util/sem/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/set/main_test.go b/util/set/main_test.go index d119da45ccb6b..1a068e7eed801 100644 --- a/util/set/main_test.go +++ b/util/set/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/slice/main_test.go b/util/slice/main_test.go index 2da7631bca34f..08c038fe064dd 100644 --- a/util/slice/main_test.go +++ b/util/slice/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/sqlexec/main_test.go b/util/sqlexec/main_test.go index 7aa194b084bf0..672e7b4343d24 100644 --- a/util/sqlexec/main_test.go +++ b/util/sqlexec/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/stmtsummary/main_test.go b/util/stmtsummary/main_test.go index 99143834fd420..43e9e63d30c30 100644 --- a/util/stmtsummary/main_test.go +++ b/util/stmtsummary/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/stringutil/main_test.go b/util/stringutil/main_test.go index 7f012cc923dba..0690bad2d85f4 100644 --- a/util/stringutil/main_test.go +++ b/util/stringutil/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/sys/linux/main_test.go b/util/sys/linux/main_test.go index 4c190736347de..052580b6f3b86 100644 --- a/util/sys/linux/main_test.go +++ b/util/sys/linux/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/sys/storage/main_test.go b/util/sys/storage/main_test.go index 846992285e92a..ac3a5f06caeaa 100644 --- a/util/sys/storage/main_test.go +++ b/util/sys/storage/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/systimemon/main_test.go b/util/systimemon/main_test.go index 2d5053b2b2a93..2a216674ce419 100644 --- a/util/systimemon/main_test.go +++ b/util/systimemon/main_test.go @@ -22,7 +22,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() opts := []goleak.Option{ goleak.IgnoreTopFunction("github.com/pingcap/tidb/util/systimemon.StartMonitor"), diff --git a/util/testbridge/bridge.go b/util/testbridge/bridge.go index 1aee33f9c0a15..a682a6574fa3c 100644 --- a/util/testbridge/bridge.go +++ b/util/testbridge/bridge.go @@ -19,18 +19,47 @@ package testbridge import ( "flag" + "fmt" + "os" + + "github.com/pingcap/log" + "go.uber.org/zap" + "go.uber.org/zap/zapcore" ) -// WorkaroundGoCheckFlags registers flags of go-check for pkg does not import go-check +// SetupForCommonTest runs before all the tests. +func SetupForCommonTest() { + workaroundGoCheckFlags() + applyOSLogLevel() +} + +// workaroundGoCheckFlags registers flags of go-check for pkg does not import go-check // to workaround the go-check flags passed in Makefile. // // TODO: Remove this function when the migration from go-check to testify[1] is done. // [1] https://github.com/pingcap/tidb/issues/26022 -func WorkaroundGoCheckFlags() { +func workaroundGoCheckFlags() { if flag.Lookup("check.timeout") == nil { - _ = flag.Duration("check.timeout", 0, "WorkaroundGoCheckFlags: check.timeout") + _ = flag.Duration("check.timeout", 0, "workaroundGoCheckFlags: check.timeout") } if flag.Lookup("check.p") == nil { - _ = flag.Bool("check.p", false, "WorkaroundGoCheckFlags: check.p") + _ = flag.Bool("check.p", false, "workaroundGoCheckFlags: check.p") + } +} + +func applyOSLogLevel() { + osLoglevel := os.Getenv("log_level") + if len(osLoglevel) > 0 { + cfg := log.Config{ + Level: osLoglevel, + Format: "text", + File: log.FileLogConfig{}, + } + gl, props, err := log.InitLogger(&cfg, zap.AddStacktrace(zapcore.FatalLevel)) + if err != nil { + _, _ = fmt.Fprintf(os.Stderr, "applyOSLogLevel failed: %v", err) + os.Exit(-1) + } + log.ReplaceGlobals(gl, props) } } diff --git a/util/texttree/main_test.go b/util/texttree/main_test.go index af4e400aeaa61..8aa6c0a957425 100644 --- a/util/texttree/main_test.go +++ b/util/texttree/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/timeutil/main_test.go b/util/timeutil/main_test.go index 910501d567d05..9d30909025632 100644 --- a/util/timeutil/main_test.go +++ b/util/timeutil/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/topsql/main_test.go b/util/topsql/main_test.go index f5e3dc3f7d0cf..b567382484285 100644 --- a/util/topsql/main_test.go +++ b/util/topsql/main_test.go @@ -25,7 +25,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() // set up variable.TopSQLVariable.Enable.Store(true) diff --git a/util/topsql/reporter/main_test.go b/util/topsql/reporter/main_test.go index a828a948fb4fd..b06eb67a844cf 100644 --- a/util/topsql/reporter/main_test.go +++ b/util/topsql/reporter/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/topsql/stmtstats/main_test.go b/util/topsql/stmtstats/main_test.go index 24f6c2574c522..cee86b8217e18 100644 --- a/util/topsql/stmtstats/main_test.go +++ b/util/topsql/stmtstats/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/topsql/stmtstats/stmtstatstest/main_test.go b/util/topsql/stmtstats/stmtstatstest/main_test.go index ecf1220642ecf..033634a360cc0 100644 --- a/util/topsql/stmtstats/stmtstatstest/main_test.go +++ b/util/topsql/stmtstats/stmtstatstest/main_test.go @@ -22,7 +22,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() opts := []goleak.Option{ goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), diff --git a/util/topsql/tracecpu/main_test.go b/util/topsql/tracecpu/main_test.go index 74352d78d7419..2e6d98c89b32e 100644 --- a/util/topsql/tracecpu/main_test.go +++ b/util/topsql/tracecpu/main_test.go @@ -28,7 +28,7 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() variable.TopSQLVariable.Enable.Store(false) config.UpdateGlobal(func(conf *config.Config) { diff --git a/util/tracing/main_test.go b/util/tracing/main_test.go index 17e67ecd40ea8..c684666e3216f 100644 --- a/util/tracing/main_test.go +++ b/util/tracing/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } diff --git a/util/vitess/main_test.go b/util/vitess/main_test.go index a66f334f9bba9..4151a95a5b7d1 100644 --- a/util/vitess/main_test.go +++ b/util/vitess/main_test.go @@ -22,6 +22,6 @@ import ( ) func TestMain(m *testing.M) { - testbridge.WorkaroundGoCheckFlags() + testbridge.SetupForCommonTest() goleak.VerifyTestMain(m) } From ab821eb315657a7b85de9d7519c2d377b6dda79a Mon Sep 17 00:00:00 2001 From: you06 Date: Fri, 24 Dec 2021 18:23:53 +0800 Subject: [PATCH 43/44] planner: add usage of paging copr in optimizer (#30536) close pingcap/tidb#30578 --- distsql/distsql.go | 1 + distsql/select_result.go | 9 +++-- distsql/stream.go | 2 +- executor/builder.go | 6 ++++ executor/distsql.go | 2 ++ metrics/distsql.go | 2 +- metrics/grafana/tidb.json | 4 +-- metrics/session.go | 1 + planner/core/explain.go | 11 ++++-- planner/core/find_best_task.go | 1 + planner/core/physical_plans.go | 1 + planner/core/plan_test.go | 64 ++++++++++++++++++++++++++++++++++ planner/core/task.go | 52 ++++++++++++++++++++++++++- store/copr/coprocessor.go | 25 ++----------- store/copr/coprocessor_test.go | 3 +- util/paging/main_test.go | 27 ++++++++++++++ util/paging/paging.go | 61 ++++++++++++++++++++++++++++++++ util/paging/paging_test.go | 36 +++++++++++++++++++ 18 files changed, 275 insertions(+), 33 deletions(-) create mode 100644 util/paging/main_test.go create mode 100644 util/paging/paging.go create mode 100644 util/paging/paging_test.go diff --git a/distsql/distsql.go b/distsql/distsql.go index 77b75efc480fd..5581f9c269c99 100644 --- a/distsql/distsql.go +++ b/distsql/distsql.go @@ -133,6 +133,7 @@ func Select(ctx context.Context, sctx sessionctx.Context, kvReq *kv.Request, fie memTracker: kvReq.MemTracker, encodeType: encodetype, storeType: kvReq.StoreType, + paging: kvReq.Paging, }, nil } diff --git a/distsql/select_result.go b/distsql/select_result.go index 3ac7f1db94a97..b2eef6e6f5300 100644 --- a/distsql/select_result.go +++ b/distsql/select_result.go @@ -152,7 +152,8 @@ type selectResult struct { durationReported bool memTracker *memory.Tracker - stats *selectResultRuntimeStats + stats *selectResultRuntimeStats + paging bool } func (r *selectResult) fetchResp(ctx context.Context) error { @@ -206,7 +207,11 @@ func (r *selectResult) fetchResp(ctx context.Context) error { // final round of fetch // TODO: Add a label to distinguish between success or failure. // https://github.com/pingcap/tidb/issues/11397 - metrics.DistSQLQueryHistogram.WithLabelValues(r.label, r.sqlType).Observe(r.fetchDuration.Seconds()) + if r.paging { + metrics.DistSQLQueryHistogram.WithLabelValues(r.label, r.sqlType, "paging").Observe(r.fetchDuration.Seconds()) + } else { + metrics.DistSQLQueryHistogram.WithLabelValues(r.label, r.sqlType, "common").Observe(r.fetchDuration.Seconds()) + } r.durationReported = true } return nil diff --git a/distsql/stream.go b/distsql/stream.go index 73d8f96b8fe79..2f10e72b5c0f5 100644 --- a/distsql/stream.go +++ b/distsql/stream.go @@ -82,7 +82,7 @@ func (r *streamResult) readDataFromResponse(ctx context.Context, resp kv.Respons if !r.durationReported { // TODO: Add a label to distinguish between success or failure. // https://github.com/pingcap/tidb/issues/11397 - metrics.DistSQLQueryHistogram.WithLabelValues(r.label, r.sqlType).Observe(r.fetchDuration.Seconds()) + metrics.DistSQLQueryHistogram.WithLabelValues(r.label, r.sqlType, "streaming").Observe(r.fetchDuration.Seconds()) r.durationReported = true } return true, nil diff --git a/executor/builder.go b/executor/builder.go index 78c94fd02f1aa..ef932da1211fa 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -3409,6 +3409,11 @@ func buildNoRangeIndexLookUpReader(b *executorBuilder, v *plannercore.PhysicalIn if err != nil { return nil, err } + indexPaging := false + if v.Paging { + indexPaging = true + indexStreaming = false + } tableReq, tableStreaming, tbl, err := buildTableReq(b, v.Schema().Len(), v.TablePlans) if err != nil { return nil, err @@ -3430,6 +3435,7 @@ func buildNoRangeIndexLookUpReader(b *executorBuilder, v *plannercore.PhysicalIn columns: ts.Columns, indexStreaming: indexStreaming, tableStreaming: tableStreaming, + indexPaging: indexPaging, dataReaderBuilder: &dataReaderBuilder{executorBuilder: b}, corColInIdxSide: b.corColInDistPlan(v.IndexPlans), corColInTblSide: b.corColInDistPlan(v.TablePlans), diff --git a/executor/distsql.go b/executor/distsql.go index 1c31130dc53b5..9d380883703cd 100644 --- a/executor/distsql.go +++ b/executor/distsql.go @@ -361,6 +361,7 @@ type IndexLookUpExecutor struct { indexStreaming bool tableStreaming bool + indexPaging bool corColInIdxSide bool corColInTblSide bool @@ -560,6 +561,7 @@ func (e *IndexLookUpExecutor) startIndexWorker(ctx context.Context, workCh chan< SetDesc(e.desc). SetKeepOrder(e.keepOrder). SetStreaming(e.indexStreaming). + SetPaging(e.indexPaging). SetReadReplicaScope(e.readReplicaScope). SetIsStaleness(e.isStaleness). SetFromSessionVars(e.ctx.GetSessionVars()). diff --git a/metrics/distsql.go b/metrics/distsql.go index 9bec9d7646827..3a4527da510ae 100644 --- a/metrics/distsql.go +++ b/metrics/distsql.go @@ -27,7 +27,7 @@ var ( Name: "handle_query_duration_seconds", Help: "Bucketed histogram of processing time (s) of handled queries.", Buckets: prometheus.ExponentialBuckets(0.0005, 2, 29), // 0.5ms ~ 1.5days - }, []string{LblType, LblSQLType}) + }, []string{LblType, LblSQLType, LblCoprType}) DistSQLScanKeysPartialHistogram = prometheus.NewHistogram( prometheus.HistogramOpts{ diff --git a/metrics/grafana/tidb.json b/metrics/grafana/tidb.json index f6606d18ef4c6..a828967a8ada9 100644 --- a/metrics/grafana/tidb.json +++ b/metrics/grafana/tidb.json @@ -6168,10 +6168,10 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tidb_distsql_handle_query_duration_seconds_count{tidb_cluster=\"$tidb_cluster\"}[1m]))", + "expr": "sum(rate(tidb_distsql_handle_query_duration_seconds_count{tidb_cluster=\"$tidb_cluster\"}[1m])) by (copr_type)", "format": "time_series", "intervalFactor": 2, - "legendFormat": "", + "legendFormat": "{{copr_type}}", "metric": "tidb_distsql_query_total", "refId": "A", "step": 4 diff --git a/metrics/session.go b/metrics/session.go index 0058104788f21..83df91439d311 100644 --- a/metrics/session.go +++ b/metrics/session.go @@ -142,6 +142,7 @@ const ( LblDb = "db" LblResult = "result" LblSQLType = "sql_type" + LblCoprType = "copr_type" LblGeneral = "general" LblInternal = "internal" LbTxnMode = "txn_mode" diff --git a/planner/core/explain.go b/planner/core/explain.go index 26bfa775fc417..d0ae474b79e7f 100644 --- a/planner/core/explain.go +++ b/planner/core/explain.go @@ -456,17 +456,22 @@ func (p *PhysicalIndexReader) accessObject(sctx sessionctx.Context) string { // ExplainInfo implements Plan interface. func (p *PhysicalIndexLookUpReader) ExplainInfo() string { + var str strings.Builder // The children can be inferred by the relation symbol. if p.PushedLimit != nil { - var str strings.Builder str.WriteString("limit embedded(offset:") str.WriteString(strconv.FormatUint(p.PushedLimit.Offset, 10)) str.WriteString(", count:") str.WriteString(strconv.FormatUint(p.PushedLimit.Count, 10)) str.WriteString(")") - return str.String() } - return "" + if p.Paging { + if p.PushedLimit != nil { + str.WriteString(", ") + } + str.WriteString("paging:true") + } + return str.String() } func (p *PhysicalIndexLookUpReader) accessObject(sctx sessionctx.Context) string { diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index 5157ffb738b24..ff90a92b9b497 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -1249,6 +1249,7 @@ func (ds *DataSource) convertToIndexScan(prop *property.PhysicalProperty, candid indexPlan: is, tblColHists: ds.TblColHists, tblCols: ds.TblCols, + expectCnt: uint64(prop.ExpectedCnt), } cop.partitionInfo = PartitionInfo{ PruningConds: ds.allConds, diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index 17e84b6efcf47..6293bba4b5073 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -270,6 +270,7 @@ type PhysicalIndexLookUpReader struct { TablePlans []PhysicalPlan indexPlan PhysicalPlan tablePlan PhysicalPlan + Paging bool ExtraHandleCol *expression.Column // PushedLimit is used to avoid unnecessary table scan tasks of IndexLookUpReader. diff --git a/planner/core/plan_test.go b/planner/core/plan_test.go index 4437a354b1757..288444c9f2c12 100644 --- a/planner/core/plan_test.go +++ b/planner/core/plan_test.go @@ -18,6 +18,7 @@ import ( "bytes" "fmt" "strings" + "testing" . "github.com/pingcap/check" "github.com/pingcap/tidb/config" @@ -26,6 +27,7 @@ import ( "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx/variable" + kit "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/util/israce" "github.com/pingcap/tidb/util/plancodec" "github.com/pingcap/tidb/util/testkit" @@ -639,3 +641,65 @@ func (s *testPlanNormalize) TestIssue25729(c *C) { tk.MustExec("insert into t1 values(\"a\", \"adwa\");") tk.MustQuery("select * from t1 where concat(a, b) like \"aadwa\" and a = \"a\";").Check(testkit.Rows("a adwa")) } + +func TestCopPaging(t *testing.T) { + store, clean := kit.CreateMockStore(t) + defer clean() + + tk := kit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("set session tidb_enable_paging = 1") + tk.MustExec("create table t(id int, c1 int, c2 int, primary key (id), key i(c1))") + defer tk.MustExec("drop table t") + for i := 0; i < 1024; i++ { + tk.MustExec("insert into t values(?, ?, ?)", i, i, i) + } + tk.MustExec("analyze table t") + + // limit 960 should go paging + for i := 0; i < 10; i++ { + tk.MustQuery("explain format='brief' select * from t force index(i) where id <= 1024 and c1 >= 0 and c1 <= 1024 and c2 in (2, 4, 6, 8) order by c1 limit 960").Check(kit.Rows( + "Limit 4.00 root offset:0, count:960", + "└─IndexLookUp 4.00 root paging:true", + " ├─Selection(Build) 1024.00 cop[tikv] le(test.t.id, 1024)", + " │ └─IndexRangeScan 1024.00 cop[tikv] table:t, index:i(c1) range:[0,1024], keep order:true", + " └─Selection(Probe) 4.00 cop[tikv] in(test.t.c2, 2, 4, 6, 8)", + " └─TableRowIDScan 1024.00 cop[tikv] table:t keep order:false")) + } + + // selection between limit and indexlookup, limit 960 should also go paging + for i := 0; i < 10; i++ { + tk.MustQuery("explain format='brief' select * from t force index(i) where mod(id, 2) > 0 and id <= 1024 and c1 >= 0 and c1 <= 1024 and c2 in (2, 4, 6, 8) order by c1 limit 960").Check(kit.Rows( + "Limit 3.20 root offset:0, count:960", + "└─Selection 2.56 root gt(mod(test.t.id, 2), 0)", + " └─IndexLookUp 3.20 root paging:true", + " ├─Selection(Build) 819.20 cop[tikv] le(test.t.id, 1024)", + " │ └─IndexRangeScan 1024.00 cop[tikv] table:t, index:i(c1) range:[0,1024], keep order:true", + " └─Selection(Probe) 3.20 cop[tikv] in(test.t.c2, 2, 4, 6, 8)", + " └─TableRowIDScan 819.20 cop[tikv] table:t keep order:false")) + } + + // limit 961 exceeds the threshold, it should not go paging + for i := 0; i < 10; i++ { + tk.MustQuery("explain format='brief' select * from t force index(i) where id <= 1024 and c1 >= 0 and c1 <= 1024 and c2 in (2, 4, 6, 8) order by c1 limit 961").Check(kit.Rows( + "Limit 4.00 root offset:0, count:961", + "└─IndexLookUp 4.00 root ", + " ├─Selection(Build) 1024.00 cop[tikv] le(test.t.id, 1024)", + " │ └─IndexRangeScan 1024.00 cop[tikv] table:t, index:i(c1) range:[0,1024], keep order:true", + " └─Selection(Probe) 4.00 cop[tikv] in(test.t.c2, 2, 4, 6, 8)", + " └─TableRowIDScan 1024.00 cop[tikv] table:t keep order:false")) + } + + // selection between limit and indexlookup, limit 961 should not go paging too + for i := 0; i < 10; i++ { + tk.MustQuery("explain format='brief' select * from t force index(i) where mod(id, 2) > 0 and id <= 1024 and c1 >= 0 and c1 <= 1024 and c2 in (2, 4, 6, 8) order by c1 limit 961").Check(kit.Rows( + "Limit 3.20 root offset:0, count:961", + "└─Selection 2.56 root gt(mod(test.t.id, 2), 0)", + " └─IndexLookUp 3.20 root ", + " ├─Selection(Build) 819.20 cop[tikv] le(test.t.id, 1024)", + " │ └─IndexRangeScan 1024.00 cop[tikv] table:t, index:i(c1) range:[0,1024], keep order:true", + " └─Selection(Probe) 3.20 cop[tikv] in(test.t.c2, 2, 4, 6, 8)", + " └─TableRowIDScan 819.20 cop[tikv] table:t keep order:false")) + } +} diff --git a/planner/core/task.go b/planner/core/task.go index 4b9c5692ca29d..90fbcedacaa9b 100644 --- a/planner/core/task.go +++ b/planner/core/task.go @@ -35,6 +35,7 @@ import ( "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/logutil" + "github.com/pingcap/tidb/util/paging" "github.com/pingcap/tidb/util/plancodec" "github.com/pingcap/tipb/go-tipb" "go.uber.org/zap" @@ -89,6 +90,10 @@ type copTask struct { // For table partition. partitionInfo PartitionInfo + + // expectCnt is the expected row count of upper task, 0 for unlimited. + // It's used for deciding whether using paging distsql. + expectCnt uint64 } func (t *copTask) invalid() bool { @@ -914,7 +919,17 @@ func buildIndexLookUpTask(ctx sessionctx.Context, t *copTask) *rootTask { // (indexRows / batchSize) * batchSize * CPUFactor // Since we don't know the number of copTasks built, ignore these network cost now. indexRows := t.indexPlan.statsInfo().RowCount - newTask.cst += indexRows * sessVars.CPUFactor + idxCst := indexRows * sessVars.CPUFactor + // if the expectCnt is below the paging threshold, using paging API, recalculate idxCst. + // paging API reduces the count of index and table rows, however introduces more seek cost. + if ctx.GetSessionVars().EnablePaging && t.expectCnt > 0 && t.expectCnt <= paging.Threshold { + p.Paging = true + pagingCst := calcPagingCost(ctx, t) + // prevent enlarging the cost because we take paging as a better plan, + // if the cost is enlarged, it'll be easier to go another plan. + idxCst = math.Min(idxCst, pagingCst) + } + newTask.cst += idxCst // Add cost of worker goroutines in index lookup. numTblWorkers := float64(sessVars.IndexLookupConcurrency()) newTask.cst += (numTblWorkers + 1) * sessVars.ConcurrencyFactor @@ -951,6 +966,41 @@ func buildIndexLookUpTask(ctx sessionctx.Context, t *copTask) *rootTask { return newTask } +func extractRows(p PhysicalPlan) float64 { + f := float64(0) + for _, c := range p.Children() { + if len(c.Children()) != 0 { + f += extractRows(c) + } else { + f += c.statsInfo().RowCount + } + } + return f +} + +// calcPagingCost calculates the cost for paging processing which may increase the seekCnt and reduce scanned rows. +func calcPagingCost(ctx sessionctx.Context, t *copTask) float64 { + sessVars := ctx.GetSessionVars() + indexRows := t.indexPlan.statsInfo().RowCount + expectCnt := t.expectCnt + sourceRows := extractRows(t.indexPlan) + // with paging, the scanned rows is always less than or equal to source rows. + if uint64(sourceRows) < expectCnt { + expectCnt = uint64(sourceRows) + } + seekCnt := paging.CalculateSeekCnt(expectCnt) + indexSelectivity := float64(1) + if sourceRows > indexRows { + indexSelectivity = indexRows / sourceRows + } + pagingCst := seekCnt*sessVars.GetSeekFactor(nil) + float64(expectCnt)*sessVars.CPUFactor + pagingCst *= indexSelectivity + + // we want the diff between idxCst and pagingCst here, + // however, the idxCst does not contain seekFactor, so a seekFactor needs to be removed + return pagingCst - sessVars.GetSeekFactor(nil) +} + func (t *rootTask) convertToRootTask(_ sessionctx.Context) *rootTask { return t.copy().(*rootTask) } diff --git a/store/copr/coprocessor.go b/store/copr/coprocessor.go index 25ed965ea15e3..04e29a8507656 100644 --- a/store/copr/coprocessor.go +++ b/store/copr/coprocessor.go @@ -42,6 +42,7 @@ import ( "github.com/pingcap/tidb/util/execdetails" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/memory" + "github.com/pingcap/tidb/util/paging" "github.com/pingcap/tidb/util/trxevents" "github.com/pingcap/tipb/go-tipb" "github.com/tikv/client-go/v2/metrics" @@ -61,18 +62,6 @@ const ( copNextMaxBackoff = 20000 ) -// A paging request may be separated into multi requests if there are more data than a page. -// The paging size grows from min to max, it's not well tuned yet. -// e.g. a paging request scans over range (r1, r200), it requires 64 rows in the first batch, -// if it's not drained, then the paging size grows, the new range is calculated like (r100, r200), then send a request again. -// Compare with the common unary request, paging request allows early access of data, it offers a streaming-like way processing data. -// TODO: may make the paging parameters configurable. -const ( - minPagingSize uint64 = 64 - maxPagingSize = minPagingSize * 128 - pagingSizeGrow uint64 = 2 -) - // CopClient is coprocessor client. type CopClient struct { kv.RequestTypeSupportedChecker @@ -212,7 +201,7 @@ func buildCopTasks(bo *Backoffer, cache *RegionCache, ranges *KeyRanges, req *kv // the size will grow every round. pagingSize := uint64(0) if req.Paging { - pagingSize = minPagingSize + pagingSize = paging.MinPagingSize } tasks = append(tasks, &copTask{ region: loc.Location.Region, @@ -928,7 +917,7 @@ func (worker *copIteratorWorker) handleCopPagingResult(bo *Backoffer, rpcCtx *ti if task.ranges.Len() == 0 { return nil, nil } - task.pagingSize = growPagingSize(task.pagingSize) + task.pagingSize = paging.GrowPagingSize(task.pagingSize) return []*copTask{task}, nil } @@ -1332,11 +1321,3 @@ func isolationLevelToPB(level kv.IsoLevel) kvrpcpb.IsolationLevel { return kvrpcpb.IsolationLevel_SI } } - -func growPagingSize(size uint64) uint64 { - size *= pagingSizeGrow - if size > maxPagingSize { - return maxPagingSize - } - return size -} diff --git a/store/copr/coprocessor_test.go b/store/copr/coprocessor_test.go index 88ad5568f68eb..b628b4eaab831 100644 --- a/store/copr/coprocessor_test.go +++ b/store/copr/coprocessor_test.go @@ -21,6 +21,7 @@ import ( "github.com/pingcap/kvproto/pkg/coprocessor" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/store/driver/backoff" + "github.com/pingcap/tidb/util/paging" "github.com/stretchr/testify/require" "github.com/tikv/client-go/v2/testutils" "github.com/tikv/client-go/v2/tikv" @@ -318,7 +319,7 @@ func TestBuildPagingTasks(t *testing.T) { require.Len(t, tasks, 1) taskEqual(t, tasks[0], regionIDs[0], "a", "c") require.True(t, tasks[0].paging) - require.Equal(t, tasks[0].pagingSize, minPagingSize) + require.Equal(t, tasks[0].pagingSize, paging.MinPagingSize) } func toCopRange(r kv.KeyRange) *coprocessor.KeyRange { diff --git a/util/paging/main_test.go b/util/paging/main_test.go new file mode 100644 index 0000000000000..af568af279474 --- /dev/null +++ b/util/paging/main_test.go @@ -0,0 +1,27 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package paging + +import ( + "testing" + + "github.com/pingcap/tidb/util/testbridge" + "go.uber.org/goleak" +) + +func TestMain(m *testing.M) { + testbridge.SetupForCommonTest() + goleak.VerifyTestMain(m) +} diff --git a/util/paging/paging.go b/util/paging/paging.go new file mode 100644 index 0000000000000..5f2618ea341db --- /dev/null +++ b/util/paging/paging.go @@ -0,0 +1,61 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package paging + +import "math" + +// A paging request may be separated into multi requests if there are more data than a page. +// The paging size grows from min to max, it's not well tuned yet. +// e.g. a paging request scans over range (r1, r200), it requires 64 rows in the first batch, +// if it's not drained, then the paging size grows, the new range is calculated like (r100, r200), then send a request again. +// Compare with the common unary request, paging request allows early access of data, it offers a streaming-like way processing data. +// TODO: may make the paging parameters configurable. +const ( + MinPagingSize uint64 = 64 + maxPagingSizeShift = 7 + pagingSizeGrow = 2 + MaxPagingSize = MinPagingSize << maxPagingSizeShift + pagingGrowingSum = ((2 << maxPagingSizeShift) - 1) * MinPagingSize + Threshold uint64 = 960 +) + +// GrowPagingSize grows the paging size and ensures it does not exceed MaxPagingSize +func GrowPagingSize(size uint64) uint64 { + size <<= 1 + if size > MaxPagingSize { + return MaxPagingSize + } + return size +} + +// CalculateSeekCnt calculates the seek count from expect count +func CalculateSeekCnt(expectCnt uint64) float64 { + if expectCnt == 0 { + return 0 + } + if expectCnt > pagingGrowingSum { + // if the expectCnt is larger than pagingGrowingSum, calculate the seekCnt for the excess. + return float64(8 + (expectCnt-pagingGrowingSum+MaxPagingSize-1)/MaxPagingSize) + } + if expectCnt > MinPagingSize { + // if the expectCnt is less than pagingGrowingSum, + // calculate the seekCnt(number of terms) from the sum of a geometric progression. + // expectCnt = minPagingSize * (pagingSizeGrow ^ seekCnt - 1) / (pagingSizeGrow - 1) + // simplify (pagingSizeGrow ^ seekCnt - 1) to pagingSizeGrow ^ seekCnt, we can infer that + // seekCnt = log((pagingSizeGrow - 1) * expectCnt / minPagingSize) / log(pagingSizeGrow) + return 1 + float64(int(math.Log(float64((pagingSizeGrow-1)*expectCnt)/float64(MinPagingSize))/math.Log(float64(pagingSizeGrow)))) + } + return 1 +} diff --git a/util/paging/paging_test.go b/util/paging/paging_test.go new file mode 100644 index 0000000000000..1890b4d754d54 --- /dev/null +++ b/util/paging/paging_test.go @@ -0,0 +1,36 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package paging + +import ( + "testing" + + "github.com/stretchr/testify/require" +) + +func TestGrowPagingSize(t *testing.T) { + require.Equal(t, GrowPagingSize(MinPagingSize), MinPagingSize*pagingSizeGrow) + require.Equal(t, GrowPagingSize(MaxPagingSize), MaxPagingSize) + require.Equal(t, GrowPagingSize(MaxPagingSize/pagingSizeGrow+1), MaxPagingSize) +} + +func TestCalculateSeekCnt(t *testing.T) { + require.InDelta(t, CalculateSeekCnt(0), 0, 0.1) + require.InDelta(t, CalculateSeekCnt(1), 1, 0.1) + require.InDelta(t, CalculateSeekCnt(MinPagingSize), 1, 0.1) + require.InDelta(t, CalculateSeekCnt(pagingGrowingSum), maxPagingSizeShift+1, 0.1) + require.InDelta(t, CalculateSeekCnt(pagingGrowingSum+1), maxPagingSizeShift+2, 0.1) + require.InDelta(t, CalculateSeekCnt(pagingGrowingSum+MaxPagingSize), maxPagingSizeShift+2, 0.1) +} From 9ad00962f6561b25fe0fe5290ba16de1c3901ff7 Mon Sep 17 00:00:00 2001 From: tangenta Date: Fri, 24 Dec 2021 20:21:53 +0800 Subject: [PATCH 44/44] test: merge serial tests in cmd, planner, server, util (#31003) --- .../{column_serial_test.go => column_test.go} | 0 .../{ddl_serial_test.go => ddl_test.go} | 0 .../{index_serial_test.go => index_test.go} | 0 ...o_pb_serial_test.go => plan_to_pb_test.go} | 0 ...ial_test.go => statistics_handler_test.go} | 0 ...ext_serial_test.go => txn_context_test.go} | 0 util/chunk/row_container_serial_test.go | 111 ------ util/chunk/row_container_test.go | 87 ++++ util/ranger/ranger_serial_test.go | 374 ------------------ util/ranger/ranger_test.go | 345 ++++++++++++++++ 10 files changed, 432 insertions(+), 485 deletions(-) rename cmd/ddltest/{column_serial_test.go => column_test.go} (100%) rename cmd/ddltest/{ddl_serial_test.go => ddl_test.go} (100%) rename cmd/ddltest/{index_serial_test.go => index_test.go} (100%) rename planner/core/{plan_to_pb_serial_test.go => plan_to_pb_test.go} (100%) rename server/{statistics_handler_serial_test.go => statistics_handler_test.go} (100%) rename sessiontxn/{txn_context_serial_test.go => txn_context_test.go} (100%) delete mode 100644 util/chunk/row_container_serial_test.go delete mode 100644 util/ranger/ranger_serial_test.go diff --git a/cmd/ddltest/column_serial_test.go b/cmd/ddltest/column_test.go similarity index 100% rename from cmd/ddltest/column_serial_test.go rename to cmd/ddltest/column_test.go diff --git a/cmd/ddltest/ddl_serial_test.go b/cmd/ddltest/ddl_test.go similarity index 100% rename from cmd/ddltest/ddl_serial_test.go rename to cmd/ddltest/ddl_test.go diff --git a/cmd/ddltest/index_serial_test.go b/cmd/ddltest/index_test.go similarity index 100% rename from cmd/ddltest/index_serial_test.go rename to cmd/ddltest/index_test.go diff --git a/planner/core/plan_to_pb_serial_test.go b/planner/core/plan_to_pb_test.go similarity index 100% rename from planner/core/plan_to_pb_serial_test.go rename to planner/core/plan_to_pb_test.go diff --git a/server/statistics_handler_serial_test.go b/server/statistics_handler_test.go similarity index 100% rename from server/statistics_handler_serial_test.go rename to server/statistics_handler_test.go diff --git a/sessiontxn/txn_context_serial_test.go b/sessiontxn/txn_context_test.go similarity index 100% rename from sessiontxn/txn_context_serial_test.go rename to sessiontxn/txn_context_test.go diff --git a/util/chunk/row_container_serial_test.go b/util/chunk/row_container_serial_test.go deleted file mode 100644 index 0be690a444a35..0000000000000 --- a/util/chunk/row_container_serial_test.go +++ /dev/null @@ -1,111 +0,0 @@ -// Copyright 2021 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package chunk - -import ( - "testing" - "time" - - "github.com/pingcap/failpoint" - "github.com/pingcap/tidb/parser/mysql" - "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/memory" - "github.com/stretchr/testify/require" -) - -func TestSpillActionDeadLock(t *testing.T) { - // Maybe get deadlock if we use two RLock in one goroutine, for oom-action call stack. - // Now the implement avoids the situation. - // Goroutine 1: rc.Add() (RLock) -> list.Add() -> tracker.Consume() -> SpillDiskAction -> rc.AlreadySpilledSafeForTest() (RLock) - // Goroutine 2: ------------------> SpillDiskAction -> new Goroutine to spill -> ------------------ - // new Goroutine created by 2: ---> rc.SpillToDisk (Lock) - // In golang, RLock will be blocked after try to get Lock. So it will cause deadlock. - require.Nil(t, failpoint.Enable("github.com/pingcap/tidb/util/chunk/testRowContainerDeadLock", "return(true)")) - defer func() { - require.Nil(t, failpoint.Disable("github.com/pingcap/tidb/util/chunk/testRowContainerDeadLock")) - }() - sz := 4 - fields := []*types.FieldType{types.NewFieldType(mysql.TypeLonglong)} - rc := NewRowContainer(fields, sz) - - chk := NewChunkWithCapacity(fields, sz) - for i := 0; i < sz; i++ { - chk.AppendInt64(0, int64(i)) - } - var tracker *memory.Tracker - var err error - tracker = rc.GetMemTracker() - tracker.SetBytesLimit(1) - ac := rc.ActionSpillForTest() - tracker.FallbackOldAndSetNewAction(ac) - require.False(t, rc.AlreadySpilledSafeForTest()) - go func() { - time.Sleep(200 * time.Millisecond) - ac.Action(tracker) - }() - err = rc.Add(chk) - require.NoError(t, err) - rc.actionSpill.WaitForTest() - require.True(t, rc.AlreadySpilledSafeForTest()) -} - -func TestActionBlocked(t *testing.T) { - sz := 4 - fields := []*types.FieldType{types.NewFieldType(mysql.TypeLonglong)} - rc := NewRowContainer(fields, sz) - - chk := NewChunkWithCapacity(fields, sz) - for i := 0; i < sz; i++ { - chk.AppendInt64(0, int64(i)) - } - var tracker *memory.Tracker - var err error - // Case 1, test Broadcast in Action. - tracker = rc.GetMemTracker() - tracker.SetBytesLimit(1450) - ac := rc.ActionSpill() - tracker.FallbackOldAndSetNewAction(ac) - for i := 0; i < 10; i++ { - err = rc.Add(chk) - require.NoError(t, err) - } - - ac.cond.L.Lock() - for ac.cond.status == notSpilled || - ac.cond.status == spilling { - ac.cond.Wait() - } - ac.cond.L.Unlock() - ac.cond.L.Lock() - require.Equal(t, spilledYet, ac.cond.status) - ac.cond.L.Unlock() - require.Equal(t, int64(0), tracker.BytesConsumed()) - require.Greater(t, tracker.MaxConsumed(), int64(0)) - require.Greater(t, rc.GetDiskTracker().BytesConsumed(), int64(0)) - - // Case 2, test Action will block when spilling. - rc = NewRowContainer(fields, sz) - tracker = rc.GetMemTracker() - ac = rc.ActionSpill() - starttime := time.Now() - ac.setStatus(spilling) - go func() { - time.Sleep(200 * time.Millisecond) - ac.setStatus(spilledYet) - ac.cond.Broadcast() - }() - ac.Action(tracker) - require.GreaterOrEqual(t, time.Since(starttime), 200*time.Millisecond) -} diff --git a/util/chunk/row_container_test.go b/util/chunk/row_container_test.go index 553863ddf3141..2bf8ef26644bc 100644 --- a/util/chunk/row_container_test.go +++ b/util/chunk/row_container_test.go @@ -16,7 +16,9 @@ package chunk import ( "testing" + "time" + "github.com/pingcap/failpoint" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/memory" @@ -216,3 +218,88 @@ func TestRowContainerResetAndAction(t *testing.T) { rc.actionSpill.WaitForTest() require.Greater(t, rc.GetDiskTracker().BytesConsumed(), int64(0)) } + +func TestSpillActionDeadLock(t *testing.T) { + // Maybe get deadlock if we use two RLock in one goroutine, for oom-action call stack. + // Now the implement avoids the situation. + // Goroutine 1: rc.Add() (RLock) -> list.Add() -> tracker.Consume() -> SpillDiskAction -> rc.AlreadySpilledSafeForTest() (RLock) + // Goroutine 2: ------------------> SpillDiskAction -> new Goroutine to spill -> ------------------ + // new Goroutine created by 2: ---> rc.SpillToDisk (Lock) + // In golang, RLock will be blocked after try to get Lock. So it will cause deadlock. + require.Nil(t, failpoint.Enable("github.com/pingcap/tidb/util/chunk/testRowContainerDeadLock", "return(true)")) + defer func() { + require.Nil(t, failpoint.Disable("github.com/pingcap/tidb/util/chunk/testRowContainerDeadLock")) + }() + sz := 4 + fields := []*types.FieldType{types.NewFieldType(mysql.TypeLonglong)} + rc := NewRowContainer(fields, sz) + + chk := NewChunkWithCapacity(fields, sz) + for i := 0; i < sz; i++ { + chk.AppendInt64(0, int64(i)) + } + var tracker *memory.Tracker + var err error + tracker = rc.GetMemTracker() + tracker.SetBytesLimit(1) + ac := rc.ActionSpillForTest() + tracker.FallbackOldAndSetNewAction(ac) + require.False(t, rc.AlreadySpilledSafeForTest()) + go func() { + time.Sleep(200 * time.Millisecond) + ac.Action(tracker) + }() + err = rc.Add(chk) + require.NoError(t, err) + rc.actionSpill.WaitForTest() + require.True(t, rc.AlreadySpilledSafeForTest()) +} + +func TestActionBlocked(t *testing.T) { + sz := 4 + fields := []*types.FieldType{types.NewFieldType(mysql.TypeLonglong)} + rc := NewRowContainer(fields, sz) + + chk := NewChunkWithCapacity(fields, sz) + for i := 0; i < sz; i++ { + chk.AppendInt64(0, int64(i)) + } + var tracker *memory.Tracker + var err error + // Case 1, test Broadcast in Action. + tracker = rc.GetMemTracker() + tracker.SetBytesLimit(1450) + ac := rc.ActionSpill() + tracker.FallbackOldAndSetNewAction(ac) + for i := 0; i < 10; i++ { + err = rc.Add(chk) + require.NoError(t, err) + } + + ac.cond.L.Lock() + for ac.cond.status == notSpilled || + ac.cond.status == spilling { + ac.cond.Wait() + } + ac.cond.L.Unlock() + ac.cond.L.Lock() + require.Equal(t, spilledYet, ac.cond.status) + ac.cond.L.Unlock() + require.Equal(t, int64(0), tracker.BytesConsumed()) + require.Greater(t, tracker.MaxConsumed(), int64(0)) + require.Greater(t, rc.GetDiskTracker().BytesConsumed(), int64(0)) + + // Case 2, test Action will block when spilling. + rc = NewRowContainer(fields, sz) + tracker = rc.GetMemTracker() + ac = rc.ActionSpill() + starttime := time.Now() + ac.setStatus(spilling) + go func() { + time.Sleep(200 * time.Millisecond) + ac.setStatus(spilledYet) + ac.cond.Broadcast() + }() + ac.Action(tracker) + require.GreaterOrEqual(t, time.Since(starttime), 200*time.Millisecond) +} diff --git a/util/ranger/ranger_serial_test.go b/util/ranger/ranger_serial_test.go deleted file mode 100644 index 49cfc3638d060..0000000000000 --- a/util/ranger/ranger_serial_test.go +++ /dev/null @@ -1,374 +0,0 @@ -// Copyright 2021 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package ranger_test - -import ( - "context" - "fmt" - "testing" - - "github.com/pingcap/tidb/expression" - plannercore "github.com/pingcap/tidb/planner/core" - "github.com/pingcap/tidb/session" - "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/testkit" - "github.com/pingcap/tidb/util/collate" - "github.com/pingcap/tidb/util/ranger" - "github.com/stretchr/testify/require" -) - -func TestIndexRange(t *testing.T) { - dom, store, err := newDomainStoreWithBootstrap(t) - defer func() { - dom.Close() - require.NoError(t, store.Close()) - }() - require.NoError(t, err) - testKit := testkit.NewTestKit(t, store) - testKit.MustExec("use test") - testKit.MustExec("drop table if exists t") - testKit.MustExec(` -create table t( - a varchar(50), - b int, - c double, - d varchar(10), - e binary(10), - f varchar(10) collate utf8mb4_general_ci, - g enum('A','B','C') collate utf8mb4_general_ci, - index idx_ab(a(50), b), - index idx_cb(c, a), - index idx_d(d(2)), - index idx_e(e(2)), - index idx_f(f), - index idx_de(d(2), e), - index idx_g(g) -)`) - - tests := []struct { - indexPos int - exprStr string - accessConds string - filterConds string - resultStr string - }{ - { - indexPos: 0, - exprStr: `a LIKE 'abc%'`, - accessConds: `[like(test.t.a, abc%, 92)]`, - filterConds: "[]", - resultStr: "[[\"abc\",\"abd\")]", - }, - { - indexPos: 0, - exprStr: "a LIKE 'abc_'", - accessConds: "[like(test.t.a, abc_, 92)]", - filterConds: "[like(test.t.a, abc_, 92)]", - resultStr: "[(\"abc\",\"abd\")]", - }, - { - indexPos: 0, - exprStr: "a LIKE 'abc'", - accessConds: "[like(test.t.a, abc, 92)]", - filterConds: "[]", - resultStr: "[[\"abc\",\"abc\"]]", - }, - { - indexPos: 0, - exprStr: `a LIKE "ab\_c"`, - accessConds: "[like(test.t.a, ab\\_c, 92)]", - filterConds: "[]", - resultStr: "[[\"ab_c\",\"ab_c\"]]", - }, - { - indexPos: 0, - exprStr: `a LIKE '%'`, - accessConds: "[]", - filterConds: `[like(test.t.a, %, 92)]`, - resultStr: "[[NULL,+inf]]", - }, - { - indexPos: 0, - exprStr: `a LIKE '\%a'`, - accessConds: "[like(test.t.a, \\%a, 92)]", - filterConds: "[]", - resultStr: `[["%a","%a"]]`, - }, - { - indexPos: 0, - exprStr: `a LIKE "\\"`, - accessConds: "[like(test.t.a, \\, 92)]", - filterConds: "[]", - resultStr: "[[\"\\\",\"\\\"]]", - }, - { - indexPos: 0, - exprStr: `a LIKE "\\\\a%"`, - accessConds: `[like(test.t.a, \\a%, 92)]`, - filterConds: "[]", - resultStr: "[[\"\\a\",\"\\b\")]", - }, - { - indexPos: 0, - exprStr: `a > NULL`, - accessConds: "[gt(test.t.a, )]", - filterConds: "[]", - resultStr: `[]`, - }, - { - indexPos: 0, - exprStr: `a = 'a' and b in (1, 2, 3)`, - accessConds: "[eq(test.t.a, a) in(test.t.b, 1, 2, 3)]", - filterConds: "[]", - resultStr: "[[\"a\" 1,\"a\" 1] [\"a\" 2,\"a\" 2] [\"a\" 3,\"a\" 3]]", - }, - { - indexPos: 0, - exprStr: `a = 'a' and b not in (1, 2, 3)`, - accessConds: "[eq(test.t.a, a) not(in(test.t.b, 1, 2, 3))]", - filterConds: "[]", - resultStr: "[(\"a\" NULL,\"a\" 1) (\"a\" 3,\"a\" +inf]]", - }, - { - indexPos: 0, - exprStr: `a in ('a') and b in ('1', 2.0, NULL)`, - accessConds: "[eq(test.t.a, a) in(test.t.b, 1, 2, )]", - filterConds: "[]", - resultStr: `[["a" 1,"a" 1] ["a" 2,"a" 2]]`, - }, - { - indexPos: 1, - exprStr: `c in ('1.1', 1, 1.1) and a in ('1', 'a', NULL)`, - accessConds: "[in(test.t.c, 1.1, 1, 1.1) in(test.t.a, 1, a, )]", - filterConds: "[]", - resultStr: "[[1 \"1\",1 \"1\"] [1 \"a\",1 \"a\"] [1.1 \"1\",1.1 \"1\"] [1.1 \"a\",1.1 \"a\"]]", - }, - { - indexPos: 1, - exprStr: "c in (1, 1, 1, 1, 1, 1, 2, 1, 2, 3, 2, 3, 4, 4, 1, 2)", - accessConds: "[in(test.t.c, 1, 1, 1, 1, 1, 1, 2, 1, 2, 3, 2, 3, 4, 4, 1, 2)]", - filterConds: "[]", - resultStr: "[[1,1] [2,2] [3,3] [4,4]]", - }, - { - indexPos: 1, - exprStr: "c not in (1, 2, 3)", - accessConds: "[not(in(test.t.c, 1, 2, 3))]", - filterConds: "[]", - resultStr: "[(NULL,1) (1,2) (2,3) (3,+inf]]", - }, - { - indexPos: 1, - exprStr: "c in (1, 2) and c in (1, 3)", - accessConds: "[eq(test.t.c, 1)]", - filterConds: "[]", - resultStr: "[[1,1]]", - }, - { - indexPos: 1, - exprStr: "c = 1 and c = 2", - accessConds: "[]", - filterConds: "[]", - resultStr: "[]", - }, - { - indexPos: 0, - exprStr: "a in (NULL)", - accessConds: "[eq(test.t.a, )]", - filterConds: "[]", - resultStr: "[]", - }, - { - indexPos: 0, - exprStr: "a not in (NULL, '1', '2', '3')", - accessConds: "[not(in(test.t.a, , 1, 2, 3))]", - filterConds: "[]", - resultStr: "[]", - }, - { - indexPos: 0, - exprStr: "not (a not in (NULL, '1', '2', '3') and a > '2')", - accessConds: "[or(in(test.t.a, , 1, 2, 3), le(test.t.a, 2))]", - filterConds: "[]", - resultStr: "[[-inf,\"2\"] [\"3\",\"3\"]]", - }, - { - indexPos: 0, - exprStr: "not (a not in (NULL) and a > '2')", - accessConds: "[or(eq(test.t.a, ), le(test.t.a, 2))]", - filterConds: "[]", - resultStr: "[[-inf,\"2\"]]", - }, - { - indexPos: 0, - exprStr: "not (a not in (NULL) or a > '2')", - accessConds: "[and(eq(test.t.a, ), le(test.t.a, 2))]", - filterConds: "[]", - resultStr: "[]", - }, - { - indexPos: 0, - exprStr: "(a > 'b' and a < 'bbb') or (a < 'cb' and a > 'a')", - accessConds: "[or(and(gt(test.t.a, b), lt(test.t.a, bbb)), and(lt(test.t.a, cb), gt(test.t.a, a)))]", - filterConds: "[]", - resultStr: "[(\"a\",\"cb\")]", - }, - { - indexPos: 0, - exprStr: "(a > 'a' and a < 'b') or (a >= 'b' and a < 'c')", - accessConds: "[or(and(gt(test.t.a, a), lt(test.t.a, b)), and(ge(test.t.a, b), lt(test.t.a, c)))]", - filterConds: "[]", - resultStr: "[(\"a\",\"c\")]", - }, - { - indexPos: 0, - exprStr: "(a > 'a' and a < 'b' and b < 1) or (a >= 'b' and a < 'c')", - accessConds: "[or(and(gt(test.t.a, a), lt(test.t.a, b)), and(ge(test.t.a, b), lt(test.t.a, c)))]", - filterConds: "[or(and(and(gt(test.t.a, a), lt(test.t.a, b)), lt(test.t.b, 1)), and(ge(test.t.a, b), lt(test.t.a, c)))]", - resultStr: "[(\"a\",\"c\")]", - }, - { - indexPos: 0, - exprStr: "(a in ('a', 'b') and b < 1) or (a >= 'b' and a < 'c')", - accessConds: "[or(and(in(test.t.a, a, b), lt(test.t.b, 1)), and(ge(test.t.a, b), lt(test.t.a, c)))]", - filterConds: "[]", - resultStr: `[["a" -inf,"a" 1) ["b","c")]`, - }, - { - indexPos: 0, - exprStr: "(a > 'a') or (c > 1)", - accessConds: "[]", - filterConds: "[or(gt(test.t.a, a), gt(test.t.c, 1))]", - resultStr: "[[NULL,+inf]]", - }, - { - indexPos: 2, - exprStr: `d = "你好啊"`, - accessConds: "[eq(test.t.d, 你好啊)]", - filterConds: "[eq(test.t.d, 你好啊)]", - resultStr: "[[\"你好\",\"你好\"]]", - }, - { - indexPos: 3, - exprStr: `e = "你好啊"`, - accessConds: "[eq(test.t.e, 你好啊)]", - filterConds: "[eq(test.t.e, 你好啊)]", - resultStr: "[[0xE4BD,0xE4BD]]", - }, - { - indexPos: 2, - exprStr: `d in ("你好啊", "再见")`, - accessConds: "[in(test.t.d, 你好啊, 再见)]", - filterConds: "[in(test.t.d, 你好啊, 再见)]", - resultStr: "[[\"你好\",\"你好\"] [\"再见\",\"再见\"]]", - }, - { - indexPos: 2, - exprStr: `d not in ("你好啊")`, - accessConds: "[]", - filterConds: "[ne(test.t.d, 你好啊)]", - resultStr: "[[NULL,+inf]]", - }, - { - indexPos: 2, - exprStr: `d < "你好" || d > "你好"`, - accessConds: "[or(lt(test.t.d, 你好), gt(test.t.d, 你好))]", - filterConds: "[or(lt(test.t.d, 你好), gt(test.t.d, 你好))]", - resultStr: "[[-inf,+inf]]", - }, - { - indexPos: 2, - exprStr: `not(d < "你好" || d > "你好")`, - accessConds: "[and(ge(test.t.d, 你好), le(test.t.d, 你好))]", - filterConds: "[and(ge(test.t.d, 你好), le(test.t.d, 你好))]", - resultStr: "[[\"你好\",\"你好\"]]", - }, - { - indexPos: 4, - exprStr: "f >= 'a' and f <= 'B'", - accessConds: "[ge(test.t.f, a) le(test.t.f, B)]", - filterConds: "[]", - resultStr: "[[\"a\",\"B\"]]", - }, - { - indexPos: 4, - exprStr: "f in ('a', 'B')", - accessConds: "[in(test.t.f, a, B)]", - filterConds: "[]", - resultStr: "[[\"a\",\"a\"] [\"B\",\"B\"]]", - }, - { - indexPos: 4, - exprStr: "f = 'a' and f = 'B' collate utf8mb4_bin", - accessConds: "[eq(test.t.f, a)]", - filterConds: "[eq(test.t.f, B)]", - resultStr: "[[\"a\",\"a\"]]", - }, - { - indexPos: 4, - exprStr: "f like '@%' collate utf8mb4_bin", - accessConds: "[]", - filterConds: "[like(test.t.f, @%, 92)]", - resultStr: "[[NULL,+inf]]", - }, - { - indexPos: 5, - exprStr: "d in ('aab', 'aac') and e = 'a'", - accessConds: "[in(test.t.d, aab, aac) eq(test.t.e, a)]", - filterConds: "[in(test.t.d, aab, aac)]", - resultStr: "[[\"aa\" 0x61,\"aa\" 0x61]]", - }, - { - indexPos: 6, - exprStr: "g = 'a'", - accessConds: "[eq(test.t.g, a)]", - filterConds: "[]", - resultStr: "[[\"A\",\"A\"]]", - }, - } - - collate.SetNewCollationEnabledForTest(true) - defer func() { collate.SetNewCollationEnabledForTest(false) }() - ctx := context.Background() - for _, tt := range tests { - t.Run(tt.exprStr, func(t *testing.T) { - sql := "select * from t where " + tt.exprStr - sctx := testKit.Session().(sessionctx.Context) - stmts, err := session.Parse(sctx, sql) - require.NoError(t, err) - require.Len(t, stmts, 1) - ret := &plannercore.PreprocessorReturn{} - err = plannercore.Preprocess(sctx, stmts[0], plannercore.WithPreprocessorReturn(ret)) - require.NoError(t, err) - p, _, err := plannercore.BuildLogicalPlanForTest(ctx, sctx, stmts[0], ret.InfoSchema) - require.NoError(t, err) - selection := p.(plannercore.LogicalPlan).Children()[0].(*plannercore.LogicalSelection) - tbl := selection.Children()[0].(*plannercore.DataSource).TableInfo() - require.NotNil(t, selection) - conds := make([]expression.Expression, len(selection.Conditions)) - for i, cond := range selection.Conditions { - conds[i] = expression.PushDownNot(sctx, cond) - } - cols, lengths := expression.IndexInfo2PrefixCols(tbl.Columns, selection.Schema().Columns, tbl.Indices[tt.indexPos]) - require.NotNil(t, cols) - res, err := ranger.DetachCondAndBuildRangeForIndex(sctx, conds, cols, lengths) - require.NoError(t, err) - require.Equal(t, tt.accessConds, fmt.Sprintf("%s", res.AccessConds)) - require.Equal(t, tt.filterConds, fmt.Sprintf("%s", res.RemainedConds)) - got := fmt.Sprintf("%v", res.Ranges) - require.Equal(t, tt.resultStr, got) - }) - } -} diff --git a/util/ranger/ranger_test.go b/util/ranger/ranger_test.go index 03de4f42c2eea..7fc93cca7d8cc 100644 --- a/util/ranger/ranger_test.go +++ b/util/ranger/ranger_test.go @@ -31,6 +31,7 @@ import ( "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/testkit/testdata" "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/ranger" "github.com/stretchr/testify/require" ) @@ -1420,3 +1421,347 @@ func TestPrefixIndexAppendPointRanges(t *testing.T) { testKit.MustQuery(tt).Check(testkit.Rows(output[i].Result...)) } } + +func TestIndexRange(t *testing.T) { + dom, store, err := newDomainStoreWithBootstrap(t) + require.NoError(t, err) + defer func() { + dom.Close() + require.NoError(t, store.Close()) + }() + testKit := testkit.NewTestKit(t, store) + testKit.MustExec("use test") + testKit.MustExec("drop table if exists t") + testKit.MustExec(` +create table t( + a varchar(50), + b int, + c double, + d varchar(10), + e binary(10), + f varchar(10) collate utf8mb4_general_ci, + g enum('A','B','C') collate utf8mb4_general_ci, + index idx_ab(a(50), b), + index idx_cb(c, a), + index idx_d(d(2)), + index idx_e(e(2)), + index idx_f(f), + index idx_de(d(2), e), + index idx_g(g) +)`) + + tests := []struct { + indexPos int + exprStr string + accessConds string + filterConds string + resultStr string + }{ + { + indexPos: 0, + exprStr: `a LIKE 'abc%'`, + accessConds: `[like(test.t.a, abc%, 92)]`, + filterConds: "[]", + resultStr: "[[\"abc\",\"abd\")]", + }, + { + indexPos: 0, + exprStr: "a LIKE 'abc_'", + accessConds: "[like(test.t.a, abc_, 92)]", + filterConds: "[like(test.t.a, abc_, 92)]", + resultStr: "[(\"abc\",\"abd\")]", + }, + { + indexPos: 0, + exprStr: "a LIKE 'abc'", + accessConds: "[like(test.t.a, abc, 92)]", + filterConds: "[]", + resultStr: "[[\"abc\",\"abc\"]]", + }, + { + indexPos: 0, + exprStr: `a LIKE "ab\_c"`, + accessConds: "[like(test.t.a, ab\\_c, 92)]", + filterConds: "[]", + resultStr: "[[\"ab_c\",\"ab_c\"]]", + }, + { + indexPos: 0, + exprStr: `a LIKE '%'`, + accessConds: "[]", + filterConds: `[like(test.t.a, %, 92)]`, + resultStr: "[[NULL,+inf]]", + }, + { + indexPos: 0, + exprStr: `a LIKE '\%a'`, + accessConds: "[like(test.t.a, \\%a, 92)]", + filterConds: "[]", + resultStr: `[["%a","%a"]]`, + }, + { + indexPos: 0, + exprStr: `a LIKE "\\"`, + accessConds: "[like(test.t.a, \\, 92)]", + filterConds: "[]", + resultStr: "[[\"\\\",\"\\\"]]", + }, + { + indexPos: 0, + exprStr: `a LIKE "\\\\a%"`, + accessConds: `[like(test.t.a, \\a%, 92)]`, + filterConds: "[]", + resultStr: "[[\"\\a\",\"\\b\")]", + }, + { + indexPos: 0, + exprStr: `a > NULL`, + accessConds: "[gt(test.t.a, )]", + filterConds: "[]", + resultStr: `[]`, + }, + { + indexPos: 0, + exprStr: `a = 'a' and b in (1, 2, 3)`, + accessConds: "[eq(test.t.a, a) in(test.t.b, 1, 2, 3)]", + filterConds: "[]", + resultStr: "[[\"a\" 1,\"a\" 1] [\"a\" 2,\"a\" 2] [\"a\" 3,\"a\" 3]]", + }, + { + indexPos: 0, + exprStr: `a = 'a' and b not in (1, 2, 3)`, + accessConds: "[eq(test.t.a, a) not(in(test.t.b, 1, 2, 3))]", + filterConds: "[]", + resultStr: "[(\"a\" NULL,\"a\" 1) (\"a\" 3,\"a\" +inf]]", + }, + { + indexPos: 0, + exprStr: `a in ('a') and b in ('1', 2.0, NULL)`, + accessConds: "[eq(test.t.a, a) in(test.t.b, 1, 2, )]", + filterConds: "[]", + resultStr: `[["a" 1,"a" 1] ["a" 2,"a" 2]]`, + }, + { + indexPos: 1, + exprStr: `c in ('1.1', 1, 1.1) and a in ('1', 'a', NULL)`, + accessConds: "[in(test.t.c, 1.1, 1, 1.1) in(test.t.a, 1, a, )]", + filterConds: "[]", + resultStr: "[[1 \"1\",1 \"1\"] [1 \"a\",1 \"a\"] [1.1 \"1\",1.1 \"1\"] [1.1 \"a\",1.1 \"a\"]]", + }, + { + indexPos: 1, + exprStr: "c in (1, 1, 1, 1, 1, 1, 2, 1, 2, 3, 2, 3, 4, 4, 1, 2)", + accessConds: "[in(test.t.c, 1, 1, 1, 1, 1, 1, 2, 1, 2, 3, 2, 3, 4, 4, 1, 2)]", + filterConds: "[]", + resultStr: "[[1,1] [2,2] [3,3] [4,4]]", + }, + { + indexPos: 1, + exprStr: "c not in (1, 2, 3)", + accessConds: "[not(in(test.t.c, 1, 2, 3))]", + filterConds: "[]", + resultStr: "[(NULL,1) (1,2) (2,3) (3,+inf]]", + }, + { + indexPos: 1, + exprStr: "c in (1, 2) and c in (1, 3)", + accessConds: "[eq(test.t.c, 1)]", + filterConds: "[]", + resultStr: "[[1,1]]", + }, + { + indexPos: 1, + exprStr: "c = 1 and c = 2", + accessConds: "[]", + filterConds: "[]", + resultStr: "[]", + }, + { + indexPos: 0, + exprStr: "a in (NULL)", + accessConds: "[eq(test.t.a, )]", + filterConds: "[]", + resultStr: "[]", + }, + { + indexPos: 0, + exprStr: "a not in (NULL, '1', '2', '3')", + accessConds: "[not(in(test.t.a, , 1, 2, 3))]", + filterConds: "[]", + resultStr: "[]", + }, + { + indexPos: 0, + exprStr: "not (a not in (NULL, '1', '2', '3') and a > '2')", + accessConds: "[or(in(test.t.a, , 1, 2, 3), le(test.t.a, 2))]", + filterConds: "[]", + resultStr: "[[-inf,\"2\"] [\"3\",\"3\"]]", + }, + { + indexPos: 0, + exprStr: "not (a not in (NULL) and a > '2')", + accessConds: "[or(eq(test.t.a, ), le(test.t.a, 2))]", + filterConds: "[]", + resultStr: "[[-inf,\"2\"]]", + }, + { + indexPos: 0, + exprStr: "not (a not in (NULL) or a > '2')", + accessConds: "[and(eq(test.t.a, ), le(test.t.a, 2))]", + filterConds: "[]", + resultStr: "[]", + }, + { + indexPos: 0, + exprStr: "(a > 'b' and a < 'bbb') or (a < 'cb' and a > 'a')", + accessConds: "[or(and(gt(test.t.a, b), lt(test.t.a, bbb)), and(lt(test.t.a, cb), gt(test.t.a, a)))]", + filterConds: "[]", + resultStr: "[(\"a\",\"cb\")]", + }, + { + indexPos: 0, + exprStr: "(a > 'a' and a < 'b') or (a >= 'b' and a < 'c')", + accessConds: "[or(and(gt(test.t.a, a), lt(test.t.a, b)), and(ge(test.t.a, b), lt(test.t.a, c)))]", + filterConds: "[]", + resultStr: "[(\"a\",\"c\")]", + }, + { + indexPos: 0, + exprStr: "(a > 'a' and a < 'b' and b < 1) or (a >= 'b' and a < 'c')", + accessConds: "[or(and(gt(test.t.a, a), lt(test.t.a, b)), and(ge(test.t.a, b), lt(test.t.a, c)))]", + filterConds: "[or(and(and(gt(test.t.a, a), lt(test.t.a, b)), lt(test.t.b, 1)), and(ge(test.t.a, b), lt(test.t.a, c)))]", + resultStr: "[(\"a\",\"c\")]", + }, + { + indexPos: 0, + exprStr: "(a in ('a', 'b') and b < 1) or (a >= 'b' and a < 'c')", + accessConds: "[or(and(in(test.t.a, a, b), lt(test.t.b, 1)), and(ge(test.t.a, b), lt(test.t.a, c)))]", + filterConds: "[]", + resultStr: `[["a" -inf,"a" 1) ["b","c")]`, + }, + { + indexPos: 0, + exprStr: "(a > 'a') or (c > 1)", + accessConds: "[]", + filterConds: "[or(gt(test.t.a, a), gt(test.t.c, 1))]", + resultStr: "[[NULL,+inf]]", + }, + { + indexPos: 2, + exprStr: `d = "你好啊"`, + accessConds: "[eq(test.t.d, 你好啊)]", + filterConds: "[eq(test.t.d, 你好啊)]", + resultStr: "[[\"你好\",\"你好\"]]", + }, + { + indexPos: 3, + exprStr: `e = "你好啊"`, + accessConds: "[eq(test.t.e, 你好啊)]", + filterConds: "[eq(test.t.e, 你好啊)]", + resultStr: "[[0xE4BD,0xE4BD]]", + }, + { + indexPos: 2, + exprStr: `d in ("你好啊", "再见")`, + accessConds: "[in(test.t.d, 你好啊, 再见)]", + filterConds: "[in(test.t.d, 你好啊, 再见)]", + resultStr: "[[\"你好\",\"你好\"] [\"再见\",\"再见\"]]", + }, + { + indexPos: 2, + exprStr: `d not in ("你好啊")`, + accessConds: "[]", + filterConds: "[ne(test.t.d, 你好啊)]", + resultStr: "[[NULL,+inf]]", + }, + { + indexPos: 2, + exprStr: `d < "你好" || d > "你好"`, + accessConds: "[or(lt(test.t.d, 你好), gt(test.t.d, 你好))]", + filterConds: "[or(lt(test.t.d, 你好), gt(test.t.d, 你好))]", + resultStr: "[[-inf,+inf]]", + }, + { + indexPos: 2, + exprStr: `not(d < "你好" || d > "你好")`, + accessConds: "[and(ge(test.t.d, 你好), le(test.t.d, 你好))]", + filterConds: "[and(ge(test.t.d, 你好), le(test.t.d, 你好))]", + resultStr: "[[\"你好\",\"你好\"]]", + }, + { + indexPos: 4, + exprStr: "f >= 'a' and f <= 'B'", + accessConds: "[ge(test.t.f, a) le(test.t.f, B)]", + filterConds: "[]", + resultStr: "[[\"a\",\"B\"]]", + }, + { + indexPos: 4, + exprStr: "f in ('a', 'B')", + accessConds: "[in(test.t.f, a, B)]", + filterConds: "[]", + resultStr: "[[\"a\",\"a\"] [\"B\",\"B\"]]", + }, + { + indexPos: 4, + exprStr: "f = 'a' and f = 'B' collate utf8mb4_bin", + accessConds: "[eq(test.t.f, a)]", + filterConds: "[eq(test.t.f, B)]", + resultStr: "[[\"a\",\"a\"]]", + }, + { + indexPos: 4, + exprStr: "f like '@%' collate utf8mb4_bin", + accessConds: "[]", + filterConds: "[like(test.t.f, @%, 92)]", + resultStr: "[[NULL,+inf]]", + }, + { + indexPos: 5, + exprStr: "d in ('aab', 'aac') and e = 'a'", + accessConds: "[in(test.t.d, aab, aac) eq(test.t.e, a)]", + filterConds: "[in(test.t.d, aab, aac)]", + resultStr: "[[\"aa\" 0x61,\"aa\" 0x61]]", + }, + { + indexPos: 6, + exprStr: "g = 'a'", + accessConds: "[eq(test.t.g, a)]", + filterConds: "[]", + resultStr: "[[\"A\",\"A\"]]", + }, + } + + collate.SetNewCollationEnabledForTest(true) + defer func() { collate.SetNewCollationEnabledForTest(false) }() + ctx := context.Background() + for _, tt := range tests { + t.Run(tt.exprStr, func(t *testing.T) { + sql := "select * from t where " + tt.exprStr + sctx := testKit.Session().(sessionctx.Context) + stmts, err := session.Parse(sctx, sql) + require.NoError(t, err) + require.Len(t, stmts, 1) + ret := &plannercore.PreprocessorReturn{} + err = plannercore.Preprocess(sctx, stmts[0], plannercore.WithPreprocessorReturn(ret)) + require.NoError(t, err) + p, _, err := plannercore.BuildLogicalPlanForTest(ctx, sctx, stmts[0], ret.InfoSchema) + require.NoError(t, err) + selection := p.(plannercore.LogicalPlan).Children()[0].(*plannercore.LogicalSelection) + tbl := selection.Children()[0].(*plannercore.DataSource).TableInfo() + require.NotNil(t, selection) + conds := make([]expression.Expression, len(selection.Conditions)) + for i, cond := range selection.Conditions { + conds[i] = expression.PushDownNot(sctx, cond) + } + cols, lengths := expression.IndexInfo2PrefixCols(tbl.Columns, selection.Schema().Columns, tbl.Indices[tt.indexPos]) + require.NotNil(t, cols) + res, err := ranger.DetachCondAndBuildRangeForIndex(sctx, conds, cols, lengths) + require.NoError(t, err) + require.Equal(t, tt.accessConds, fmt.Sprintf("%s", res.AccessConds)) + require.Equal(t, tt.filterConds, fmt.Sprintf("%s", res.RemainedConds)) + got := fmt.Sprintf("%v", res.Ranges) + require.Equal(t, tt.resultStr, got) + }) + } +}