From 0e5c1f07196734b2390cd71e993e4cd312a92559 Mon Sep 17 00:00:00 2001 From: xiongjiwei Date: Sun, 2 May 2021 22:37:51 +0800 Subject: [PATCH 01/67] ddl, partition: make partition table can use binary type column (#23528) --- ddl/db_partition_test.go | 9 ++++++++- ddl/ddl_api.go | 2 +- 2 files changed, 9 insertions(+), 2 deletions(-) diff --git a/ddl/db_partition_test.go b/ddl/db_partition_test.go index 6481fd9d8fe11..59bd721536d59 100644 --- a/ddl/db_partition_test.go +++ b/ddl/db_partition_test.go @@ -583,6 +583,12 @@ create table log_message_1 ( tk.MustExec("drop table if exists t;") tk.MustExec(`create table t(a int) partition by range (a) (partition p0 values less than (18446744073709551615));`) + + tk.MustExec("drop table if exists t;") + tk.MustExec(`create table t(a binary) partition by range columns (a) (partition p0 values less than (X'0C'));`) + tk.MustExec(`alter table t add partition (partition p1 values less than (X'0D'), partition p2 values less than (X'0E'));`) + tk.MustExec(`insert into t values (X'0B'), (X'0C'), (X'0D')`) + tk.MustQuery(`select * from t where a < X'0D'`).Check(testkit.Rows("\x0B", "\x0C")) } func (s *testIntegrationSuite1) TestDisableTablePartition(c *C) { @@ -743,6 +749,7 @@ func (s *testIntegrationSuite1) TestCreateTableWithListPartition(c *C) { "create table t (a bigint) partition by list (a) (partition p0 values in (to_seconds('2020-09-28 17:03:38'),to_seconds('2020-09-28 17:03:39')));", "create table t (a datetime) partition by list (to_seconds(a)) (partition p0 values in (to_seconds('2020-09-28 17:03:38'),to_seconds('2020-09-28 17:03:39')));", "create table t (a int, b int generated always as (a+1) virtual) partition by list (b + 1) (partition p0 values in (1));", + "create table t(a binary) partition by list columns (a) (partition p0 values in (X'0C'));", s.generatePartitionTableByNum(ddl.PartitionCountLimit), } @@ -3334,7 +3341,7 @@ func (s *testIntegrationSuite7) TestAddPartitionForTableWithWrongType(c *C) { _, err = tk.Exec("alter table t_char add partition (partition p1 values less than (0x20))") c.Assert(err, NotNil) - c.Assert(ddl.ErrWrongTypeColumnValue.Equal(err), IsTrue) + c.Assert(ddl.ErrRangeNotIncreasing.Equal(err), IsTrue) _, err = tk.Exec("alter table t_char add partition (partition p1 values less than (10))") c.Assert(err, NotNil) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 67b51f7a3750e..31de57a6e572e 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -5479,7 +5479,7 @@ func checkColumnsTypeAndValuesMatch(ctx sessionctx.Context, meta *model.TableInf } case mysql.TypeString, mysql.TypeVarString: switch vkind { - case types.KindString, types.KindBytes, types.KindNull: + case types.KindString, types.KindBytes, types.KindNull, types.KindBinaryLiteral: default: return ErrWrongTypeColumnValue.GenWithStackByArgs() } From 2026977abc1a9f02c530e7a7c459c7ea03e823eb Mon Sep 17 00:00:00 2001 From: Ben Ye Date: Sun, 2 May 2021 11:01:51 -0400 Subject: [PATCH 02/67] planner: Remove redundant call to expression.ColumnSubstitute (#24097) --- planner/core/rule_predicate_push_down.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/planner/core/rule_predicate_push_down.go b/planner/core/rule_predicate_push_down.go index 2b8862d8ae3a5..936c4720cbc87 100644 --- a/planner/core/rule_predicate_push_down.go +++ b/planner/core/rule_predicate_push_down.go @@ -371,7 +371,7 @@ func (p *LogicalProjection) PredicatePushDown(predicates []expression.Expression for _, cond := range predicates { newFilter := expression.ColumnSubstitute(cond, p.Schema(), p.Exprs) if !expression.HasGetSetVarFunc(newFilter) { - canBePushed = append(canBePushed, expression.ColumnSubstitute(cond, p.Schema(), p.Exprs)) + canBePushed = append(canBePushed, newFilter) } else { canNotBePushed = append(canNotBePushed, cond) } From 18d519f0563759c4e38dd96503776eaac2aa0833 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Sun, 2 May 2021 23:11:51 +0800 Subject: [PATCH 03/67] stats, executor: use a correct sampling to collect stats (#23997) --- executor/analyze.go | 162 +++++++++++++++++- executor/builder.go | 58 +++++++ planner/core/common_plans.go | 1 + planner/core/planbuilder.go | 51 +++++- sessionctx/variable/sysvar.go | 2 +- sessionctx/variable/varsutil_test.go | 4 +- statistics/cmsketch.go | 68 +++++++- statistics/handle/handle.go | 9 +- statistics/handle/handle_test.go | 2 +- statistics/histogram.go | 8 +- statistics/integration_test.go | 34 ++++ statistics/row_sampler.go | 156 +++++++++++++++++ statistics/sample_test.go | 57 ++++++ statistics/testdata/integration_suite_in.json | 7 + .../testdata/integration_suite_out.json | 17 ++ 15 files changed, 616 insertions(+), 20 deletions(-) diff --git a/executor/analyze.go b/executor/analyze.go index 8a2e665587aa4..2d3187842845c 100644 --- a/executor/analyze.go +++ b/executor/analyze.go @@ -158,7 +158,12 @@ func (e *AnalyzeExec) Next(ctx context.Context, req *chunk.Chunk) error { globalStatsMap[globalStatsID] = globalStatsInfo{result.IsIndex, hg.ID, result.StatsVer} } } - err1 := statsHandle.SaveStatsToStorage(statisticsID, result.Count, result.IsIndex, hg, result.Cms[i], result.TopNs[i], result.Fms[i], result.StatsVer, 1) + var err1 error + if result.StatsVer == statistics.Version3 { + err1 = statsHandle.SaveStatsToStorage(statisticsID, result.Count, result.IsIndex, hg, nil, result.TopNs[i], result.Fms[i], result.StatsVer, 1) + } else { + err1 = statsHandle.SaveStatsToStorage(statisticsID, result.Count, result.IsIndex, hg, result.Cms[i], result.TopNs[i], result.Fms[i], result.StatsVer, 1) + } if err1 != nil { err = err1 logutil.Logger(ctx).Error("save stats to storage failed", zap.Error(err)) @@ -517,6 +522,33 @@ func analyzeColumnsPushdown(colExec *AnalyzeColumnsExec) []analyzeResult { } else { ranges = ranger.FullIntRange(false) } + if colExec.analyzeVer == statistics.Version3 { + count, hists, topns, fmSketches, err := colExec.buildSamplingStats(ranges) + if err != nil { + return []analyzeResult{{Err: err, job: colExec.job}} + } + cLen := len(colExec.analyzePB.ColReq.ColumnsInfo) + colResult := analyzeResult{ + TableID: colExec.tableID, + Hist: hists[:cLen], + TopNs: topns[:cLen], + Fms: fmSketches[:cLen], + job: colExec.job, + StatsVer: colExec.analyzeVer, + Count: count, + } + colGroupResult := analyzeResult{ + TableID: colExec.tableID, + Hist: hists[cLen:], + TopNs: topns[cLen:], + Fms: fmSketches[cLen:], + job: colExec.job, + StatsVer: colExec.analyzeVer, + Count: count, + IsIndex: 1, + } + return []analyzeResult{colResult, colGroupResult} + } collExtStats := colExec.ctx.GetSessionVars().EnableExtendedStats hists, cms, topNs, fms, extStats, err := colExec.buildStats(ranges, collExtStats) if err != nil { @@ -594,6 +626,7 @@ type AnalyzeColumnsExec struct { opts map[ast.AnalyzeOptionType]uint64 job *statistics.AnalyzeJob analyzeVer int + indexes []*model.IndexInfo } func (e *AnalyzeColumnsExec) open(ranges []*ranger.Range) error { @@ -639,6 +672,129 @@ func (e *AnalyzeColumnsExec) buildResp(ranges []*ranger.Range) (distsql.SelectRe return result, nil } +func (e *AnalyzeColumnsExec) buildSamplingStats(ranges []*ranger.Range) ( + count int64, + hists []*statistics.Histogram, + topns []*statistics.TopN, + fmSketches []*statistics.FMSketch, + err error, +) { + if err = e.open(ranges); err != nil { + return 0, nil, nil, nil, err + } + defer func() { + if err1 := e.resultHandler.Close(); err1 != nil { + err = err1 + } + }() + l := len(e.analyzePB.ColReq.ColumnsInfo) + len(e.analyzePB.ColReq.ColumnGroups) + rootRowCollector := &statistics.RowSampleCollector{ + NullCount: make([]int64, l), + FMSketches: make([]*statistics.FMSketch, 0, l), + TotalSizes: make([]int64, l), + Samples: make(statistics.WeightedRowSampleHeap, 0, e.analyzePB.ColReq.SampleSize), + MaxSampleSize: int(e.analyzePB.ColReq.SampleSize), + } + for i := 0; i < l; i++ { + rootRowCollector.FMSketches = append(rootRowCollector.FMSketches, statistics.NewFMSketch(maxSketchSize)) + } + sc := e.ctx.GetSessionVars().StmtCtx + for { + data, err1 := e.resultHandler.nextRaw(context.TODO()) + if err1 != nil { + return 0, nil, nil, nil, err1 + } + if data == nil { + break + } + colResp := &tipb.AnalyzeColumnsResp{} + err = colResp.Unmarshal(data) + if err != nil { + return 0, nil, nil, nil, err + } + subCollector := &statistics.RowSampleCollector{ + MaxSampleSize: int(e.analyzePB.ColReq.SampleSize), + } + subCollector.FromProto(colResp.RowCollector) + e.job.Update(subCollector.Count) + rootRowCollector.MergeCollector(subCollector) + } + for _, sample := range rootRowCollector.Samples { + for i := range sample.Columns { + sample.Columns[i], err = tablecodec.DecodeColumnValue(sample.Columns[i].GetBytes(), &e.colsInfo[i].FieldType, sc.TimeZone) + if err != nil { + return 0, nil, nil, nil, err + } + if sample.Columns[i].Kind() == types.KindBytes { + sample.Columns[i].SetBytes(sample.Columns[i].GetBytes()) + } + } + } + hists = make([]*statistics.Histogram, 0, len(e.colsInfo)) + topns = make([]*statistics.TopN, 0, len(e.colsInfo)) + fmSketches = make([]*statistics.FMSketch, 0, len(e.colsInfo)) + for i, col := range e.colsInfo { + sampleItems := make([]*statistics.SampleItem, 0, rootRowCollector.MaxSampleSize) + for _, row := range rootRowCollector.Samples { + if row.Columns[i].IsNull() { + continue + } + sampleItems = append(sampleItems, &statistics.SampleItem{ + Value: row.Columns[i], + }) + } + collector := &statistics.SampleCollector{ + Samples: sampleItems, + NullCount: rootRowCollector.NullCount[i], + Count: rootRowCollector.Count - rootRowCollector.NullCount[i], + FMSketch: rootRowCollector.FMSketches[i], + TotalSize: rootRowCollector.TotalSizes[i], + } + hg, topn, err := statistics.BuildHistAndTopNOnRowSample(e.ctx, int(e.opts[ast.AnalyzeOptNumBuckets]), int(e.opts[ast.AnalyzeOptNumTopN]), col.ID, collector, &col.FieldType, true) + if err != nil { + return 0, nil, nil, nil, err + } + hists = append(hists, hg) + topns = append(topns, topn) + fmSketches = append(fmSketches, rootRowCollector.FMSketches[i]) + } + colLen := len(e.colsInfo) + for i, idx := range e.indexes { + sampleItems := make([]*statistics.SampleItem, 0, rootRowCollector.MaxSampleSize) + for _, row := range rootRowCollector.Samples { + if len(idx.Columns) == 1 && row.Columns[idx.Columns[0].Offset].IsNull() { + continue + } + b := make([]byte, 0, 8) + for _, col := range idx.Columns { + b, err = codec.EncodeKey(e.ctx.GetSessionVars().StmtCtx, b, row.Columns[col.Offset]) + if err != nil { + return 0, nil, nil, nil, err + } + } + sampleItems = append(sampleItems, &statistics.SampleItem{ + Value: types.NewBytesDatum(b), + }) + } + collector := &statistics.SampleCollector{ + Samples: sampleItems, + NullCount: rootRowCollector.NullCount[colLen+i], + Count: rootRowCollector.Count - rootRowCollector.NullCount[colLen+i], + FMSketch: rootRowCollector.FMSketches[colLen+i], + TotalSize: rootRowCollector.TotalSizes[colLen+i], + } + hg, topn, err := statistics.BuildHistAndTopNOnRowSample(e.ctx, int(e.opts[ast.AnalyzeOptNumBuckets]), int(e.opts[ast.AnalyzeOptNumTopN]), idx.ID, collector, types.NewFieldType(mysql.TypeBlob), false) + if err != nil { + return 0, nil, nil, nil, err + } + hists = append(hists, hg) + topns = append(topns, topn) + fmSketches = append(fmSketches, rootRowCollector.FMSketches[colLen+i]) + } + count = rootRowCollector.Count + return +} + func (e *AnalyzeColumnsExec) buildStats(ranges []*ranger.Range, needExtStats bool) (hists []*statistics.Histogram, cms []*statistics.CMSketch, topNs []*statistics.TopN, fms []*statistics.FMSketch, extStats *statistics.ExtendedStatsColl, err error) { if err = e.open(ranges); err != nil { return nil, nil, nil, nil, nil, err @@ -1331,7 +1487,9 @@ func (e *AnalyzeFastExec) runTasks() ([]*statistics.Histogram, []*statistics.CMS // Build collector properties. collector := e.collectors[i] collector.Samples = collector.Samples[:e.sampCursor] - sort.Slice(collector.Samples, func(i, j int) bool { return collector.Samples[i].Handle.Compare(collector.Samples[j].Handle) < 0 }) + sort.Slice(collector.Samples, func(i, j int) bool { + return collector.Samples[i].Handle.Compare(collector.Samples[j].Handle) < 0 + }) collector.CalcTotalSize() // Adjust the row count in case the count of `tblStats` is not accurate and too small. rowCount = mathutil.MaxInt64(rowCount, int64(len(collector.Samples))) diff --git a/executor/builder.go b/executor/builder.go index 45e5447f252c1..da361f4192395 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -2040,7 +2040,65 @@ func (b *executorBuilder) buildAnalyzeIndexIncremental(task plannercore.AnalyzeI return analyzeTask } +func (b *executorBuilder) buildAnalyzeSamplingPushdown( + task plannercore.AnalyzeColumnsTask, + opts map[ast.AnalyzeOptionType]uint64, + autoAnalyze string, +) *analyzeTask { + availableIdx := make([]*model.IndexInfo, 0, len(task.Indexes)) + colGroups := make([]*tipb.AnalyzeColumnGroup, 0, len(task.Indexes)) + if len(task.Indexes) > 0 { + for _, idx := range task.Indexes { + availableIdx = append(availableIdx, idx) + colGroup := &tipb.AnalyzeColumnGroup{ + ColumnOffsets: make([]int64, 0, len(idx.Columns)), + } + for _, col := range idx.Columns { + colGroup.ColumnOffsets = append(colGroup.ColumnOffsets, int64(col.Offset)) + } + colGroups = append(colGroups, colGroup) + } + } + + _, offset := timeutil.Zone(b.ctx.GetSessionVars().Location()) + sc := b.ctx.GetSessionVars().StmtCtx + e := &AnalyzeColumnsExec{ + ctx: b.ctx, + tableID: task.TableID, + colsInfo: task.ColsInfo, + handleCols: task.HandleCols, + concurrency: b.ctx.GetSessionVars().DistSQLScanConcurrency(), + analyzePB: &tipb.AnalyzeReq{ + Tp: tipb.AnalyzeType_TypeFullSampling, + Flags: sc.PushDownFlags(), + TimeZoneOffset: offset, + }, + opts: opts, + analyzeVer: task.StatsVersion, + indexes: availableIdx, + } + e.analyzePB.ColReq = &tipb.AnalyzeColumnsReq{ + BucketSize: int64(opts[ast.AnalyzeOptNumBuckets]), + SampleSize: int64(opts[ast.AnalyzeOptNumSamples]), + SketchSize: maxSketchSize, + ColumnsInfo: util.ColumnsToProto(task.ColsInfo, task.TblInfo.PKIsHandle), + ColumnGroups: colGroups, + } + if task.TblInfo != nil { + e.analyzePB.ColReq.PrimaryColumnIds = tables.TryGetCommonPkColumnIds(task.TblInfo) + if task.TblInfo.IsCommonHandle { + e.analyzePB.ColReq.PrimaryPrefixColumnIds = tables.PrimaryPrefixColumnIDs(task.TblInfo) + } + } + b.err = plannercore.SetPBColumnsDefaultValue(b.ctx, e.analyzePB.ColReq.ColumnsInfo, task.ColsInfo) + job := &statistics.AnalyzeJob{DBName: task.DBName, TableName: task.TableName, PartitionName: task.PartitionName, JobInfo: autoAnalyze + "analyze table"} + return &analyzeTask{taskType: colTask, colExec: e, job: job} +} + func (b *executorBuilder) buildAnalyzeColumnsPushdown(task plannercore.AnalyzeColumnsTask, opts map[ast.AnalyzeOptionType]uint64, autoAnalyze string) *analyzeTask { + if task.StatsVersion == statistics.Version3 { + return b.buildAnalyzeSamplingPushdown(task, opts, autoAnalyze) + } cols := task.ColsInfo if hasPkHist(task.HandleCols) { colInfo := task.TblInfo.Columns[task.HandleCols.GetCol(0).Index] diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index eea0e9a3d1339..3818486955646 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -856,6 +856,7 @@ type AnalyzeColumnsTask struct { CommonHandleInfo *model.IndexInfo ColsInfo []*model.ColumnInfo TblInfo *model.TableInfo + Indexes []*model.IndexInfo analyzeInfo } diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 88f8e54e6be3f..6fc98bc522508 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -1633,6 +1633,43 @@ func GetPhysicalIDsAndPartitionNames(tblInfo *model.TableInfo, partitionNames [] return ids, names, nil } +func (b *PlanBuilder) buildAnalyzeFullSamplingTask( + as *ast.AnalyzeTableStmt, + taskSlice []AnalyzeColumnsTask, + physicalIDs []int64, + names []string, + tbl *ast.TableName, + version int, +) []AnalyzeColumnsTask { + idxInfos := make([]*model.IndexInfo, 0, len(tbl.TableInfo.Indices)) + for _, idx := range tbl.TableInfo.Indices { + if idx.State != model.StatePublic { + continue + } + idxInfos = append(idxInfos, idx) + } + for i, id := range physicalIDs { + if id == tbl.TableInfo.ID { + id = -1 + } + info := analyzeInfo{ + DBName: tbl.Schema.O, + TableName: tbl.Name.O, + PartitionName: names[i], + TableID: AnalyzeTableID{TableID: tbl.TableInfo.ID, PartitionID: id}, + Incremental: as.Incremental, + StatsVersion: version, + } + taskSlice = append(taskSlice, AnalyzeColumnsTask{ + ColsInfo: tbl.TableInfo.Columns, + analyzeInfo: info, + TblInfo: tbl.TableInfo, + Indexes: idxInfos, + }) + } + return taskSlice +} + func (b *PlanBuilder) buildAnalyzeTable(as *ast.AnalyzeTableStmt, opts map[ast.AnalyzeOptionType]uint64, version int) (Plan, error) { p := &Analyze{Opts: opts} for _, tbl := range as.TableNames { @@ -1659,6 +1696,10 @@ func (b *PlanBuilder) buildAnalyzeTable(as *ast.AnalyzeTableStmt, opts map[ast.A "If you want to switch to version 2 statistics, please first disable query feedback by setting feedback-probability to 0.0 in the config file.")) } } + if version == statistics.Version3 { + p.ColTasks = b.buildAnalyzeFullSamplingTask(as, p.ColTasks, physicalIDs, names, tbl, version) + continue + } for _, idx := range idxInfo { // For prefix common handle. We don't use analyze mixed to handle it with columns. Because the full value // is read by coprocessor, the prefix index would get wrong stats in this case. @@ -1730,6 +1771,10 @@ func (b *PlanBuilder) buildAnalyzeIndex(as *ast.AnalyzeTableStmt, opts map[ast.A } b.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("The analyze version from the session is not compatible with the existing statistics of the table. Use the existing version instead")) } + if version == statistics.Version3 { + b.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("The version 3 would collect all statistics not only the selected indexes")) + return b.buildAnalyzeTable(as, opts, version) + } for _, idxName := range as.IndexNames { if isPrimaryIndex(idxName) { handleCols := BuildHandleColsForAnalyze(b.ctx, tblInfo) @@ -1791,6 +1836,10 @@ func (b *PlanBuilder) buildAnalyzeAllIndex(as *ast.AnalyzeTableStmt, opts map[as } b.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("The analyze version from the session is not compatible with the existing statistics of the table. Use the existing version instead")) } + if version == statistics.Version3 { + b.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("The version 3 would collect all statistics not only the selected indexes")) + return b.buildAnalyzeTable(as, opts, version) + } for _, idx := range tblInfo.Indices { if idx.State == model.StatePublic { for i, id := range physicalIDs { @@ -1837,7 +1886,7 @@ var analyzeOptionLimit = map[ast.AnalyzeOptionType]uint64{ ast.AnalyzeOptNumTopN: 1024, ast.AnalyzeOptCMSketchWidth: CMSketchSizeLimit, ast.AnalyzeOptCMSketchDepth: CMSketchSizeLimit, - ast.AnalyzeOptNumSamples: 100000, + ast.AnalyzeOptNumSamples: 500000, } var analyzeOptionDefault = map[ast.AnalyzeOptionType]uint64{ diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 0b6b41cb37b86..751f0e70f0469 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -1317,7 +1317,7 @@ var defaultSysVars = []*SysVar{ s.GuaranteeLinearizability = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBAnalyzeVersion, Value: strconv.Itoa(DefTiDBAnalyzeVersion), Hidden: true, Type: TypeInt, MinValue: 1, MaxValue: 2, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + {Scope: ScopeGlobal | ScopeSession, Name: TiDBAnalyzeVersion, Value: strconv.Itoa(DefTiDBAnalyzeVersion), Hidden: true, Type: TypeInt, MinValue: 1, MaxValue: 3, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { if normalizedValue == "2" && FeedbackProbability.Load() > 0 { var original string var err error diff --git a/sessionctx/variable/varsutil_test.go b/sessionctx/variable/varsutil_test.go index 3cd546eba7637..f844ef1c57757 100644 --- a/sessionctx/variable/varsutil_test.go +++ b/sessionctx/variable/varsutil_test.go @@ -505,8 +505,8 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) { err = SetSessionSystemVar(v, "UnknownVariable", "on") c.Assert(err, ErrorMatches, ".*]Unknown system variable 'UnknownVariable'") - err = SetSessionSystemVar(v, TiDBAnalyzeVersion, "3") - c.Assert(err, ErrorMatches, ".*Variable 'tidb_analyze_version' can't be set to the value of '3'") + err = SetSessionSystemVar(v, TiDBAnalyzeVersion, "4") + c.Assert(err, ErrorMatches, ".*Variable 'tidb_analyze_version' can't be set to the value of '4'") } func (s *testVarsutilSuite) TestSetOverflowBehave(c *C) { diff --git a/statistics/cmsketch.go b/statistics/cmsketch.go index a512b21a484a5..f682a1507a4bf 100644 --- a/statistics/cmsketch.go +++ b/statistics/cmsketch.go @@ -15,9 +15,13 @@ package statistics import ( "bytes" + "fmt" "math" "reflect" "sort" + "strings" + + "github.com/pingcap/tidb/sessionctx" "github.com/cznic/mathutil" "github.com/cznic/sortutil" @@ -426,19 +430,27 @@ func EncodeCMSketchWithoutTopN(c *CMSketch) ([]byte, error) { // DecodeCMSketchAndTopN decode a CMSketch from the given byte slice. func DecodeCMSketchAndTopN(data []byte, topNRows []chunk.Row) (*CMSketch, *TopN, error) { - if data == nil { + if data == nil && len(topNRows) == 0 { return nil, nil, nil } + pbTopN := make([]*tipb.CMSketchTopN, 0, len(topNRows)) + for _, row := range topNRows { + data := make([]byte, len(row.GetBytes(0))) + copy(data, row.GetBytes(0)) + pbTopN = append(pbTopN, &tipb.CMSketchTopN{ + Data: data, + Count: row.GetUint64(1), + }) + } + if len(data) == 0 { + return nil, TopNFromProto(pbTopN), nil + } p := &tipb.CMSketch{} err := p.Unmarshal(data) if err != nil { return nil, nil, errors.Trace(err) } - for _, row := range topNRows { - data := make([]byte, len(row.GetBytes(0))) - copy(data, row.GetBytes(0)) - p.TopN = append(p.TopN, &tipb.CMSketchTopN{Data: data, Count: row.GetUint64(1)}) - } + p.TopN = pbTopN cm, topN := CMSketchAndTopNFromProto(p) return cm, topN, nil } @@ -487,6 +499,46 @@ type TopN struct { TopN []TopNMeta } +func (c *TopN) String() string { + if c == nil { + return "EmptyTopN" + } + builder := &strings.Builder{} + fmt.Fprintf(builder, "TopN{length: %v, ", len(c.TopN)) + fmt.Fprint(builder, "[") + for i := 0; i < len(c.TopN); i++ { + fmt.Fprintf(builder, "(%v, %v)", c.TopN[i].Encoded, c.TopN[i].Count) + if i+1 != len(c.TopN) { + fmt.Fprint(builder, ", ") + } + } + fmt.Fprint(builder, "]") + fmt.Fprint(builder, "}") + return builder.String() +} + +// DecodedString returns the value with decoded result. +func (c *TopN) DecodedString(ctx sessionctx.Context, colTypes []byte) (string, error) { + builder := &strings.Builder{} + fmt.Fprintf(builder, "TopN{length: %v, ", len(c.TopN)) + fmt.Fprint(builder, "[") + var tmpDatum types.Datum + for i := 0; i < len(c.TopN); i++ { + tmpDatum.SetBytes(c.TopN[i].Encoded) + valStr, err := ValueToString(ctx.GetSessionVars(), &tmpDatum, len(colTypes), colTypes) + if err != nil { + return "", err + } + fmt.Fprintf(builder, "(%v, %v)", valStr, c.TopN[i].Count) + if i+1 != len(c.TopN) { + fmt.Fprint(builder, ", ") + } + } + fmt.Fprint(builder, "]") + fmt.Fprint(builder, "}") + return builder.String(), nil +} + // Copy makes a copy for current TopN. func (c *TopN) Copy() *TopN { if c == nil { @@ -593,9 +645,9 @@ func (c *TopN) TotalCount() uint64 { // Equal checks whether the two TopN are equal. func (c *TopN) Equal(cc *TopN) bool { - if c == nil && cc == nil { + if c.TotalCount() == 0 && cc.TotalCount() == 0 { return true - } else if c == nil || cc == nil { + } else if c.TotalCount() != cc.TotalCount() { return false } if len(c.TopN) != len(cc.TopN) { diff --git a/statistics/handle/handle.go b/statistics/handle/handle.go index 8ea4f2fe15aa5..7036242b4e3a7 100644 --- a/statistics/handle/handle.go +++ b/statistics/handle/handle.go @@ -684,14 +684,17 @@ func (h *Handle) FlushStats() { } func (h *Handle) cmSketchAndTopNFromStorage(reader *statsReader, tblID int64, isIndex, histID int64) (_ *statistics.CMSketch, _ *statistics.TopN, err error) { - rows, _, err := reader.read("select cm_sketch from mysql.stats_histograms where table_id = %? and is_index = %? and hist_id = %?", tblID, isIndex, histID) - if err != nil || len(rows) == 0 { + topNRows, _, err := reader.read("select HIGH_PRIORITY value, count from mysql.stats_top_n where table_id = %? and is_index = %? and hist_id = %?", tblID, isIndex, histID) + if err != nil { return nil, nil, err } - topNRows, _, err := reader.read("select HIGH_PRIORITY value, count from mysql.stats_top_n where table_id = %? and is_index = %? and hist_id = %?", tblID, isIndex, histID) + rows, _, err := reader.read("select cm_sketch from mysql.stats_histograms where table_id = %? and is_index = %? and hist_id = %?", tblID, isIndex, histID) if err != nil { return nil, nil, err } + if len(rows) == 0 { + return statistics.DecodeCMSketchAndTopN(nil, topNRows) + } return statistics.DecodeCMSketchAndTopN(rows[0].GetBytes(0), topNRows) } diff --git a/statistics/handle/handle_test.go b/statistics/handle/handle_test.go index 16561c78ecc2b..3bb95e8219478 100644 --- a/statistics/handle/handle_test.go +++ b/statistics/handle/handle_test.go @@ -176,7 +176,7 @@ func assertTableEqual(c *C, a *statistics.Table, b *statistics.Table) { c.Assert(a.Columns[i].CMSketch.Equal(b.Columns[i].CMSketch), IsTrue) } // The nil case has been considered in (*TopN).Equal() so we don't need to consider it here. - c.Assert(a.Columns[i].TopN.Equal(b.Columns[i].TopN), IsTrue) + c.Assert(a.Columns[i].TopN.Equal(b.Columns[i].TopN), IsTrue, Commentf("%v, %v", a.Columns[i].TopN, b.Columns[i].TopN)) } c.Assert(len(a.Indices), Equals, len(b.Indices)) for i := range a.Indices { diff --git a/statistics/histogram.go b/statistics/histogram.go index cdf0ee9f965ff..37a78fe1b92de 100644 --- a/statistics/histogram.go +++ b/statistics/histogram.go @@ -285,7 +285,7 @@ func HistogramEqual(a, b *Histogram, ignoreID bool) bool { const ( // Version0 is the state that no statistics is actually collected, only the meta info.(the total count and the average col size) Version0 = 0 - // In Version1 + // Version1 maintains the statistics in the following way. // Column stats: CM Sketch is built in TiKV using full data. Histogram is built from samples. TopN is extracted from CM Sketch. // TopN + CM Sketch represent all data. Histogram also represents all data. // Index stats: CM Sketch and Histogram is built in TiKV using full data. TopN is extracted from histogram. Then values covered by TopN is removed from CM Sketch. @@ -293,11 +293,15 @@ const ( // Int PK column stats is always Version1 because it only has histogram built from full data. // Fast analyze is always Version1 currently. Version1 = 1 - // In Version2 + // Version2 maintains the statistics in the following way. // Column stats: CM Sketch is not used. TopN and Histogram are built from samples. TopN + Histogram represent all data. // Index stats: CM SKetch is not used. TopN and Histograms are built in TiKV using full data. NDV is also collected for each bucket in histogram. // Then values covered by TopN is removed from Histogram. TopN + Histogram represent all data. Version2 = 2 + // Version3 is used for testing now. Once it finished, we will fallback the Version3 to Version2. + // The difference between Version2 and Version3 is that we construct the index's statistics based on sampling also. + // The data structure between them are then same. + Version3 = 3 ) // AnalyzeFlag is set when the statistics comes from analyze and has not been modified by feedback. diff --git a/statistics/integration_test.go b/statistics/integration_test.go index d080a32c539e5..9edef42f17698 100644 --- a/statistics/integration_test.go +++ b/statistics/integration_test.go @@ -326,3 +326,37 @@ func (s *testIntegrationSuite) TestGlobalStats(c *C) { "TableReader 6.00 root partition:all data:TableFullScan", "└─TableFullScan 6.00 cop[tikv] table:t keep order:false")) } + +func (s *testIntegrationSuite) TestNULLOnFullSampling(c *C) { + defer cleanEnv(c, s.store, s.do) + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t;") + tk.MustExec("set @@session.tidb_analyze_version = 3;") + tk.MustExec("create table t(a int, index idx(a))") + tk.MustExec("insert into t values(1), (1), (1), (2), (2), (3), (4), (null), (null), (null)") + var ( + input []string + output [][]string + ) + tk.MustExec("analyze table t with 2 topn") + is := s.do.InfoSchema() + tblT, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + c.Assert(err, IsNil) + h := s.do.StatsHandle() + c.Assert(h.Update(is), IsNil) + statsTblT := h.GetTableStats(tblT.Meta()) + // Check the null count is 3. + for _, col := range statsTblT.Columns { + c.Assert(col.NullCount, Equals, int64(3)) + } + + s.testData.GetTestCases(c, &input, &output) + // Check the topn and buckets contains no null values. + for i := 0; i < len(input); i++ { + s.testData.OnRecord(func() { + output[i] = s.testData.ConvertRowsToStrings(tk.MustQuery(input[i]).Rows()) + }) + tk.MustQuery(input[i]).Check(testkit.Rows(output[i]...)) + } +} diff --git a/statistics/row_sampler.go b/statistics/row_sampler.go index e78895e19da13..f4f40945af697 100644 --- a/statistics/row_sampler.go +++ b/statistics/row_sampler.go @@ -14,14 +14,18 @@ package statistics import ( + "bytes" "container/heap" "context" "math/rand" + "github.com/pingcap/errors" + "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/sqlexec" "github.com/pingcap/tipb/go-tipb" @@ -285,9 +289,161 @@ func RowSamplesToProto(samples WeightedRowSampleHeap) []*tipb.RowSample { Weight: sample.Weight, } for _, c := range sample.Columns { + if c.IsNull() { + pbRow.Row = append(pbRow.Row, []byte{codec.NilFlag}) + continue + } pbRow.Row = append(pbRow.Row, c.GetBytes()) } rows = append(rows, pbRow) } return rows } + +// BuildHistAndTopNOnRowSample build a histogram and TopN for a column from samples. +func BuildHistAndTopNOnRowSample( + ctx sessionctx.Context, + numBuckets, numTopN int, + id int64, + collector *SampleCollector, + tp *types.FieldType, + isColumn bool, +) (*Histogram, *TopN, error) { + var getComparedBytes func(datum types.Datum) ([]byte, error) + if isColumn { + getComparedBytes = func(datum types.Datum) ([]byte, error) { + return codec.EncodeKey(ctx.GetSessionVars().StmtCtx, nil, datum) + } + } else { + getComparedBytes = func(datum types.Datum) ([]byte, error) { + return datum.GetBytes(), nil + } + } + count := collector.Count + ndv := collector.FMSketch.NDV() + nullCount := collector.NullCount + if ndv > count { + ndv = count + } + if count == 0 || len(collector.Samples) == 0 { + return NewHistogram(id, ndv, nullCount, 0, tp, 0, collector.TotalSize), nil, nil + } + sc := ctx.GetSessionVars().StmtCtx + samples := collector.Samples + samples, err := SortSampleItems(sc, samples) + if err != nil { + return nil, nil, err + } + hg := NewHistogram(id, ndv, nullCount, 0, tp, numBuckets, collector.TotalSize) + + sampleNum := int64(len(samples)) + // As we use samples to build the histogram, the bucket number and repeat should multiply a factor. + sampleFactor := float64(count) / float64(len(samples)) + + // Step1: collect topn from samples + + // the topNList is always sorted by count from more to less + topNList := make([]TopNMeta, 0, numTopN) + cur, err := getComparedBytes(samples[0].Value) + if err != nil { + return nil, nil, errors.Trace(err) + } + curCnt := float64(0) + + // Iterate through the samples + for i := int64(0); i < sampleNum; i++ { + + sampleBytes, err := getComparedBytes(samples[i].Value) + if err != nil { + return nil, nil, errors.Trace(err) + } + // case 1, this value is equal to the last one: current count++ + if bytes.Equal(cur, sampleBytes) { + curCnt += 1 + continue + } + // case 2, meet a different value: counting for the "current" is complete + // case 2-1, now topn is empty: append the "current" count directly + if len(topNList) == 0 { + topNList = append(topNList, TopNMeta{Encoded: cur, Count: uint64(curCnt)}) + cur, curCnt = sampleBytes, 1 + continue + } + // case 2-2, now topn is full, and the "current" count is less than the least count in the topn: no need to insert the "current" + if len(topNList) >= numTopN && uint64(curCnt) <= topNList[len(topNList)-1].Count { + cur, curCnt = sampleBytes, 1 + continue + } + // case 2-3, now topn is not full, or the "current" count is larger than the least count in the topn: need to find a slot to insert the "current" + j := len(topNList) + for ; j > 0; j-- { + if uint64(curCnt) < topNList[j-1].Count { + break + } + } + topNList = append(topNList, TopNMeta{}) + copy(topNList[j+1:], topNList[j:]) + topNList[j] = TopNMeta{Encoded: cur, Count: uint64(curCnt)} + if len(topNList) > numTopN { + topNList = topNList[:numTopN] + } + cur, curCnt = sampleBytes, 1 + } + + // Handle the counting for the last value. Basically equal to the case 2 above. + // now topn is empty: append the "current" count directly + if len(topNList) == 0 { + topNList = append(topNList, TopNMeta{Encoded: cur, Count: uint64(curCnt)}) + } else if len(topNList) < numTopN || uint64(curCnt) > topNList[len(topNList)-1].Count { + // now topn is not full, or the "current" count is larger than the least count in the topn: need to find a slot to insert the "current" + j := len(topNList) + for ; j > 0; j-- { + if uint64(curCnt) < topNList[j-1].Count { + break + } + } + topNList = append(topNList, TopNMeta{}) + copy(topNList[j+1:], topNList[j:]) + topNList[j] = TopNMeta{Encoded: cur, Count: uint64(curCnt)} + if len(topNList) > numTopN { + topNList = topNList[:numTopN] + } + } + + // Step2: exclude topn from samples + for i := int64(0); i < int64(len(samples)); i++ { + sampleBytes, err := getComparedBytes(samples[i].Value) + if err != nil { + return nil, nil, errors.Trace(err) + } + for j := 0; j < len(topNList); j++ { + if bytes.Equal(sampleBytes, topNList[j].Encoded) { + // find the same value in topn: need to skip over this value in samples + copy(samples[i:], samples[uint64(i)+topNList[j].Count:]) + samples = samples[:uint64(len(samples))-topNList[j].Count] + i-- + continue + } + } + } + + for i := 0; i < len(topNList); i++ { + topNList[i].Count *= uint64(sampleFactor) + } + topn := &TopN{TopN: topNList} + + if uint64(count) <= topn.TotalCount() || int(hg.NDV) <= len(topn.TopN) { + // TopN includes all sample data + return hg, topn, nil + } + + // Step3: build histogram with the rest samples + if len(samples) > 0 { + _, err = buildHist(sc, hg, samples, count-int64(topn.TotalCount()), ndv-int64(len(topn.TopN)), int64(numBuckets)) + if err != nil { + return nil, nil, err + } + } + + return hg, topn, nil +} diff --git a/statistics/sample_test.go b/statistics/sample_test.go index 0183a2a57dac2..61ec41b85870a 100644 --- a/statistics/sample_test.go +++ b/statistics/sample_test.go @@ -254,3 +254,60 @@ func (s *testSampleSuite) TestDistributedWeightedSampling(c *C) { } // } } + +func (s *testSampleSuite) TestBuildStatsOnRowSample(c *C) { + ctx := mock.NewContext() + sketch := NewFMSketch(1000) + data := make([]*SampleItem, 0, 8) + for i := 1; i <= 1000; i++ { + d := types.NewIntDatum(int64(i)) + err := sketch.InsertValue(ctx.GetSessionVars().StmtCtx, d) + c.Assert(err, IsNil) + data = append(data, &SampleItem{Value: d}) + } + for i := 1; i < 10; i++ { + d := types.NewIntDatum(int64(2)) + err := sketch.InsertValue(ctx.GetSessionVars().StmtCtx, d) + c.Assert(err, IsNil) + data = append(data, &SampleItem{Value: d}) + } + for i := 1; i < 7; i++ { + d := types.NewIntDatum(int64(4)) + err := sketch.InsertValue(ctx.GetSessionVars().StmtCtx, d) + c.Assert(err, IsNil) + data = append(data, &SampleItem{Value: d}) + } + for i := 1; i < 5; i++ { + d := types.NewIntDatum(int64(7)) + err := sketch.InsertValue(ctx.GetSessionVars().StmtCtx, d) + c.Assert(err, IsNil) + data = append(data, &SampleItem{Value: d}) + } + for i := 1; i < 3; i++ { + d := types.NewIntDatum(int64(11)) + err := sketch.InsertValue(ctx.GetSessionVars().StmtCtx, d) + c.Assert(err, IsNil) + data = append(data, &SampleItem{Value: d}) + } + collector := &SampleCollector{ + Samples: data, + NullCount: 0, + Count: int64(len(data)), + FMSketch: sketch, + TotalSize: int64(len(data)) * 8, + } + tp := types.NewFieldType(mysql.TypeLonglong) + hist, topN, err := BuildHistAndTopNOnRowSample(ctx, 5, 4, 1, collector, tp, true) + c.Assert(err, IsNil, Commentf("%+v", err)) + topNStr, err := topN.DecodedString(ctx, []byte{tp.Tp}) + c.Assert(err, IsNil) + c.Assert(topNStr, Equals, "TopN{length: 4, [(2, 10), (4, 7), (7, 5), (11, 3)]}") + c.Assert(hist.ToString(0), Equals, "column:1 ndv:1000 totColSize:8168\n"+ + "num: 200 lower_bound: 1 upper_bound: 204 repeats: 1 ndv: 0\n"+ + "num: 200 lower_bound: 205 upper_bound: 404 repeats: 1 ndv: 0\n"+ + "num: 200 lower_bound: 405 upper_bound: 604 repeats: 1 ndv: 0\n"+ + "num: 200 lower_bound: 605 upper_bound: 804 repeats: 1 ndv: 0\n"+ + "num: 196 lower_bound: 805 upper_bound: 1000 repeats: 1 ndv: 0", + ) + +} diff --git a/statistics/testdata/integration_suite_in.json b/statistics/testdata/integration_suite_in.json index 61f4badc3bb72..17afe048b3419 100644 --- a/statistics/testdata/integration_suite_in.json +++ b/statistics/testdata/integration_suite_in.json @@ -9,5 +9,12 @@ "explain select * from exp_backoff where a = 1 and b = 1 and c = 1 and d >= 3 and d<= 5", "explain select * from exp_backoff where a = 1 and b = 1 and c = 1 and d >= 3 and d<= 5" ] + }, + { + "name": "TestNULLOnFullSampling", + "cases": [ + "show stats_topn", + "show stats_buckets" + ] } ] diff --git a/statistics/testdata/integration_suite_out.json b/statistics/testdata/integration_suite_out.json index d5f6ff224a282..550d10caba8e1 100644 --- a/statistics/testdata/integration_suite_out.json +++ b/statistics/testdata/integration_suite_out.json @@ -30,5 +30,22 @@ "└─IndexRangeScan_5 0.00 cop[tikv] table:exp_backoff, index:idx(a, b, c, d) range:[1 1 1 3,1 1 1 5], keep order:false" ] ] + }, + { + "Name": "TestNULLOnFullSampling", + "Cases": [ + [ + "test t a 0 1 3", + "test t a 0 2 2", + "test t idx 1 1 3", + "test t idx 1 2 2" + ], + [ + "test t a 0 0 1 1 3 3 0", + "test t a 0 1 2 1 4 4 0", + "test t idx 1 0 1 1 3 3 0", + "test t idx 1 1 2 1 4 4 0" + ] + ] } ] From 207ce344cbb044ffb1b2681f1ba320a154979f6d Mon Sep 17 00:00:00 2001 From: ZhuoZhi <517770911@qq.com> Date: Sun, 2 May 2021 23:21:51 +0800 Subject: [PATCH 04/67] planner: remove useless predicates after partition pruning (#24282) --- planner/core/partition_prune.go | 2 +- planner/core/partition_pruner_test.go | 31 +++++++++ planner/core/rule_partition_processor.go | 43 ++++++++++-- .../core/testdata/partition_pruner_in.json | 12 ++++ .../core/testdata/partition_pruner_out.json | 68 +++++++++++++++++++ 5 files changed, 148 insertions(+), 8 deletions(-) diff --git a/planner/core/partition_prune.go b/planner/core/partition_prune.go index e8859a1890375..c10032c0a498a 100644 --- a/planner/core/partition_prune.go +++ b/planner/core/partition_prune.go @@ -37,7 +37,7 @@ func PartitionPruning(ctx sessionctx.Context, tbl table.PartitionedTable, conds case model.PartitionTypeHash: return s.pruneHashPartition(ctx, tbl, partitionNames, conds, columns, names) case model.PartitionTypeRange: - rangeOr, err := s.pruneRangePartition(ctx, pi, tbl, conds, columns, names) + rangeOr, _, err := s.pruneRangePartition(ctx, pi, tbl, conds, columns, names, nil) if err != nil { return nil, err } diff --git a/planner/core/partition_pruner_test.go b/planner/core/partition_pruner_test.go index 45440db71e6f4..df5eb0c47b739 100644 --- a/planner/core/partition_pruner_test.go +++ b/planner/core/partition_pruner_test.go @@ -454,3 +454,34 @@ partition by range (a) ( tk.MustQuery("select * from t3 where not (a = 1)").Sort().Check(testkit.Rows("11", "12", "13", "2", "3")) tk.MustQuery("select * from t3 where not (a != 1)").Check(testkit.Rows("1")) } + +//issue 22079 +func (s *testPartitionPruneSuit) TestRangePartitionPredicatePruner(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("set @@tidb_partition_prune_mode='" + string(variable.Static) + "'") + tk.MustExec("drop database if exists test_partition;") + tk.MustExec("create database test_partition") + tk.MustExec("use test_partition") + tk.MustExec("drop table if exists t") + tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly + tk.MustExec(`create table t (a int(11) default null) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin + partition by range(a) ( + partition p0 values less than (1), + partition p1 values less than (2), + partition p2 values less than (3), + partition p_max values less than (maxvalue));`) + + var input []string + var output []struct { + SQL string + Result []string + } + s.testData.GetTestCases(c, &input, &output) + for i, tt := range input { + s.testData.OnRecord(func() { + output[i].SQL = tt + output[i].Result = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + }) + tk.MustQuery(tt).Check(testkit.Rows(output[i].Result...)) + } +} diff --git a/planner/core/rule_partition_processor.go b/planner/core/rule_partition_processor.go index 6121d92e0fb66..46ae7282e346b 100644 --- a/planner/core/rule_partition_processor.go +++ b/planner/core/rule_partition_processor.go @@ -771,25 +771,26 @@ func intersectionRange(start, end, newStart, newEnd int) (int, int) { } func (s *partitionProcessor) pruneRangePartition(ctx sessionctx.Context, pi *model.PartitionInfo, tbl table.PartitionedTable, conds []expression.Expression, - columns []*expression.Column, names types.NameSlice) (partitionRangeOR, error) { + columns []*expression.Column, names types.NameSlice, condsToBePruned *[]expression.Expression) (partitionRangeOR, []expression.Expression, error) { partExpr, err := tbl.(partitionTable).PartitionExpr() if err != nil { - return nil, err + return nil, nil, err } // Partition by range columns. if len(pi.Columns) > 0 { - return s.pruneRangeColumnsPartition(ctx, conds, pi, partExpr, columns, names) + result, err := s.pruneRangeColumnsPartition(ctx, conds, pi, partExpr, columns, names) + return result, nil, err } // Partition by range. col, fn, mono, err := makePartitionByFnCol(ctx, columns, names, pi.Expr) if err != nil { - return nil, err + return nil, nil, err } result := fullRange(len(pi.Definitions)) if col == nil { - return result, nil + return result, nil, nil } // Extract the partition column, if the column is not null, it's possible to prune. @@ -803,14 +804,42 @@ func (s *partitionProcessor) pruneRangePartition(ctx sessionctx.Context, pi *mod monotonous: mono, } result = partitionRangeForCNFExpr(ctx, conds, &pruner, result) - return result, nil + + if condsToBePruned == nil { + return result, nil, nil + } + // remove useless predicates after partition pruning + newConds := make([]expression.Expression, 0, len(*condsToBePruned)) + for _, cond := range *condsToBePruned { + if dataForPrune, ok := pruner.extractDataForPrune(ctx, cond); ok { + switch dataForPrune.op { + case ast.EQ: + unsigned := mysql.HasUnsignedFlag(pruner.col.RetType.Flag) + start, _ := pruneUseBinarySearch(pruner.lessThan, dataForPrune, unsigned) + // if the type of partition key is Int + if pk, ok := partExpr.Expr.(*expression.Column); ok && pk.RetType.EvalType() == types.ETInt { + // see if can be removed + // see issue #22079: https://github.com/pingcap/tidb/issues/22079 for details + if start > 0 && pruner.lessThan.data[start-1] == dataForPrune.c && (pruner.lessThan.data[start]-1) == dataForPrune.c { + continue + } + } + } + } + newConds = append(newConds, cond) + } + + return result, newConds, nil } func (s *partitionProcessor) processRangePartition(ds *DataSource, pi *model.PartitionInfo) (LogicalPlan, error) { - used, err := s.pruneRangePartition(ds.ctx, pi, ds.table.(table.PartitionedTable), ds.allConds, ds.TblCols, ds.names) + used, prunedConds, err := s.pruneRangePartition(ds.ctx, pi, ds.table.(table.PartitionedTable), ds.allConds, ds.TblCols, ds.names, &ds.pushedDownConds) if err != nil { return nil, err } + if prunedConds != nil { + ds.pushedDownConds = prunedConds + } return s.makeUnionAllChildren(ds, pi, used) } diff --git a/planner/core/testdata/partition_pruner_in.json b/planner/core/testdata/partition_pruner_in.json index 1f6c197ba61a9..376d60d83b5a1 100644 --- a/planner/core/testdata/partition_pruner_in.json +++ b/planner/core/testdata/partition_pruner_in.json @@ -501,5 +501,17 @@ "Pruner": "t1: p0,p1; t2: p0" } ] + }, + { + "name": "TestRangePartitionPredicatePruner", + "cases": [ + "explain format = 'brief' select * from t where a = 1", + "explain format = 'brief' select * from t where a = 2", + "explain format = 'brief' select * from t where a = 10", + "explain format = 'brief' select * from t where a = -1", + "explain format = 'brief' select * from t where a is NULL", + "explain format = 'brief' select * from t where a = 1.5", + "explain format = 'brief' select * from t where a in (2)" + ] } ] diff --git a/planner/core/testdata/partition_pruner_out.json b/planner/core/testdata/partition_pruner_out.json index 0d71b5a0d2427..31ca209396c87 100644 --- a/planner/core/testdata/partition_pruner_out.json +++ b/planner/core/testdata/partition_pruner_out.json @@ -2827,5 +2827,73 @@ ] } ] + }, + { + "Name": "TestRangePartitionPredicatePruner", + "Cases": [ + { + "SQL": "explain format = 'brief' select * from t where a = 1", + "Result": [ + "TableReader 10000.00 root data:TableFullScan", + "└─TableFullScan 10000.00 cop[tikv] table:t, partition:p1 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a = 2", + "Result": [ + "TableReader 10000.00 root data:TableFullScan", + "└─TableFullScan 10000.00 cop[tikv] table:t, partition:p2 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a = 10", + "Result": [ + "TableReader 10.00 root data:Selection", + "└─Selection 10.00 cop[tikv] eq(test_partition.t.a, 10)", + " └─TableFullScan 10000.00 cop[tikv] table:t, partition:p_max keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a = -1", + "Result": [ + "TableReader 10.00 root data:Selection", + "└─Selection 10.00 cop[tikv] eq(test_partition.t.a, -1)", + " └─TableFullScan 10000.00 cop[tikv] table:t, partition:p0 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a is NULL", + "Result": [ + "TableReader 10.00 root data:Selection", + "└─Selection 10.00 cop[tikv] isnull(test_partition.t.a)", + " └─TableFullScan 10000.00 cop[tikv] table:t, partition:p0 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a = 1.5", + "Result": [ + "PartitionUnion 32000.00 root ", + "├─TableReader 8000.00 root data:Selection", + "│ └─Selection 8000.00 cop[tikv] eq(cast(test_partition.t.a, decimal(20,0) BINARY), 1.5)", + "│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:p0 keep order:false, stats:pseudo", + "├─TableReader 8000.00 root data:Selection", + "│ └─Selection 8000.00 cop[tikv] eq(cast(test_partition.t.a, decimal(20,0) BINARY), 1.5)", + "│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:p1 keep order:false, stats:pseudo", + "├─TableReader 8000.00 root data:Selection", + "│ └─Selection 8000.00 cop[tikv] eq(cast(test_partition.t.a, decimal(20,0) BINARY), 1.5)", + "│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:p2 keep order:false, stats:pseudo", + "└─TableReader 8000.00 root data:Selection", + " └─Selection 8000.00 cop[tikv] eq(cast(test_partition.t.a, decimal(20,0) BINARY), 1.5)", + " └─TableFullScan 10000.00 cop[tikv] table:t, partition:p_max keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a in (2)", + "Result": [ + "TableReader 10000.00 root data:TableFullScan", + "└─TableFullScan 10000.00 cop[tikv] table:t, partition:p2 keep order:false, stats:pseudo" + ] + } + ] } ] From 337beb1f89163ccc4242d7bf86bb94fede2aeebe Mon Sep 17 00:00:00 2001 From: Howie Date: Thu, 6 May 2021 11:05:53 +0800 Subject: [PATCH 05/67] *: the value of tikv-client.copr-cache.capacity-mb should >=0 (#24232) --- store/copr/coprocessor_cache.go | 2 +- store/copr/coprocessor_cache_test.go | 5 +++++ store/mockstore/mockcopr/executor_test.go | 3 ++- store/mockstore/mockstorage/storage.go | 7 ++++--- store/mockstore/tikv.go | 2 +- store/mockstore/unistore.go | 2 +- 6 files changed, 14 insertions(+), 7 deletions(-) diff --git a/store/copr/coprocessor_cache.go b/store/copr/coprocessor_cache.go index 377690fc8d543..d4ddf1806d35b 100644 --- a/store/copr/coprocessor_cache.go +++ b/store/copr/coprocessor_cache.go @@ -61,7 +61,7 @@ func newCoprCache(config *config.CoprocessorCache) (*coprCache, error) { return nil, nil } capacityInBytes := int64(config.CapacityMB * 1024.0 * 1024.0) - if capacityInBytes == 0 { + if capacityInBytes <= 0 { return nil, errors.New("Capacity must be > 0 to enable the cache") } maxEntityInBytes := int64(config.AdmissionMaxResultMB * 1024.0 * 1024.0) diff --git a/store/copr/coprocessor_cache_test.go b/store/copr/coprocessor_cache_test.go index 14084f978d20a..f257e161e2920 100644 --- a/store/copr/coprocessor_cache_test.go +++ b/store/copr/coprocessor_cache_test.go @@ -192,3 +192,8 @@ func (s *testCoprocessorSuite) TestGetSet(c *C) { c.Assert(v, NotNil) c.Assert(v.Data, DeepEquals, []byte("bar")) } + +func (s *testCoprocessorSuite) TestIssue24118(c *C) { + _, err := newCoprCache(&config.CoprocessorCache{AdmissionMinProcessMs: 5, AdmissionMaxResultMB: 1, CapacityMB: -1}) + c.Assert(err.Error(), Equals, "Capacity must be > 0 to enable the cache") +} diff --git a/store/mockstore/mockcopr/executor_test.go b/store/mockstore/mockcopr/executor_test.go index 39dd945d23f89..af9ac45beae96 100644 --- a/store/mockstore/mockcopr/executor_test.go +++ b/store/mockstore/mockcopr/executor_test.go @@ -55,7 +55,8 @@ func (s *testExecutorSuite) SetUpSuite(c *C) { s.mvccStore = rpcClient.MvccStore store, err := tikv.NewTestTiKVStore(rpcClient, pdClient, nil, nil, 0) c.Assert(err, IsNil) - s.store = mockstorage.NewMockStorage(store) + s.store, err = mockstorage.NewMockStorage(store) + c.Assert(err, IsNil) session.SetSchemaLease(0) session.DisableStats4Test() s.dom, err = session.BootstrapSession(s.store) diff --git a/store/mockstore/mockstorage/storage.go b/store/mockstore/mockstorage/storage.go index 63875acaad222..05ece29c57a6a 100644 --- a/store/mockstore/mockstorage/storage.go +++ b/store/mockstore/mockstorage/storage.go @@ -33,17 +33,18 @@ type mockStorage struct { } // NewMockStorage wraps tikv.KVStore as kv.Storage. -func NewMockStorage(tikvStore *tikv.KVStore) kv.Storage { +func NewMockStorage(tikvStore *tikv.KVStore) (kv.Storage, error) { coprConfig := config.DefaultConfig().TiKVClient.CoprCache coprStore, err := copr.NewStore(tikvStore, &coprConfig) if err != nil { - panic(err) + return nil, err } return &mockStorage{ KVStore: tikvStore, Store: coprStore, memCache: kv.NewCacheDB(), - } + }, nil + } func (s *mockStorage) EtcdAddrs() ([]string, error) { diff --git a/store/mockstore/tikv.go b/store/mockstore/tikv.go index 10378a51e34b2..ff179c4544c59 100644 --- a/store/mockstore/tikv.go +++ b/store/mockstore/tikv.go @@ -35,5 +35,5 @@ func newMockTikvStore(opt *mockOptions) (kv.Storage, error) { if err != nil { return nil, err } - return mockstorage.NewMockStorage(kvstore), nil + return mockstorage.NewMockStorage(kvstore) } diff --git a/store/mockstore/unistore.go b/store/mockstore/unistore.go index c712a979f821c..5476a6f6e7499 100644 --- a/store/mockstore/unistore.go +++ b/store/mockstore/unistore.go @@ -36,5 +36,5 @@ func newUnistore(opts *mockOptions) (kv.Storage, error) { if err != nil { return nil, err } - return mockstorage.NewMockStorage(kvstore), nil + return mockstorage.NewMockStorage(kvstore) } From 0d4e66ee8d6c1008f247a693f371fe2eb4b23fa8 Mon Sep 17 00:00:00 2001 From: Shirly Date: Thu, 6 May 2021 12:09:52 +0800 Subject: [PATCH 06/67] store/tikv: make tikv.ErrTiKV* as normal errors instead of dberror (#24396) --- store/driver/txn/error.go | 16 ++++++++++++++-- store/tikv/error/errcode.go | 6 ++---- store/tikv/error/error.go | 17 ++++++++++------- 3 files changed, 26 insertions(+), 13 deletions(-) diff --git a/store/driver/txn/error.go b/store/driver/txn/error.go index bd1ecc2c9aa2b..10e08aaf35d08 100644 --- a/store/driver/txn/error.go +++ b/store/driver/txn/error.go @@ -39,8 +39,12 @@ import ( // tikv error instance var ( // ErrTiKVServerTimeout is the error when tikv server is timeout. - ErrTiKVServerTimeout = dbterror.ClassTiKV.NewStd(errno.ErrTiKVServerTimeout) - ErrResolveLockTimeout = dbterror.ClassTiKV.NewStd(errno.ErrResolveLockTimeout) + ErrTiKVServerTimeout = dbterror.ClassTiKV.NewStd(errno.ErrTiKVServerTimeout) + // ErrTiKVStaleCommand is the error that the command is stale in tikv. + ErrTiKVStaleCommand = dbterror.ClassTiKV.NewStd(errno.ErrTiKVStaleCommand) + // ErrTiKVMaxTimestampNotSynced is the error that tikv's max timestamp is not synced. + ErrTiKVMaxTimestampNotSynced = dbterror.ClassTiKV.NewStd(errno.ErrTiKVMaxTimestampNotSynced) + ErrResolveLockTimeout = dbterror.ClassTiKV.NewStd(errno.ErrResolveLockTimeout) ) func genKeyExistsError(name string, value string, err error) error { @@ -189,6 +193,14 @@ func toTiDBErr(err error) error { return ErrTiKVServerTimeout } + if errors.ErrorEqual(err, tikverr.ErrTiKVStaleCommand) { + return ErrTiKVStaleCommand + } + + if errors.ErrorEqual(err, tikverr.ErrTiKVMaxTimestampNotSynced) { + return ErrTiKVMaxTimestampNotSynced + } + if errors.ErrorEqual(err, tikverr.ErrResolveLockTimeout) { return ErrResolveLockTimeout } diff --git a/store/tikv/error/errcode.go b/store/tikv/error/errcode.go index 192d2163b3b62..1c6c77eb2c210 100644 --- a/store/tikv/error/errcode.go +++ b/store/tikv/error/errcode.go @@ -32,8 +32,6 @@ const ( CodeTiKVStoreLimit = 9008 - CodeTiKVStaleCommand = 9010 - CodeTiKVMaxTimestampNotSynced = 9011 - CodeTiFlashServerTimeout = 9012 - CodeTiFlashServerBusy = 9013 + CodeTiFlashServerTimeout = 9012 + CodeTiFlashServerBusy = 9013 ) diff --git a/store/tikv/error/error.go b/store/tikv/error/error.go index f623c9f0f7491..57efa7c82bea7 100644 --- a/store/tikv/error/error.go +++ b/store/tikv/error/error.go @@ -35,6 +35,10 @@ var ( ErrInvalidTxn = errors.New("invalid transaction") // ErrTiKVServerTimeout is the error when tikv server is timeout. ErrTiKVServerTimeout = errors.New("tikv server timeout") + // ErrTiKVStaleCommand is the error that the command is stale in tikv. + ErrTiKVStaleCommand = errors.New("tikv stale command") + // ErrTiKVMaxTimestampNotSynced is the error that tikv's max timestamp is not synced. + ErrTiKVMaxTimestampNotSynced = errors.New("tikv max timestamp not synced") // ErrResolveLockTimeout is the error that resolve lock timeout. ErrResolveLockTimeout = errors.New("resolve lock timeout") ) @@ -44,13 +48,12 @@ const MismatchClusterID = "mismatch cluster id" // error instances. var ( - ErrTiFlashServerTimeout = dbterror.ClassTiKV.NewStd(CodeTiFlashServerTimeout) - ErrPDServerTimeout = dbterror.ClassTiKV.NewStd(CodePDServerTimeout) - ErrRegionUnavailable = dbterror.ClassTiKV.NewStd(CodeRegionUnavailable) - ErrTiKVServerBusy = dbterror.ClassTiKV.NewStd(CodeTiKVServerBusy) - ErrTiFlashServerBusy = dbterror.ClassTiKV.NewStd(CodeTiFlashServerBusy) - ErrTiKVStaleCommand = dbterror.ClassTiKV.NewStd(CodeTiKVStaleCommand) - ErrTiKVMaxTimestampNotSynced = dbterror.ClassTiKV.NewStd(CodeTiKVMaxTimestampNotSynced) + ErrTiFlashServerTimeout = dbterror.ClassTiKV.NewStd(CodeTiFlashServerTimeout) + ErrPDServerTimeout = dbterror.ClassTiKV.NewStd(CodePDServerTimeout) + ErrRegionUnavailable = dbterror.ClassTiKV.NewStd(CodeRegionUnavailable) + ErrTiKVServerBusy = dbterror.ClassTiKV.NewStd(CodeTiKVServerBusy) + ErrTiFlashServerBusy = dbterror.ClassTiKV.NewStd(CodeTiFlashServerBusy) + ErrGCTooEarly = dbterror.ClassTiKV.NewStd(CodeGCTooEarly) ErrQueryInterrupted = dbterror.ClassTiKV.NewStd(CodeQueryInterrupted) ErrLockAcquireFailAndNoWaitSet = dbterror.ClassTiKV.NewStd(CodeLockAcquireFailAndNoWaitSet) From c269b117054ad73feb642033e2e2d1f713c70bc9 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Thu, 6 May 2021 13:15:52 +0800 Subject: [PATCH 07/67] store/copr: invalidate stale regions for Mpp query. (#24410) --- go.mod | 2 +- go.sum | 4 ++-- store/copr/mpp.go | 7 ++++++ store/mockstore/unistore/tikv/server.go | 30 ------------------------- store/tikv/region_cache.go | 5 +++++ store/tikv/region_request_test.go | 24 -------------------- 6 files changed, 15 insertions(+), 57 deletions(-) diff --git a/go.mod b/go.mod index 4ad29e29e6273..bf927f9cc55ce 100644 --- a/go.mod +++ b/go.mod @@ -45,7 +45,7 @@ require ( github.com/pingcap/failpoint v0.0.0-20210316064728-7acb0f0a3dfd github.com/pingcap/fn v0.0.0-20200306044125-d5540d389059 github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 - github.com/pingcap/kvproto v0.0.0-20210402093459-65aa336ccbbf + github.com/pingcap/kvproto v0.0.0-20210429093846-65f54a202d7e github.com/pingcap/log v0.0.0-20210317133921-96f4fcab92a4 github.com/pingcap/parser v0.0.0-20210427084954-8e8ed7927bde github.com/pingcap/sysutil v0.0.0-20210315073920-cc0985d983a3 diff --git a/go.sum b/go.sum index d4863b5d8b369..74b4f623789b8 100644 --- a/go.sum +++ b/go.sum @@ -436,8 +436,8 @@ github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989/go.mod h1:O17Xtb github.com/pingcap/kvproto v0.0.0-20191211054548-3c6b38ea5107/go.mod h1:WWLmULLO7l8IOcQG+t+ItJ3fEcrL5FxF0Wu+HrMy26w= github.com/pingcap/kvproto v0.0.0-20200411081810-b85805c9476c/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= github.com/pingcap/kvproto v0.0.0-20210219064844-c1844a4775d6/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= -github.com/pingcap/kvproto v0.0.0-20210402093459-65aa336ccbbf h1:y8ZVU2X20+3XZW2M0/B8YAZ8RhsTnOuneXr1UfIKeNU= -github.com/pingcap/kvproto v0.0.0-20210402093459-65aa336ccbbf/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= +github.com/pingcap/kvproto v0.0.0-20210429093846-65f54a202d7e h1:oUMZ6X/Kpaoxfejh9/jQ+4UZ5xk9MRYcouWJ0oXRKNE= +github.com/pingcap/kvproto v0.0.0-20210429093846-65f54a202d7e/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= github.com/pingcap/log v0.0.0-20191012051959-b742a5d432e9/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20200511115504-543df19646ad/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20201112100606-8f1e84a3abc8/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= diff --git a/store/copr/mpp.go b/store/copr/mpp.go index 74fe82627a036..b170e9fee7f02 100644 --- a/store/copr/mpp.go +++ b/store/copr/mpp.go @@ -245,6 +245,13 @@ func (m *mppIterator) handleDispatchReq(ctx context.Context, bo *tikv.Backoffer, m.sendError(errors.New(realResp.Error.Msg)) return } + if len(realResp.RetryRegions) > 0 { + for _, retry := range realResp.RetryRegions { + id := tikv.NewRegionVerID(retry.Id, retry.RegionEpoch.ConfVer, retry.RegionEpoch.Version) + logutil.BgLogger().Info("invalid region because tiflash detected stale region", zap.String("region id", id.String())) + m.store.GetRegionCache().InvalidateCachedRegionWithReason(id, tikv.EpochNotMatch) + } + } failpoint.Inject("mppNonRootTaskError", func(val failpoint.Value) { if val.(bool) && !req.IsRoot { time.Sleep(1 * time.Second) diff --git a/store/mockstore/unistore/tikv/server.go b/store/mockstore/unistore/tikv/server.go index 478d11d16e131..f571ff4fe963f 100644 --- a/store/mockstore/unistore/tikv/server.go +++ b/store/mockstore/unistore/tikv/server.go @@ -951,36 +951,6 @@ func (svr *Server) RemoveLockObserver(context.Context, *kvrpcpb.RemoveLockObserv return &kvrpcpb.RemoveLockObserverResponse{}, nil } -// VerGet implements implements the tikvpb.TikvServer interface. -func (svr *Server) VerGet(context.Context, *kvrpcpb.VerGetRequest) (*kvrpcpb.VerGetResponse, error) { - panic("unimplemented") -} - -// VerBatchGet implements implements the tikvpb.TikvServer interface. -func (svr *Server) VerBatchGet(context.Context, *kvrpcpb.VerBatchGetRequest) (*kvrpcpb.VerBatchGetResponse, error) { - panic("unimplemented") -} - -// VerMut implements implements the tikvpb.TikvServer interface. -func (svr *Server) VerMut(context.Context, *kvrpcpb.VerMutRequest) (*kvrpcpb.VerMutResponse, error) { - panic("unimplemented") -} - -// VerBatchMut implements implements the tikvpb.TikvServer interface. -func (svr *Server) VerBatchMut(context.Context, *kvrpcpb.VerBatchMutRequest) (*kvrpcpb.VerBatchMutResponse, error) { - panic("unimplemented") -} - -// VerScan implements implements the tikvpb.TikvServer interface. -func (svr *Server) VerScan(context.Context, *kvrpcpb.VerScanRequest) (*kvrpcpb.VerScanResponse, error) { - panic("unimplemented") -} - -// VerDeleteRange implements implements the tikvpb.TikvServer interface. -func (svr *Server) VerDeleteRange(context.Context, *kvrpcpb.VerDeleteRangeRequest) (*kvrpcpb.VerDeleteRangeResponse, error) { - panic("unimplemented") -} - // CheckLeader implements implements the tikvpb.TikvServer interface. func (svr *Server) CheckLeader(context.Context, *kvrpcpb.CheckLeaderRequest) (*kvrpcpb.CheckLeaderResponse, error) { panic("unimplemented") diff --git a/store/tikv/region_cache.go b/store/tikv/region_cache.go index 9adfa65dd316a..c06394c6b166c 100644 --- a/store/tikv/region_cache.go +++ b/store/tikv/region_cache.go @@ -1545,6 +1545,11 @@ type RegionVerID struct { ver uint64 } +// NewRegionVerID creates a region ver id, which used for invalidating regions. +func NewRegionVerID(id, confVer, ver uint64) RegionVerID { + return RegionVerID{id, confVer, ver} +} + // GetID returns the id of the region func (r *RegionVerID) GetID() uint64 { return r.id diff --git a/store/tikv/region_request_test.go b/store/tikv/region_request_test.go index dab843013314b..81e9cc4498a07 100644 --- a/store/tikv/region_request_test.go +++ b/store/tikv/region_request_test.go @@ -471,30 +471,6 @@ func (s *mockTikvGrpcServer) ReadIndex(context.Context, *kvrpcpb.ReadIndexReques return nil, errors.New("unreachable") } -func (s *mockTikvGrpcServer) VerGet(context.Context, *kvrpcpb.VerGetRequest) (*kvrpcpb.VerGetResponse, error) { - return nil, errors.New("unreachable") -} - -func (s *mockTikvGrpcServer) VerBatchGet(context.Context, *kvrpcpb.VerBatchGetRequest) (*kvrpcpb.VerBatchGetResponse, error) { - return nil, errors.New("unreachable") -} - -func (s *mockTikvGrpcServer) VerMut(context.Context, *kvrpcpb.VerMutRequest) (*kvrpcpb.VerMutResponse, error) { - return nil, errors.New("unreachable") -} - -func (s *mockTikvGrpcServer) VerBatchMut(context.Context, *kvrpcpb.VerBatchMutRequest) (*kvrpcpb.VerBatchMutResponse, error) { - return nil, errors.New("unreachable") -} - -func (s *mockTikvGrpcServer) VerScan(context.Context, *kvrpcpb.VerScanRequest) (*kvrpcpb.VerScanResponse, error) { - return nil, errors.New("unreachable") -} - -func (s *mockTikvGrpcServer) VerDeleteRange(context.Context, *kvrpcpb.VerDeleteRangeRequest) (*kvrpcpb.VerDeleteRangeResponse, error) { - return nil, errors.New("unreachable") -} - func (s *mockTikvGrpcServer) CheckLeader(context.Context, *kvrpcpb.CheckLeaderRequest) (*kvrpcpb.CheckLeaderResponse, error) { return nil, errors.New("unreachable") } From 1cd3788334c5c4b551587f6773234a5f469a7200 Mon Sep 17 00:00:00 2001 From: bb7133 Date: Thu, 6 May 2021 13:35:52 +0800 Subject: [PATCH 08/67] planner: convert Sequence as DataSource to TableDual (#24405) --- planner/core/integration_test.go | 24 ++++++++++ planner/core/logical_plan_builder.go | 8 ++++ .../core/testdata/integration_suite_in.json | 10 +++++ .../core/testdata/integration_suite_out.json | 44 +++++++++++++++++++ 4 files changed, 86 insertions(+) diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index 25719198b66e4..de79bb35b62e2 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -3555,3 +3555,27 @@ func (s *testIntegrationSuite) TestConflictReadFromStorage(c *C) { tk.MustQuery(`explain select /*+ read_from_storage(tikv[t], tiflash[t]) */ * from t`) tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 Storage hints are conflict, you can only specify one storage type of table test.t")) } + +// TestSequenceAsDataSource is used to test https://github.com/pingcap/tidb/issues/24383. +func (s *testIntegrationSuite) TestSequenceAsDataSource(c *C) { + tk := testkit.NewTestKit(c, s.store) + + tk.MustExec("use test") + tk.MustExec("drop sequence if exists s1, s2") + tk.MustExec("create sequence s1") + tk.MustExec("create sequence s2") + + var input []string + var output []struct { + SQL string + Plan []string + } + s.testData.GetTestCases(c, &input, &output) + for i, tt := range input { + s.testData.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + tt).Rows()) + }) + tk.MustQuery("explain format = 'brief' " + tt).Check(testkit.Rows(output[i].Plan...)) + } +} diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index 0bf3d3fd3a7e4..7ff0e2ac6c6aa 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -3592,6 +3592,14 @@ func (b *PlanBuilder) buildDataSource(ctx context.Context, tn *ast.TableName, as return b.BuildDataSourceFromView(ctx, dbName, tableInfo) } + if tableInfo.IsSequence() { + if tn.TableSample != nil { + return nil, expression.ErrInvalidTableSample.GenWithStackByArgs("Unsupported TABLESAMPLE in sequences") + } + // When the source is a Sequence, we convert it to a TableDual, as what most databases do. + return b.buildTableDual(), nil + } + if tableInfo.GetPartitionInfo() != nil { // Use the new partition implementation, clean up the code here when it's full implemented. if !b.ctx.GetSessionVars().UseDynamicPartitionPrune() { diff --git a/planner/core/testdata/integration_suite_in.json b/planner/core/testdata/integration_suite_in.json index bf549da9431e2..087b32110e18f 100644 --- a/planner/core/testdata/integration_suite_in.json +++ b/planner/core/testdata/integration_suite_in.json @@ -284,5 +284,15 @@ "cases": [ "select count(*) from t join (select t.id, t.value v1 from t join t t1 on t.id = t1.id order by t.value limit 1) v on v.id = t.id and v.v1 = t.value;" ] + }, + { + "name": "TestSequenceAsDataSource", + "cases": [ + "select 1 from s1", + "select count(1) from s1", + "select count(*) from s1", + "select sum(1) from s1", + "select count(1) as cnt from s1 union select count(1) as cnt from s2" + ] } ] diff --git a/planner/core/testdata/integration_suite_out.json b/planner/core/testdata/integration_suite_out.json index a454ff4f31891..28fc950e37991 100644 --- a/planner/core/testdata/integration_suite_out.json +++ b/planner/core/testdata/integration_suite_out.json @@ -1520,5 +1520,49 @@ ] } ] + }, + { + "Name": "TestSequenceAsDataSource", + "Cases": [ + { + "SQL": "select 1 from s1", + "Plan": [ + "Projection 1.00 root 1->Column#1", + "└─TableDual 1.00 root rows:1" + ] + }, + { + "SQL": "select count(1) from s1", + "Plan": [ + "StreamAgg 1.00 root funcs:count(1)->Column#1", + "└─TableDual 1.00 root rows:1" + ] + }, + { + "SQL": "select count(*) from s1", + "Plan": [ + "StreamAgg 1.00 root funcs:count(1)->Column#1", + "└─TableDual 1.00 root rows:1" + ] + }, + { + "SQL": "select sum(1) from s1", + "Plan": [ + "StreamAgg 1.00 root funcs:sum(1)->Column#1", + "└─TableDual 1.00 root rows:1" + ] + }, + { + "SQL": "select count(1) as cnt from s1 union select count(1) as cnt from s2", + "Plan": [ + "HashAgg 2.00 root group by:Column#3, funcs:firstrow(Column#3)->Column#3", + "└─Union 2.00 root ", + " ├─StreamAgg 1.00 root funcs:count(1)->Column#1", + " │ └─TableDual 1.00 root rows:1", + " └─StreamAgg 1.00 root funcs:count(1)->Column#2", + " └─TableDual 1.00 root rows:1" + ] + } + ] } ] From b66b43dac5e1c8348e02b3ffd79747f11b32d90e Mon Sep 17 00:00:00 2001 From: Shirly Date: Thu, 6 May 2021 14:41:53 +0800 Subject: [PATCH 09/67] store/tikv: make tikv.ErrGCTooEarly as a normal error instead of dberror (#24400) --- executor/point_get_test.go | 12 +++---- store/copr/batch_coprocessor.go | 6 ++-- store/copr/coprocessor.go | 2 +- store/copr/mpp.go | 4 +-- store/copr/store.go | 47 ++++++++++++++++++++++++--- store/driver/tikv_driver.go | 11 ++++--- store/driver/txn/error.go | 16 +++++++-- store/driver/txn/snapshot.go | 4 +-- store/driver/txn/txn_driver.go | 10 +++--- store/driver/txn/unionstore_driver.go | 22 ++++++------- store/tikv/error/errcode.go | 1 - store/tikv/error/error.go | 23 +++++++++---- store/tikv/kv.go | 5 ++- store/tikv/tests/safepoint_test.go | 7 ++-- 14 files changed, 116 insertions(+), 54 deletions(-) diff --git a/executor/point_get_test.go b/executor/point_get_test.go index 54bce074dfb22..846b6f1628fe1 100644 --- a/executor/point_get_test.go +++ b/executor/point_get_test.go @@ -27,9 +27,9 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx/variable" + txndriver "github.com/pingcap/tidb/store/driver/txn" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/store/tikv" - tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/codec" @@ -536,15 +536,15 @@ func (s *testPointGetSuite) TestSelectCheckVisibility(c *C) { c.Assert(expectErr.Equal(err), IsTrue) } // Test point get. - checkSelectResultError("select * from t where a='1'", tikverr.ErrGCTooEarly) + checkSelectResultError("select * from t where a='1'", txndriver.ErrGCTooEarly) // Test batch point get. - checkSelectResultError("select * from t where a in ('1','2')", tikverr.ErrGCTooEarly) + checkSelectResultError("select * from t where a in ('1','2')", txndriver.ErrGCTooEarly) // Test Index look up read. - checkSelectResultError("select * from t where b > 0 ", tikverr.ErrGCTooEarly) + checkSelectResultError("select * from t where b > 0 ", txndriver.ErrGCTooEarly) // Test Index read. - checkSelectResultError("select b from t where b > 0 ", tikverr.ErrGCTooEarly) + checkSelectResultError("select b from t where b > 0 ", txndriver.ErrGCTooEarly) // Test table read. - checkSelectResultError("select * from t", tikverr.ErrGCTooEarly) + checkSelectResultError("select * from t", txndriver.ErrGCTooEarly) } func (s *testPointGetSuite) TestReturnValues(c *C) { diff --git a/store/copr/batch_coprocessor.go b/store/copr/batch_coprocessor.go index cec3e49644363..8a4336e0a06b6 100644 --- a/store/copr/batch_coprocessor.go +++ b/store/copr/batch_coprocessor.go @@ -183,12 +183,12 @@ func (c *CopClient) sendBatch(ctx context.Context, req *kv.Request, vars *tikv.V return copErrorResponse{err} } it := &batchCopIterator{ - store: c.store.KVStore, + store: c.store.kvStore, req: req, finishCh: make(chan struct{}), vars: vars, memTracker: req.MemTracker, - ClientHelper: tikv.NewClientHelper(c.store.KVStore, util.NewTSSet(5)), + ClientHelper: tikv.NewClientHelper(c.store.kvStore.store, util.NewTSSet(5)), rpcCancel: tikv.NewRPCanceller(), } ctx = context.WithValue(ctx, tikv.RPCCancellerCtxKey{}, it.rpcCancel) @@ -201,7 +201,7 @@ func (c *CopClient) sendBatch(ctx context.Context, req *kv.Request, vars *tikv.V type batchCopIterator struct { *tikv.ClientHelper - store *tikv.KVStore + store *kvStore req *kv.Request finishCh chan struct{} diff --git a/store/copr/coprocessor.go b/store/copr/coprocessor.go index 19a93c217d04c..fe255ffae05c1 100644 --- a/store/copr/coprocessor.go +++ b/store/copr/coprocessor.go @@ -402,7 +402,7 @@ func (it *copIterator) open(ctx context.Context, enabledRateLimitAction bool) { respChan: it.respChan, finishCh: it.finishCh, vars: it.vars, - ClientHelper: tikv.NewClientHelper(it.store.KVStore, it.resolvedLocks), + ClientHelper: tikv.NewClientHelper(it.store.store, it.resolvedLocks), memTracker: it.memTracker, replicaReadSeed: it.replicaReadSeed, actionOnExceed: it.actionOnExceed, diff --git a/store/copr/mpp.go b/store/copr/mpp.go index b170e9fee7f02..3058f64849986 100644 --- a/store/copr/mpp.go +++ b/store/copr/mpp.go @@ -36,7 +36,7 @@ import ( // MPPClient servers MPP requests. type MPPClient struct { - store *tikv.KVStore + store *kvStore } // GetAddress returns the network address. @@ -117,7 +117,7 @@ func (m *mppResponse) RespTime() time.Duration { } type mppIterator struct { - store *tikv.KVStore + store *kvStore tasks []*kv.MPPDispatchRequest finishCh chan struct{} diff --git a/store/copr/store.go b/store/copr/store.go index 2d0d42a38efb1..8dd6825de06fb 100644 --- a/store/copr/store.go +++ b/store/copr/store.go @@ -14,31 +14,70 @@ package copr import ( + "context" "math/rand" "sync/atomic" + "time" "github.com/pingcap/errors" "github.com/pingcap/tidb/kv" + txndriver "github.com/pingcap/tidb/store/driver/txn" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/config" "github.com/pingcap/tidb/store/tikv/tikvrpc" ) +type kvStore struct { + store *tikv.KVStore +} + +// GetRegionCache returns the region cache instance. +func (s *kvStore) GetRegionCache() *tikv.RegionCache { + return s.store.GetRegionCache() +} + +// CheckVisibility checks if it is safe to read using given ts. +func (s *kvStore) CheckVisibility(startTime uint64) error { + err := s.store.CheckVisibility(startTime) + return txndriver.ToTiDBErr(err) +} + +// GetTiKVClient gets the client instance. +func (s *kvStore) GetTiKVClient() tikv.Client { + client := s.store.GetTiKVClient() + return &tikvClient{c: client} +} + +type tikvClient struct { + c tikv.Client +} + +func (c *tikvClient) Close() error { + err := c.c.Close() + return txndriver.ToTiDBErr(err) +} + +// SendRequest sends Request. +func (c *tikvClient) SendRequest(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (*tikvrpc.Response, error) { + res, err := c.c.SendRequest(ctx, addr, req, timeout) + return res, txndriver.ToTiDBErr(err) +} + // Store wraps tikv.KVStore and provides coprocessor utilities. type Store struct { - *tikv.KVStore + *kvStore coprCache *coprCache replicaReadSeed uint32 } // NewStore creates a new store instance. -func NewStore(kvStore *tikv.KVStore, coprCacheConfig *config.CoprocessorCache) (*Store, error) { +func NewStore(s *tikv.KVStore, coprCacheConfig *config.CoprocessorCache) (*Store, error) { coprCache, err := newCoprCache(coprCacheConfig) if err != nil { return nil, errors.Trace(err) } return &Store{ - KVStore: kvStore, + kvStore: &kvStore{store: s}, coprCache: coprCache, replicaReadSeed: rand.Uint32(), }, nil @@ -66,7 +105,7 @@ func (s *Store) GetClient() kv.Client { // GetMPPClient gets a mpp client instance. func (s *Store) GetMPPClient() kv.MPPClient { return &MPPClient{ - store: s.KVStore, + store: s.kvStore, } } diff --git a/store/driver/tikv_driver.go b/store/driver/tikv_driver.go index 62d54052a3f8a..17308d33c6be3 100644 --- a/store/driver/tikv_driver.go +++ b/store/driver/tikv_driver.go @@ -262,7 +262,7 @@ func (s *tikvStore) StartGCWorker() error { gcWorker, err := gcworker.NewGCWorker(s, s.pdClient) if err != nil { - return errors.Trace(err) + return txn_driver.ToTiDBErr(err) } gcWorker.Start() s.gcWorker = gcWorker @@ -286,7 +286,8 @@ func (s *tikvStore) Close() error { s.gcWorker.Close() } s.coprStore.Close() - return s.KVStore.Close() + err := s.KVStore.Close() + return txn_driver.ToTiDBErr(err) } // GetMemCache return memory manager of the storage @@ -298,7 +299,7 @@ func (s *tikvStore) GetMemCache() kv.MemManager { func (s *tikvStore) Begin() (kv.Transaction, error) { txn, err := s.KVStore.Begin() if err != nil { - return nil, errors.Trace(err) + return nil, txn_driver.ToTiDBErr(err) } return txn_driver.NewTiKVTxn(txn), err } @@ -323,7 +324,7 @@ func (s *tikvStore) BeginWithOption(option kv.TransactionOption) (kv.Transaction txn, err = s.BeginWithTxnScope(txnScope) } if err != nil { - return nil, errors.Trace(err) + return nil, txn_driver.ToTiDBErr(err) } return txn_driver.NewTiKVTxn(txn), err @@ -338,7 +339,7 @@ func (s *tikvStore) GetSnapshot(ver kv.Version) kv.Snapshot { // CurrentVersion returns current max committed version with the given txnScope (local or global). func (s *tikvStore) CurrentVersion(txnScope string) (kv.Version, error) { ver, err := s.KVStore.CurrentTimestamp(txnScope) - return kv.NewVersion(ver), err + return kv.NewVersion(ver), txn_driver.ToTiDBErr(err) } // ShowStatus returns the specified status of the storage diff --git a/store/driver/txn/error.go b/store/driver/txn/error.go index 10e08aaf35d08..08e4290805bf8 100644 --- a/store/driver/txn/error.go +++ b/store/driver/txn/error.go @@ -40,6 +40,8 @@ import ( var ( // ErrTiKVServerTimeout is the error when tikv server is timeout. ErrTiKVServerTimeout = dbterror.ClassTiKV.NewStd(errno.ErrTiKVServerTimeout) + // ErrGCTooEarly is the error that GC life time is shorter than transaction duration + ErrGCTooEarly = dbterror.ClassTiKV.NewStd(errno.ErrGCTooEarly) // ErrTiKVStaleCommand is the error that the command is stale in tikv. ErrTiKVStaleCommand = dbterror.ClassTiKV.NewStd(errno.ErrTiKVStaleCommand) // ErrTiKVMaxTimestampNotSynced is the error that tikv's max timestamp is not synced. @@ -158,13 +160,16 @@ func extractKeyErr(err error) error { notFoundDetail := prettyLockNotFoundKey(e.Retryable) return kv.ErrTxnRetryable.GenWithStackByArgs(e.Retryable + " " + notFoundDetail) } - return toTiDBErr(err) + return ToTiDBErr(err) } -func toTiDBErr(err error) error { +// ToTiDBErr checks and converts a tikv error to a tidb error. +func ToTiDBErr(err error) error { + originErr := err if err == nil { return nil } + err = errors.Cause(err) if tikverr.IsErrNotFound(err) { return kv.ErrNotExist } @@ -193,6 +198,10 @@ func toTiDBErr(err error) error { return ErrTiKVServerTimeout } + if e, ok := err.(*tikverr.ErrGCTooEarly); ok { + return ErrGCTooEarly.GenWithStackByArgs(e.TxnStartTS, e.GCSafePoint) + } + if errors.ErrorEqual(err, tikverr.ErrTiKVStaleCommand) { return ErrTiKVStaleCommand } @@ -204,7 +213,8 @@ func toTiDBErr(err error) error { if errors.ErrorEqual(err, tikverr.ErrResolveLockTimeout) { return ErrResolveLockTimeout } - return errors.Trace(err) + + return errors.Trace(originErr) } func newWriteConflictError(conflict *kvrpcpb.WriteConflict) error { diff --git a/store/driver/txn/snapshot.go b/store/driver/txn/snapshot.go index 3d41ef8cfa1ae..1a3f7bf9c3bbc 100644 --- a/store/driver/txn/snapshot.go +++ b/store/driver/txn/snapshot.go @@ -48,7 +48,7 @@ func (s *tikvSnapshot) Get(ctx context.Context, k kv.Key) ([]byte, error) { func (s *tikvSnapshot) Iter(k kv.Key, upperBound kv.Key) (kv.Iterator, error) { scanner, err := s.KVSnapshot.Iter(k, upperBound) if err != nil { - return nil, toTiDBErr(err) + return nil, ToTiDBErr(err) } return &tikvScanner{scanner.(*tikv.Scanner)}, err } @@ -57,7 +57,7 @@ func (s *tikvSnapshot) Iter(k kv.Key, upperBound kv.Key) (kv.Iterator, error) { func (s *tikvSnapshot) IterReverse(k kv.Key) (kv.Iterator, error) { scanner, err := s.KVSnapshot.IterReverse(k) if err != nil { - return nil, toTiDBErr(err) + return nil, ToTiDBErr(err) } return &tikvScanner{scanner.(*tikv.Scanner)}, err } diff --git a/store/driver/txn/txn_driver.go b/store/driver/txn/txn_driver.go index 3b5435d596c77..8595b2fe874ae 100644 --- a/store/driver/txn/txn_driver.go +++ b/store/driver/txn/txn_driver.go @@ -75,7 +75,7 @@ func (txn *tikvTxn) GetSnapshot() kv.Snapshot { // The Iterator must be Closed after use. func (txn *tikvTxn) Iter(k kv.Key, upperBound kv.Key) (kv.Iterator, error) { it, err := txn.KVTxn.Iter(k, upperBound) - return newKVIterator(it), toTiDBErr(err) + return newKVIterator(it), ToTiDBErr(err) } // IterReverse creates a reversed Iterator positioned on the first entry which key is less than k. @@ -84,7 +84,7 @@ func (txn *tikvTxn) Iter(k kv.Key, upperBound kv.Key) (kv.Iterator, error) { // TODO: Add lower bound limit func (txn *tikvTxn) IterReverse(k kv.Key) (kv.Iterator, error) { it, err := txn.KVTxn.IterReverse(k) - return newKVIterator(it), toTiDBErr(err) + return newKVIterator(it), ToTiDBErr(err) } // BatchGet gets kv from the memory buffer of statement and transaction, and the kv storage. @@ -101,17 +101,17 @@ func (txn *tikvTxn) BatchGet(ctx context.Context, keys []kv.Key) (map[string][]b func (txn *tikvTxn) Delete(k kv.Key) error { err := txn.KVTxn.Delete(k) - return toTiDBErr(err) + return ToTiDBErr(err) } func (txn *tikvTxn) Get(ctx context.Context, k kv.Key) ([]byte, error) { data, err := txn.KVTxn.Get(ctx, k) - return data, toTiDBErr(err) + return data, ToTiDBErr(err) } func (txn *tikvTxn) Set(k kv.Key, v []byte) error { err := txn.KVTxn.Set(k, v) - return toTiDBErr(err) + return ToTiDBErr(err) } func (txn *tikvTxn) GetMemBuffer() kv.MemBuffer { diff --git a/store/driver/txn/unionstore_driver.go b/store/driver/txn/unionstore_driver.go index 31cb1f2a35f06..9db2325a0148f 100644 --- a/store/driver/txn/unionstore_driver.go +++ b/store/driver/txn/unionstore_driver.go @@ -39,17 +39,17 @@ func (m *memBuffer) Delete(k kv.Key) error { func (m *memBuffer) DeleteWithFlags(k kv.Key, ops ...tikvstore.FlagsOp) error { err := m.MemDB.DeleteWithFlags(k, ops...) - return toTiDBErr(err) + return ToTiDBErr(err) } func (m *memBuffer) Get(_ context.Context, key kv.Key) ([]byte, error) { data, err := m.MemDB.Get(key) - return data, toTiDBErr(err) + return data, ToTiDBErr(err) } func (m *memBuffer) GetFlags(key kv.Key) (tikvstore.KeyFlags, error) { data, err := m.MemDB.GetFlags(key) - return data, toTiDBErr(err) + return data, ToTiDBErr(err) } func (m *memBuffer) Staging() kv.StagingHandle { @@ -73,12 +73,12 @@ func (m *memBuffer) InspectStage(handle kv.StagingHandle, f func(kv.Key, tikvsto func (m *memBuffer) Set(key kv.Key, value []byte) error { err := m.MemDB.Set(key, value) - return toTiDBErr(err) + return ToTiDBErr(err) } func (m *memBuffer) SetWithFlags(key kv.Key, value []byte, ops ...kv.FlagsOp) error { err := m.MemDB.SetWithFlags(key, value, ops...) - return toTiDBErr(err) + return ToTiDBErr(err) } // Iter creates an Iterator positioned on the first entry that k <= entry's key. @@ -87,7 +87,7 @@ func (m *memBuffer) SetWithFlags(key kv.Key, value []byte, ops ...kv.FlagsOp) er // The Iterator must be Closed after use. func (m *memBuffer) Iter(k kv.Key, upperBound kv.Key) (kv.Iterator, error) { it, err := m.MemDB.Iter(k, upperBound) - return &tikvIterator{Iterator: it}, toTiDBErr(err) + return &tikvIterator{Iterator: it}, ToTiDBErr(err) } // IterReverse creates a reversed Iterator positioned on the first entry which key is less than k. @@ -96,7 +96,7 @@ func (m *memBuffer) Iter(k kv.Key, upperBound kv.Key) (kv.Iterator, error) { // TODO: Add lower bound limit func (m *memBuffer) IterReverse(k kv.Key) (kv.Iterator, error) { it, err := m.MemDB.IterReverse(k) - return &tikvIterator{Iterator: it}, toTiDBErr(err) + return &tikvIterator{Iterator: it}, ToTiDBErr(err) } // SnapshotIter returns a Iterator for a snapshot of MemBuffer. @@ -121,7 +121,7 @@ func (u *tikvUnionStore) GetMemBuffer() kv.MemBuffer { func (u *tikvUnionStore) Get(ctx context.Context, k kv.Key) ([]byte, error) { data, err := u.KVUnionStore.Get(ctx, k) - return data, toTiDBErr(err) + return data, ToTiDBErr(err) } func (u *tikvUnionStore) HasPresumeKeyNotExists(k kv.Key) bool { @@ -134,7 +134,7 @@ func (u *tikvUnionStore) UnmarkPresumeKeyNotExists(k kv.Key) { func (u *tikvUnionStore) Iter(k kv.Key, upperBound kv.Key) (kv.Iterator, error) { it, err := u.KVUnionStore.Iter(k, upperBound) - return newKVIterator(it), toTiDBErr(err) + return newKVIterator(it), ToTiDBErr(err) } // IterReverse creates a reversed Iterator positioned on the first entry which key is less than k. @@ -143,7 +143,7 @@ func (u *tikvUnionStore) Iter(k kv.Key, upperBound kv.Key) (kv.Iterator, error) // TODO: Add lower bound limit func (u *tikvUnionStore) IterReverse(k kv.Key) (kv.Iterator, error) { it, err := u.KVUnionStore.IterReverse(k) - return newKVIterator(it), toTiDBErr(err) + return newKVIterator(it), ToTiDBErr(err) } type tikvGetter struct { @@ -156,7 +156,7 @@ func newKVGetter(getter unionstore.Getter) kv.Getter { func (g *tikvGetter) Get(_ context.Context, k kv.Key) ([]byte, error) { data, err := g.Getter.Get(k) - return data, toTiDBErr(err) + return data, ToTiDBErr(err) } // tikvIterator wraps unionstore.Iterator as kv.Iterator diff --git a/store/tikv/error/errcode.go b/store/tikv/error/errcode.go index 1c6c77eb2c210..5090214d3699a 100644 --- a/store/tikv/error/errcode.go +++ b/store/tikv/error/errcode.go @@ -28,7 +28,6 @@ const ( CodePDServerTimeout = 9001 CodeTiKVServerBusy = 9003 CodeRegionUnavailable = 9005 - CodeGCTooEarly = 9006 CodeTiKVStoreLimit = 9008 diff --git a/store/tikv/error/error.go b/store/tikv/error/error.go index 57efa7c82bea7..f3352d9b0a307 100644 --- a/store/tikv/error/error.go +++ b/store/tikv/error/error.go @@ -15,6 +15,7 @@ package error import ( "fmt" + "time" "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/kvrpcpb" @@ -48,13 +49,11 @@ const MismatchClusterID = "mismatch cluster id" // error instances. var ( - ErrTiFlashServerTimeout = dbterror.ClassTiKV.NewStd(CodeTiFlashServerTimeout) - ErrPDServerTimeout = dbterror.ClassTiKV.NewStd(CodePDServerTimeout) - ErrRegionUnavailable = dbterror.ClassTiKV.NewStd(CodeRegionUnavailable) - ErrTiKVServerBusy = dbterror.ClassTiKV.NewStd(CodeTiKVServerBusy) - ErrTiFlashServerBusy = dbterror.ClassTiKV.NewStd(CodeTiFlashServerBusy) - - ErrGCTooEarly = dbterror.ClassTiKV.NewStd(CodeGCTooEarly) + ErrTiFlashServerTimeout = dbterror.ClassTiKV.NewStd(CodeTiFlashServerTimeout) + ErrPDServerTimeout = dbterror.ClassTiKV.NewStd(CodePDServerTimeout) + ErrRegionUnavailable = dbterror.ClassTiKV.NewStd(CodeRegionUnavailable) + ErrTiKVServerBusy = dbterror.ClassTiKV.NewStd(CodeTiKVServerBusy) + ErrTiFlashServerBusy = dbterror.ClassTiKV.NewStd(CodeTiFlashServerBusy) ErrQueryInterrupted = dbterror.ClassTiKV.NewStd(CodeQueryInterrupted) ErrLockAcquireFailAndNoWaitSet = dbterror.ClassTiKV.NewStd(CodeLockAcquireFailAndNoWaitSet) ErrLockWaitTimeout = dbterror.ClassTiKV.NewStd(CodeLockWaitTimeout) @@ -171,3 +170,13 @@ type ErrEntryTooLarge struct { func (e *ErrEntryTooLarge) Error() string { return fmt.Sprintf("entry size too large, size: %v,limit: %v.", e.Size, e.Limit) } + +// ErrGCTooEarly is the error that GC life time is shorter than transaction duration +type ErrGCTooEarly struct { + TxnStartTS time.Time + GCSafePoint time.Time +} + +func (e *ErrGCTooEarly) Error() string { + return fmt.Sprintf("GC life time is shorter than transaction duration, transaction starts at %v, GC safe point is %v", e.TxnStartTS, e.GCSafePoint) +} diff --git a/store/tikv/kv.go b/store/tikv/kv.go index cdbafe080ace1..d919dc329b629 100644 --- a/store/tikv/kv.go +++ b/store/tikv/kv.go @@ -113,7 +113,10 @@ func (s *KVStore) CheckVisibility(startTime uint64) error { if startTime < cachedSafePoint { t1 := oracle.GetTimeFromTS(startTime) t2 := oracle.GetTimeFromTS(cachedSafePoint) - return tikverr.ErrGCTooEarly.GenWithStackByArgs(t1, t2) + return &tikverr.ErrGCTooEarly{ + TxnStartTS: t1, + GCSafePoint: t2, + } } return nil diff --git a/store/tikv/tests/safepoint_test.go b/store/tikv/tests/safepoint_test.go index 6135dd63bba5c..bf06e8b7e00d8 100644 --- a/store/tikv/tests/safepoint_test.go +++ b/store/tikv/tests/safepoint_test.go @@ -91,7 +91,8 @@ func (s *testSafePointSuite) TestSafePoint(c *C) { _, geterr2 := txn2.Get(context.TODO(), encodeKey(s.prefix, s08d("key", 0))) c.Assert(geterr2, NotNil) - isFallBehind := terror.ErrorEqual(errors.Cause(geterr2), tikverr.ErrGCTooEarly) + + _, isFallBehind := errors.Cause(geterr2).(*tikverr.ErrGCTooEarly) isMayFallBehind := terror.ErrorEqual(errors.Cause(geterr2), tikverr.ErrPDServerTimeout.GenWithStackByArgs("start timestamp may fall behind safe point")) isBehind := isFallBehind || isMayFallBehind c.Assert(isBehind, IsTrue) @@ -103,7 +104,7 @@ func (s *testSafePointSuite) TestSafePoint(c *C) { _, seekerr := txn3.Iter(encodeKey(s.prefix, ""), nil) c.Assert(seekerr, NotNil) - isFallBehind = terror.ErrorEqual(errors.Cause(geterr2), tikverr.ErrGCTooEarly) + _, isFallBehind = errors.Cause(geterr2).(*tikverr.ErrGCTooEarly) isMayFallBehind = terror.ErrorEqual(errors.Cause(geterr2), tikverr.ErrPDServerTimeout.GenWithStackByArgs("start timestamp may fall behind safe point")) isBehind = isFallBehind || isMayFallBehind c.Assert(isBehind, IsTrue) @@ -116,7 +117,7 @@ func (s *testSafePointSuite) TestSafePoint(c *C) { _, batchgeterr := toTiDBTxn(&txn4).BatchGet(context.Background(), toTiDBKeys(keys)) c.Assert(batchgeterr, NotNil) - isFallBehind = terror.ErrorEqual(errors.Cause(geterr2), tikverr.ErrGCTooEarly) + _, isFallBehind = errors.Cause(geterr2).(*tikverr.ErrGCTooEarly) isMayFallBehind = terror.ErrorEqual(errors.Cause(geterr2), tikverr.ErrPDServerTimeout.GenWithStackByArgs("start timestamp may fall behind safe point")) isBehind = isFallBehind || isMayFallBehind c.Assert(isBehind, IsTrue) From 82cf05e599fbe1c60699371af79ff15d0ef7af6a Mon Sep 17 00:00:00 2001 From: Shirly Date: Thu, 6 May 2021 15:06:27 +0800 Subject: [PATCH 10/67] =?UTF-8?q?store/tikv:=20make=20tikv.ErrTiKVServerBu?= =?UTF-8?q?sy=20as=20a=20normal=20error=20instead=20of=20=E2=80=A6=20(#243?= =?UTF-8?q?91)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: Ti Chi Robot --- store/driver/txn/error.go | 6 ++++++ store/tikv/error/errcode.go | 1 - store/tikv/error/error.go | 3 ++- 3 files changed, 8 insertions(+), 2 deletions(-) diff --git a/store/driver/txn/error.go b/store/driver/txn/error.go index 08e4290805bf8..d054b3b9601e5 100644 --- a/store/driver/txn/error.go +++ b/store/driver/txn/error.go @@ -47,6 +47,8 @@ var ( // ErrTiKVMaxTimestampNotSynced is the error that tikv's max timestamp is not synced. ErrTiKVMaxTimestampNotSynced = dbterror.ClassTiKV.NewStd(errno.ErrTiKVMaxTimestampNotSynced) ErrResolveLockTimeout = dbterror.ClassTiKV.NewStd(errno.ErrResolveLockTimeout) + // ErrTiKVServerBusy is the error when tikv server is busy. + ErrTiKVServerBusy = dbterror.ClassTiKV.NewStd(errno.ErrTiKVServerBusy) ) func genKeyExistsError(name string, value string, err error) error { @@ -198,6 +200,10 @@ func ToTiDBErr(err error) error { return ErrTiKVServerTimeout } + if errors.ErrorEqual(err, tikverr.ErrTiKVServerBusy) { + return ErrTiKVServerBusy + } + if e, ok := err.(*tikverr.ErrGCTooEarly); ok { return ErrGCTooEarly.GenWithStackByArgs(e.TxnStartTS, e.GCSafePoint) } diff --git a/store/tikv/error/errcode.go b/store/tikv/error/errcode.go index 5090214d3699a..db1c9585be62e 100644 --- a/store/tikv/error/errcode.go +++ b/store/tikv/error/errcode.go @@ -26,7 +26,6 @@ const ( // TiKV/PD/TiFlash errors. CodePDServerTimeout = 9001 - CodeTiKVServerBusy = 9003 CodeRegionUnavailable = 9005 CodeTiKVStoreLimit = 9008 diff --git a/store/tikv/error/error.go b/store/tikv/error/error.go index f3352d9b0a307..89685159edc40 100644 --- a/store/tikv/error/error.go +++ b/store/tikv/error/error.go @@ -42,6 +42,8 @@ var ( ErrTiKVMaxTimestampNotSynced = errors.New("tikv max timestamp not synced") // ErrResolveLockTimeout is the error that resolve lock timeout. ErrResolveLockTimeout = errors.New("resolve lock timeout") + // ErrTiKVServerBusy is the error when tikv server is busy. + ErrTiKVServerBusy = errors.New("tikv server busy") ) // MismatchClusterID represents the message that the cluster ID of the PD client does not match the PD. @@ -52,7 +54,6 @@ var ( ErrTiFlashServerTimeout = dbterror.ClassTiKV.NewStd(CodeTiFlashServerTimeout) ErrPDServerTimeout = dbterror.ClassTiKV.NewStd(CodePDServerTimeout) ErrRegionUnavailable = dbterror.ClassTiKV.NewStd(CodeRegionUnavailable) - ErrTiKVServerBusy = dbterror.ClassTiKV.NewStd(CodeTiKVServerBusy) ErrTiFlashServerBusy = dbterror.ClassTiKV.NewStd(CodeTiFlashServerBusy) ErrQueryInterrupted = dbterror.ClassTiKV.NewStd(CodeQueryInterrupted) ErrLockAcquireFailAndNoWaitSet = dbterror.ClassTiKV.NewStd(CodeLockAcquireFailAndNoWaitSet) From 26c948d53886d041d5a5397fd2d8cbafbd9fac79 Mon Sep 17 00:00:00 2001 From: Shirly Date: Thu, 6 May 2021 15:49:52 +0800 Subject: [PATCH 11/67] =?UTF-8?q?store/tikv:=20make=20tikv.PDServerTimeout?= =?UTF-8?q?=20as=20a=20normal=20error=20instead=20of=20db=E2=80=A6=20(#243?= =?UTF-8?q?88)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- store/driver/txn/error.go | 9 +++++++++ store/tikv/backoff.go | 2 +- store/tikv/error/errcode.go | 1 - store/tikv/error/error.go | 15 ++++++++++++++- store/tikv/kv.go | 4 ++-- store/tikv/split_region.go | 4 ++-- store/tikv/tests/safepoint_test.go | 6 +++--- 7 files changed, 31 insertions(+), 10 deletions(-) diff --git a/store/driver/txn/error.go b/store/driver/txn/error.go index d054b3b9601e5..42b16e5930ab1 100644 --- a/store/driver/txn/error.go +++ b/store/driver/txn/error.go @@ -49,6 +49,8 @@ var ( ErrResolveLockTimeout = dbterror.ClassTiKV.NewStd(errno.ErrResolveLockTimeout) // ErrTiKVServerBusy is the error when tikv server is busy. ErrTiKVServerBusy = dbterror.ClassTiKV.NewStd(errno.ErrTiKVServerBusy) + // ErrPDServerTimeout is the error when pd server is timeout. + ErrPDServerTimeout = dbterror.ClassTiKV.NewStd(errno.ErrPDServerTimeout) ) func genKeyExistsError(name string, value string, err error) error { @@ -200,6 +202,13 @@ func ToTiDBErr(err error) error { return ErrTiKVServerTimeout } + if e, ok := err.(*tikverr.ErrPDServerTimeout); ok { + if len(e.Error()) == 0 { + return ErrPDServerTimeout + } + return ErrPDServerTimeout.GenWithStackByArgs(e.Error()) + } + if errors.ErrorEqual(err, tikverr.ErrTiKVServerBusy) { return ErrTiKVServerBusy } diff --git a/store/tikv/backoff.go b/store/tikv/backoff.go index 03d7930071fb3..e0115c9e3904a 100644 --- a/store/tikv/backoff.go +++ b/store/tikv/backoff.go @@ -200,7 +200,7 @@ func (t BackoffType) TError() error { case BoTxnLock, BoTxnLockFast, boTxnNotFound: return tikverr.ErrResolveLockTimeout case BoPDRPC: - return tikverr.ErrPDServerTimeout + return tikverr.NewErrPDServerTimeout("") case BoRegionMiss: return tikverr.ErrRegionUnavailable case boTiKVServerBusy: diff --git a/store/tikv/error/errcode.go b/store/tikv/error/errcode.go index db1c9585be62e..6d18e35859a07 100644 --- a/store/tikv/error/errcode.go +++ b/store/tikv/error/errcode.go @@ -25,7 +25,6 @@ const ( CodeLockAcquireFailAndNoWaitSet = 3572 // TiKV/PD/TiFlash errors. - CodePDServerTimeout = 9001 CodeRegionUnavailable = 9005 CodeTiKVStoreLimit = 9008 diff --git a/store/tikv/error/error.go b/store/tikv/error/error.go index 89685159edc40..cb6698f454f5c 100644 --- a/store/tikv/error/error.go +++ b/store/tikv/error/error.go @@ -52,7 +52,6 @@ const MismatchClusterID = "mismatch cluster id" // error instances. var ( ErrTiFlashServerTimeout = dbterror.ClassTiKV.NewStd(CodeTiFlashServerTimeout) - ErrPDServerTimeout = dbterror.ClassTiKV.NewStd(CodePDServerTimeout) ErrRegionUnavailable = dbterror.ClassTiKV.NewStd(CodeRegionUnavailable) ErrTiFlashServerBusy = dbterror.ClassTiKV.NewStd(CodeTiFlashServerBusy) ErrQueryInterrupted = dbterror.ClassTiKV.NewStd(CodeQueryInterrupted) @@ -172,6 +171,20 @@ func (e *ErrEntryTooLarge) Error() string { return fmt.Sprintf("entry size too large, size: %v,limit: %v.", e.Size, e.Limit) } +// ErrPDServerTimeout is the error when pd server is timeout. +type ErrPDServerTimeout struct { + msg string +} + +// NewErrPDServerTimeout creates an ErrPDServerTimeout. +func NewErrPDServerTimeout(msg string) error { + return &ErrPDServerTimeout{msg} +} + +func (e *ErrPDServerTimeout) Error() string { + return e.msg +} + // ErrGCTooEarly is the error that GC life time is shorter than transaction duration type ErrGCTooEarly struct { TxnStartTS time.Time diff --git a/store/tikv/kv.go b/store/tikv/kv.go index d919dc329b629..0f4824a785ecb 100644 --- a/store/tikv/kv.go +++ b/store/tikv/kv.go @@ -107,7 +107,7 @@ func (s *KVStore) CheckVisibility(startTime uint64) error { diff := time.Since(cachedTime) if diff > (GcSafePointCacheInterval - gcCPUTimeInaccuracyBound) { - return tikverr.ErrPDServerTimeout.GenWithStackByArgs("start timestamp may fall behind safe point") + return tikverr.NewErrPDServerTimeout("start timestamp may fall behind safe point") } if startTime < cachedSafePoint { @@ -310,7 +310,7 @@ func (s *KVStore) getTimestampWithRetry(bo *Backoffer, txnScope string) (uint64, // This may cause duplicate data to be written. failpoint.Inject("mockGetTSErrorInRetry", func(val failpoint.Value) { if val.(bool) && !IsMockCommitErrorEnable() { - err = tikverr.ErrPDServerTimeout.GenWithStackByArgs("mock PD timeout") + err = tikverr.NewErrPDServerTimeout("mock PD timeout") } }) diff --git a/store/tikv/split_region.go b/store/tikv/split_region.go index e8d988db328ab..5839aa4d73c96 100644 --- a/store/tikv/split_region.go +++ b/store/tikv/split_region.go @@ -185,7 +185,7 @@ func (s *KVStore) batchSendSingleRegion(bo *Backoffer, batch batch, scatter bool if batchResp.err == nil { batchResp.err = err } - if tikverr.ErrPDServerTimeout.Equal(err) { + if _, ok := err.(*tikverr.ErrPDServerTimeout); ok { break } } @@ -219,7 +219,7 @@ func (s *KVStore) scatterRegion(bo *Backoffer, regionID uint64, tableID *int64) if val, err2 := util.MockScatterRegionTimeout.Eval(); err2 == nil { if val.(bool) { - err = tikverr.ErrPDServerTimeout + err = tikverr.NewErrPDServerTimeout("") } } diff --git a/store/tikv/tests/safepoint_test.go b/store/tikv/tests/safepoint_test.go index bf06e8b7e00d8..84249d0104997 100644 --- a/store/tikv/tests/safepoint_test.go +++ b/store/tikv/tests/safepoint_test.go @@ -93,7 +93,7 @@ func (s *testSafePointSuite) TestSafePoint(c *C) { c.Assert(geterr2, NotNil) _, isFallBehind := errors.Cause(geterr2).(*tikverr.ErrGCTooEarly) - isMayFallBehind := terror.ErrorEqual(errors.Cause(geterr2), tikverr.ErrPDServerTimeout.GenWithStackByArgs("start timestamp may fall behind safe point")) + isMayFallBehind := terror.ErrorEqual(errors.Cause(geterr2), tikverr.NewErrPDServerTimeout("start timestamp may fall behind safe point")) isBehind := isFallBehind || isMayFallBehind c.Assert(isBehind, IsTrue) @@ -105,7 +105,7 @@ func (s *testSafePointSuite) TestSafePoint(c *C) { _, seekerr := txn3.Iter(encodeKey(s.prefix, ""), nil) c.Assert(seekerr, NotNil) _, isFallBehind = errors.Cause(geterr2).(*tikverr.ErrGCTooEarly) - isMayFallBehind = terror.ErrorEqual(errors.Cause(geterr2), tikverr.ErrPDServerTimeout.GenWithStackByArgs("start timestamp may fall behind safe point")) + isMayFallBehind = terror.ErrorEqual(errors.Cause(geterr2), tikverr.NewErrPDServerTimeout("start timestamp may fall behind safe point")) isBehind = isFallBehind || isMayFallBehind c.Assert(isBehind, IsTrue) @@ -118,7 +118,7 @@ func (s *testSafePointSuite) TestSafePoint(c *C) { _, batchgeterr := toTiDBTxn(&txn4).BatchGet(context.Background(), toTiDBKeys(keys)) c.Assert(batchgeterr, NotNil) _, isFallBehind = errors.Cause(geterr2).(*tikverr.ErrGCTooEarly) - isMayFallBehind = terror.ErrorEqual(errors.Cause(geterr2), tikverr.ErrPDServerTimeout.GenWithStackByArgs("start timestamp may fall behind safe point")) + isMayFallBehind = terror.ErrorEqual(errors.Cause(geterr2), tikverr.NewErrPDServerTimeout("start timestamp may fall behind safe point")) isBehind = isFallBehind || isMayFallBehind c.Assert(isBehind, IsTrue) } From cb74578c117c5ea6900dbcbd8cdf9dec34dd2617 Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Thu, 6 May 2021 16:49:52 +0800 Subject: [PATCH 12/67] executor: solve the compatibility problem between UnionScan and partition tables and remove all code about PartitionTableExec (#24394) --- executor/builder.go | 13 ----- executor/partition_table.go | 100 ------------------------------------ 2 files changed, 113 deletions(-) diff --git a/executor/builder.go b/executor/builder.go index da361f4192395..e82db0d6aaccc 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -953,19 +953,6 @@ func (b *executorBuilder) buildUnionScanExec(v *plannercore.PhysicalUnionScan) E // Note that this function may be called by inner workers of index lookup join concurrently. // Be careful to avoid data race. func (b *executorBuilder) buildUnionScanFromReader(reader Executor, v *plannercore.PhysicalUnionScan) Executor { - // Adjust UnionScan->PartitionTable->Reader - // to PartitionTable->UnionScan->Reader - // The build of UnionScan executor is delay to the nextPartition() function - // because the Reader executor is available there. - if x, ok := reader.(*PartitionTableExecutor); ok { - nextPartitionForReader := x.nextPartition - x.nextPartition = nextPartitionForUnionScan{ - b: b, - us: v, - child: nextPartitionForReader, - } - return x - } // If reader is union, it means a partition table and we should transfer as above. if x, ok := reader.(*UnionExec); ok { for i, child := range x.children { diff --git a/executor/partition_table.go b/executor/partition_table.go index 3cebdec6c14e6..e888332fee396 100644 --- a/executor/partition_table.go +++ b/executor/partition_table.go @@ -17,58 +17,10 @@ import ( "context" "fmt" - "github.com/opentracing/opentracing-go" "github.com/pingcap/errors" - plannercore "github.com/pingcap/tidb/planner/core" - "github.com/pingcap/tidb/table" - "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tipb/go-tipb" ) -// PartitionTableExecutor is a Executor for partitioned table. -// It works by wrap the underlying TableReader/IndexReader/IndexLookUpReader. -type PartitionTableExecutor struct { - baseExecutor - - nextPartition - partitions []table.PhysicalTable - cursor int - curr Executor -} - -type nextPartition interface { - nextPartition(context.Context, table.PhysicalTable) (Executor, error) -} - -type nextPartitionForUnionScan struct { - b *executorBuilder - us *plannercore.PhysicalUnionScan - child nextPartition -} - -// nextPartition implements the nextPartition interface. -// For union scan on partitioned table, the executor should be PartitionTable->UnionScan->TableReader rather than -// UnionScan->PartitionTable->TableReader -func (n nextPartitionForUnionScan) nextPartition(ctx context.Context, tbl table.PhysicalTable) (Executor, error) { - childExec, err := n.child.nextPartition(ctx, tbl) - if err != nil { - return nil, err - } - - n.b.err = nil - ret := n.b.buildUnionScanFromReader(childExec, n.us) - return ret, n.b.err -} - -func nextPartitionWithTrace(ctx context.Context, n nextPartition, tbl table.PhysicalTable) (Executor, error) { - if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { - span1 := span.Tracer().StartSpan(fmt.Sprintf("nextPartition %d", tbl.GetPhysicalID()), opentracing.ChildOf(span.Context())) - defer span1.Finish() - ctx = opentracing.ContextWithSpan(ctx, span1) - } - return n.nextPartition(ctx, tbl) -} - func updateExecutorTableID(ctx context.Context, exec *tipb.Executor, partitionID int64, recursive bool) error { var child *tipb.Executor switch exec.Tp { @@ -105,55 +57,3 @@ func updateExecutorTableID(ctx context.Context, exec *tipb.Executor, partitionID } return nil } - -// Open implements the Executor interface. -func (e *PartitionTableExecutor) Open(ctx context.Context) error { - e.cursor = 0 - e.curr = nil - return nil -} - -// Next implements the Executor interface. -func (e *PartitionTableExecutor) Next(ctx context.Context, chk *chunk.Chunk) error { - chk.Reset() - var err error - for e.cursor < len(e.partitions) { - if e.curr == nil { - n := e.nextPartition - e.curr, err = nextPartitionWithTrace(ctx, n, e.partitions[e.cursor]) - if err != nil { - return err - } - if err := e.curr.Open(ctx); err != nil { - return err - } - } - - err = Next(ctx, e.curr, chk) - if err != nil { - return err - } - - if chk.NumRows() > 0 { - break - } - - err = e.curr.Close() - if err != nil { - return err - } - e.curr = nil - e.cursor++ - } - return nil -} - -// Close implements the Executor interface. -func (e *PartitionTableExecutor) Close() error { - var err error - if e.curr != nil { - err = e.curr.Close() - e.curr = nil - } - return err -} From 9d01a36a6b7095cfbe2297ef77039641c0aa8fe9 Mon Sep 17 00:00:00 2001 From: Shirly Date: Thu, 6 May 2021 17:59:53 +0800 Subject: [PATCH 13/67] =?UTF-8?q?store/tikv:=20make=20tikv.ErrTiFlashServe?= =?UTF-8?q?rBusy=20as=20a=20normal=20error=20instead=20=E2=80=A6=20(#24392?= =?UTF-8?q?)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- store/driver/txn/error.go | 6 ++++++ store/tikv/error/errcode.go | 1 - store/tikv/error/error.go | 3 ++- 3 files changed, 8 insertions(+), 2 deletions(-) diff --git a/store/driver/txn/error.go b/store/driver/txn/error.go index 42b16e5930ab1..d0248c47760a8 100644 --- a/store/driver/txn/error.go +++ b/store/driver/txn/error.go @@ -49,6 +49,8 @@ var ( ErrResolveLockTimeout = dbterror.ClassTiKV.NewStd(errno.ErrResolveLockTimeout) // ErrTiKVServerBusy is the error when tikv server is busy. ErrTiKVServerBusy = dbterror.ClassTiKV.NewStd(errno.ErrTiKVServerBusy) + // ErrTiFlashServerBusy is the error that tiflash server is busy. + ErrTiFlashServerBusy = dbterror.ClassTiKV.NewStd(errno.ErrTiFlashServerBusy) // ErrPDServerTimeout is the error when pd server is timeout. ErrPDServerTimeout = dbterror.ClassTiKV.NewStd(errno.ErrPDServerTimeout) ) @@ -213,6 +215,10 @@ func ToTiDBErr(err error) error { return ErrTiKVServerBusy } + if errors.ErrorEqual(err, tikverr.ErrTiFlashServerBusy) { + return ErrTiFlashServerBusy + } + if e, ok := err.(*tikverr.ErrGCTooEarly); ok { return ErrGCTooEarly.GenWithStackByArgs(e.TxnStartTS, e.GCSafePoint) } diff --git a/store/tikv/error/errcode.go b/store/tikv/error/errcode.go index 6d18e35859a07..0b15b97edb8d7 100644 --- a/store/tikv/error/errcode.go +++ b/store/tikv/error/errcode.go @@ -30,5 +30,4 @@ const ( CodeTiKVStoreLimit = 9008 CodeTiFlashServerTimeout = 9012 - CodeTiFlashServerBusy = 9013 ) diff --git a/store/tikv/error/error.go b/store/tikv/error/error.go index cb6698f454f5c..734851865820c 100644 --- a/store/tikv/error/error.go +++ b/store/tikv/error/error.go @@ -44,6 +44,8 @@ var ( ErrResolveLockTimeout = errors.New("resolve lock timeout") // ErrTiKVServerBusy is the error when tikv server is busy. ErrTiKVServerBusy = errors.New("tikv server busy") + // ErrTiFlashServerBusy is the error that tiflash server is busy. + ErrTiFlashServerBusy = errors.New("tiflash server busy") ) // MismatchClusterID represents the message that the cluster ID of the PD client does not match the PD. @@ -53,7 +55,6 @@ const MismatchClusterID = "mismatch cluster id" var ( ErrTiFlashServerTimeout = dbterror.ClassTiKV.NewStd(CodeTiFlashServerTimeout) ErrRegionUnavailable = dbterror.ClassTiKV.NewStd(CodeRegionUnavailable) - ErrTiFlashServerBusy = dbterror.ClassTiKV.NewStd(CodeTiFlashServerBusy) ErrQueryInterrupted = dbterror.ClassTiKV.NewStd(CodeQueryInterrupted) ErrLockAcquireFailAndNoWaitSet = dbterror.ClassTiKV.NewStd(CodeLockAcquireFailAndNoWaitSet) ErrLockWaitTimeout = dbterror.ClassTiKV.NewStd(CodeLockWaitTimeout) From 13ff0726fda3db0ae3bde3b41d66d9b78b7a3912 Mon Sep 17 00:00:00 2001 From: Ping Yu Date: Thu, 6 May 2021 18:25:53 +0800 Subject: [PATCH 14/67] executor: support global kill (32 bits) (#20749) --- docs/design/2020-06-01-global-kill.md | 105 +++++++++++++++++++------- 1 file changed, 76 insertions(+), 29 deletions(-) diff --git a/docs/design/2020-06-01-global-kill.md b/docs/design/2020-06-01-global-kill.md index 61cc6b8504368..b1f92c40b7aaa 100644 --- a/docs/design/2020-06-01-global-kill.md +++ b/docs/design/2020-06-01-global-kill.md @@ -1,12 +1,12 @@ # Global Kill - Author(s): [pingyu](https://github.com/pingyu) (Ping Yu) -- Last updated: 2020-10-25 +- Last updated: 2021-05-05 - Discussion at: https://github.com/pingcap/tidb/issues/8854 ## Abstract -This document introduces the design of global connection id, and the global `KILL ` based on it. +This document introduces the design of global connection id, and the global `KILL ` based on it. ## Background @@ -17,57 +17,104 @@ Currently connection ids are local to TiDB instances, which means that a `KILL x To support "Global Kill", we need: 1. Global connection ids, which are unique among all TiDB instances. 2. Redirect `KILL x` to target TiDB instance, on which the connection `x` is running. -3. Support both 32 & 64 bits `connId`, to be compatible with legacy 32 bits clients. In this stage, we only design the 64 bits `connId`, and left a `markup` to distinguish between these two kinds. +3. Support both 32 & 64 bits `connID`. 32 bits `connID` is used on small clusters (number of TiDB instances less than 2048), to be fully compatible with all clients including legacy 32 bits ones, while 64 bits `connID` is used for big clusters. Bit 0 in `connID` is a markup to distinguish between these two kinds. ## Rationale -#### 1. Structure of `connId` -##### 64 bits version +#### 1. Structure of `connID` +##### 32 bits +``` + 31 21 20 1 0 + +--------+------------------+------+ + |serverID| local connID |markup| + | (11b) | (20b) | =0 | + +--------+------------------+------+ +``` + +##### 64 bits ``` 63 62 41 40 1 0 +--+---------------------+--------------------------------------+------+ - | | serverId | local connId |markup| + | | serverID | local connID |markup| |=0| (22b) | (40b) | =1 | +--+---------------------+--------------------------------------+------+ ``` -##### 32 bits version -(To be discussed in another RFC) -``` - 31 1 0 - +-----------------------------+------+ - | ??? |markup| - | ??? | =0 | - +-----------------------------+------+ -``` -#### 2. bit 63 -Bit 63 is always __ZERO__, making `connId` in range of non-negative int64, to be more friendly to exists codes, and some languages don't have primitive type `uint64`. +##### Determine 32 or 64 bits +The key factor is `serverID` (see `serverID` section for detail), which depends on number of TiDB instances in cluster. +- Choose 32 bits when number of TiDB instances __less than 2048__. Otherwise choose 64 bits. +- When 32 bits chosen, upgrade to 64 bits when: 1) Fail to acquire `serverID` _(because of occupied)_ continuously __more than 3 times__ _(which will happen when size of cluster is increasing rapidly)_; 2) All `local connID` in 32 bits `connID` are in used (see `local connID` section for detail). +- When 64 bits chosen, downgrade to 32 bits in a gradually manner when cluster scales down from big to small, as TiDB instances keep `serverID` until next restart or lost connection to PD. + +#### 2. Bit 63 +Bit 63 is always __ZERO__, making `connID` in range of non-negative int64, to be more friendly to exists codes, and some languages don't have primitive type `uint64`. -#### 3. markup +#### 3. Markup - `markup == 0` indicates that the `connID` is just 32 bits long effectively, and high 32 bits should be all zeros. Compatible with legacy 32 bits clients. - `markup == 1` indicates that the `connID` is 64 bits long. Incompatible with legacy 32 bits clients. - `markup == 1` while __high 32 bits are zeros__, indicates that 32 bits truncation happens. See `Compatibility` section. +#### 4. ServerID +- `serverID` is selected RANDOMLY from `serverIDs pool`_(see next)_ by each TiDB instance on startup, and the uniqueness is guaranteed by PD (etcd). `serverID` should be larger or equal to 1, to ensure that high 32 bits of `connID` is always non-zero, and make it possible to detect truncation. + +- `serverIDs pool` is: + - All __UNUSED__ `serverIDs` within [1, 2047] acquired from [`CLUSTER_INFO`](https://docs.pingcap.com/tidb/stable/information-schema-cluster-info#cluster_info) when 32 bits `connID` chosen. + - All `serverIDs` within [2048, 2^22-1] when 64 bits `connID` chosen. + +- On failure (e.g. fail connecting to PD, or all `serverID` are unavailable when 64 bits `connID` chosen), we block any new connection. +- `serverID` is kept by PD with a lease (defaults to 12 hours, long enough to avoid brutally killing any long-run SQL). If TiDB is disconnected to PD longer than half of the lease (defaults to 6 hours), all connections are killed, and no new connection is accepted, to avoid running with stale/incorrect `serverID`. On connection to PD restored, a new `serverID` is acquired before accepting new connection. +- On single TiDB instance without PD, a `serverID` of `1` is assigned. -#### 4. serverId -`serverId` is selected RANDOMLY by each TiDB instance on startup, and the uniqueness is guaranteed by PD(etcd). `serverId` should be larger or equal to 1, to ensure that high 32 bits of `connId` is always non-zero, and make it possible to detect truncation. +#### 5. Local connID +`local connID` is allocated by each TiDB instance on establishing connections: -On failure (e.g. fail connecting to PD, or all `serverId` are unavailable), we block any new connection. +- For 32 bits `connID`, `local connID` is possible to be integer-overflow and/or used up, especially on system being busy and/or with long running SQL. So we use a __lock-free queue__ to maintain available `local connID`, dequeue on client connecting, and enqueue on disconnecting. When `local connID` exhausted, upgrade to 64 bits. -`serverId` is kept by PD with a lease (defaults to 12 hours, long enough to avoid brutally killing any long-run SQL). If TiDB is disconnected to PD longer than half of the lease (defaults to 6 hours), all connections are killed, and no new connection is accepted, to avoid running with stale/incorrect `serverId`. On connection to PD restored, a new `serverId` is acquired before accepting new connection. +- For 64 bits `connID`, allocate `local connID` by __auto-increment__. Besides, flip to zero if integer-overflow, and check `local connID` existed or not by [Server.clients](https://github.com/pingcap/tidb/blob/7e1533392030514440d27ba98001c374cdf8808f/server/server.go#L122) for correctness with trivial cost, as the conflict is very unlikely to happen (It needs more than 3 years to use up 2^40 `local connID` in a 1w TPS instance). At last, return _"Too many connections"_ error if exhausted. -On single TiDB instance without PD, a `serverId` of `1` is assigned. +#### 6. Global kill +On processing `KILL x` command, first extract `serverID` from `x`. Then if `serverID` aims to a remote TiDB instance, get the address from [`CLUSTER_INFO`](https://docs.pingcap.com/tidb/stable/information-schema-cluster-info#cluster_info), and redirect the command to it by "Coprocessor API" provided by the remote TiDB, along with the original user authentication. -#### 5. local connId -`local connId` is allocated by each TiDB instance on establishing connections incrementally. +#### 7. Summary +| | 32 bits | 64 bits | +| ---- | ---- | ---- | +| ServerID pool size | 2^11 | 2^22 - 2^11 | +| ServerID allocation | Random of __Unused__ serverIDs acquired from PD within pool. Retry if unavailable. Upgrade to 64 bits while failed more than 3 times | Random of __All__ serverIDs within pool. Retry if unavailable | +| Local connID pool size | 2^20 | 2^40 | +| Local connID allocation | Using a queue to maintain and allocate available local connID. Upgrade to 64 bits while exhausted | Auto-increment within pool. Flip to zero when overflow. Return "Too many connections" if exhausted | -Integer overflow is ignored at this stage, as `local connId` should be long enough. -#### 6. global kill -On processing `KILL x` command, first extract `serverId` from `x`. Then if `serverId` aims to a remote TiDB instance, get the address from cluster info (see also [`CLUSTER_INFO`](https://docs.pingcap.com/tidb/stable/information-schema-cluster-info#cluster_info)), and redirect the command to it by "Coprocessor API" provided by the remote TiDB, along with the original user authentication. ## Compatibility -- Incompatible with legacy 32 bits clients. (According to some quick tests by now, MySQL client v8.0.19 supports `KILL` a connection with 64 bits `connId`, while `CTRL-C` does not, because it truncates the `connId` to 32 bits). A warning is set prompting that truncation happened, but user cannot see it, because `CTRL-C` is sent by a new connection in an instant. +- 32 bits `connID` is compatible to well-known clients. + +- 64 bits `connID` is __incompatible__ with legacy 32 bits clients. (According to some quick tests by now, MySQL client v8.0.19 supports `KILL` a connection with 64 bits `connID`, while `CTRL-C` does not, as it truncates the `connID` to 32 bits). A warning is set prompting that truncation happened, but user cannot see it, because `CTRL-C` is sent by a new connection in an instant. - [`KILL TIDB`](https://docs.pingcap.com/tidb/v4.0/sql-statement-kill) syntax and [`compatible-kill-query`](https://docs.pingcap.com/tidb/v4.0/tidb-configuration-file#compatible-kill-query) configuration item are deprecated. + +## Test Design + +### Prerequisite +Set `small_cluster_size_threshold` and `local_connid_pool_size` to small numbers (e.g. 4) by variable hacking, for easily switch between 32 and 64 bits `connID`. + +### Scenario A. 32 bits `connID` with small cluster +1. A TiDB without PD, killed by Ctrl+C, and killed by KILL. +2. One TiDB with PD, killed by Ctrl+C, and killed by KILL. +3. Multiple TiDB nodes, killed {local,remote} by {Ctrl-C,KILL}. + +### Scenario B. Upgrade from 32 to 64 bits `connID` +1. Upgrade caused by cluster scaled up from small to big. +2. Upgrade caused by `local connID` used up. + +### Scenario C. 64 bits `connID` with big cluster +1. Multiple TiDB nodes, killed {local,remote} by {Ctrl-C,KILL}. + +### Scenario D. Downgrade from 64 to 32 bits `connID` +1. Downgrade caused by cluster scaled down from big to small. + +### Scenario E. Fault tolerant while disconnected with PD +1. Existing connections are killed after PD lost connection for long time. +2. New connections are not accepted after PD lost connection for long time. +3. New connections are accepted after PD lost connection for long time and then recovered. +4. Connections can be killed after PD lost connection for long time and then recovered. From 6ef4815d283c46eec3e0c77c69288052034186b7 Mon Sep 17 00:00:00 2001 From: lysu Date: Thu, 6 May 2021 18:59:53 +0800 Subject: [PATCH 15/67] *: support clustered index table produce binlog (#24135) --- ddl/ddl_api.go | 5 ----- sessionctx/binloginfo/binloginfo_test.go | 19 ------------------- table/tables/tables.go | 6 +++--- 3 files changed, 3 insertions(+), 27 deletions(-) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 31de57a6e572e..138bb69756f52 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -1440,11 +1440,6 @@ func buildTableInfo( if isSingleIntPK { tbInfo.PKIsHandle = true } else { - hasBinlog := ctx.GetSessionVars().BinlogClient != nil - if hasBinlog { - msg := mysql.Message("Cannot create clustered index table when the binlog is ON", nil) - return nil, dbterror.ClassDDL.NewStdErr(errno.ErrUnsupportedDDLOperation, msg) - } tbInfo.IsCommonHandle = true tbInfo.CommonHandleVersion = 1 } diff --git a/sessionctx/binloginfo/binloginfo_test.go b/sessionctx/binloginfo/binloginfo_test.go index ca01fe0fc90e3..2dfca57d73f4c 100644 --- a/sessionctx/binloginfo/binloginfo_test.go +++ b/sessionctx/binloginfo/binloginfo_test.go @@ -269,25 +269,6 @@ func (s *testBinlogSuite) TestBinlog(c *C) { binlog.MutationType_Insert, }) - // Cannot create common clustered index table when binlog client exists. - errMsg := "[ddl:8200]Cannot create clustered index table when the binlog is ON" - tk.MustGetErrMsg("create table local_clustered_index (c1 varchar(255) primary key clustered);", errMsg) - // Create int clustered index table when binlog client exists. - tk.MustExec("create table local_clustered_index (c1 bigint primary key clustered);") - tk.MustQuery("select tidb_pk_type from information_schema.tables where table_name = 'local_clustered_index' and table_schema = 'test';"). - Check(testkit.Rows("CLUSTERED")) - tk.MustExec("drop table if exists local_clustered_index;") - // Test common clustered index tables will not write binlog. - tk.Se.GetSessionVars().BinlogClient = nil - tk.MustExec("create table local_clustered_index (c1 varchar(255) primary key clustered);") - tk.MustQuery("select tidb_pk_type from information_schema.tables where table_name = 'local_clustered_index' and table_schema = 'test';"). - Check(testkit.Rows("CLUSTERED")) - tk.Se.GetSessionVars().BinlogClient = s.client - // This statement should not write binlog. - tk.MustExec(`insert into local_clustered_index values ("aaaaaa")`) - prewriteVal = getLatestBinlogPrewriteValue(c, pump) - c.Assert(len(prewriteVal.Mutations), Equals, 0) - checkBinlogCount(c, pump) pump.mu.Lock() diff --git a/table/tables/tables.go b/table/tables/tables.go index d1ae7d804fb58..7da187c785b4c 100644 --- a/table/tables/tables.go +++ b/table/tables/tables.go @@ -409,7 +409,7 @@ func (t *TableCommon) UpdateRecord(ctx context.Context, sctx sessionctx.Context, } memBuffer.Release(sh) if shouldWriteBinlog(sctx, t.meta) { - if !t.meta.PKIsHandle { + if !t.meta.PKIsHandle && !t.meta.IsCommonHandle { binlogColIDs = append(binlogColIDs, model.ExtraHandleID) binlogOldRow = append(binlogOldRow, types.NewIntDatum(h.IntValue())) binlogNewRow = append(binlogNewRow, types.NewIntDatum(h.IntValue())) @@ -1029,7 +1029,7 @@ func (t *TableCommon) RemoveRecord(ctx sessionctx.Context, h kv.Handle, r []type colIDs = append(colIDs, col.ID) } var binlogRow []types.Datum - if !t.meta.PKIsHandle { + if !t.meta.PKIsHandle && !t.meta.IsCommonHandle { colIDs = append(colIDs, model.ExtraHandleID) binlogRow = make([]types.Datum, 0, len(r)+1) binlogRow = append(binlogRow, r...) @@ -1407,7 +1407,7 @@ func shouldWriteBinlog(ctx sessionctx.Context, tblInfo *model.TableInfo) bool { if ctx.GetSessionVars().BinlogClient == nil { return false } - return !ctx.GetSessionVars().InRestrictedSQL && !tblInfo.IsCommonHandle + return !ctx.GetSessionVars().InRestrictedSQL } func (t *TableCommon) getMutation(ctx sessionctx.Context) *binlog.TableMutation { From a57dc3ab654327ddfc23fae1ae36ccad24758beb Mon Sep 17 00:00:00 2001 From: ZhuoZhi <517770911@qq.com> Date: Thu, 6 May 2021 19:59:52 +0800 Subject: [PATCH 16/67] planner: prune partitions that will never be used (#24376) --- planner/core/partition_pruner_test.go | 1 + planner/core/rule_partition_processor.go | 26 ++++-- .../core/testdata/partition_pruner_in.json | 3 +- .../core/testdata/partition_pruner_out.json | 7 ++ util/math/math.go | 5 ++ util/ranger/testdata/ranger_suite_out.json | 82 ++++++++----------- 6 files changed, 70 insertions(+), 54 deletions(-) diff --git a/planner/core/partition_pruner_test.go b/planner/core/partition_pruner_test.go index df5eb0c47b739..dd614239de55d 100644 --- a/planner/core/partition_pruner_test.go +++ b/planner/core/partition_pruner_test.go @@ -67,6 +67,7 @@ func (s *testPartitionPruneSuit) TestHashPartitionPruner(c *C) { tk.MustExec("create table t6(a int, b int) partition by hash(a) partitions 3;") tk.MustExec("create table t7(a int, b int) partition by hash(a + b) partitions 10;") tk.MustExec("create table t8(a int, b int) partition by hash(a) partitions 6;") + tk.MustExec("create table t9(a bit(1) default null, b int(11) default null) partition by hash(a) partitions 3;") //issue #22619 var input []string var output []struct { diff --git a/planner/core/rule_partition_processor.go b/planner/core/rule_partition_processor.go index 46ae7282e346b..57858679b7795 100644 --- a/planner/core/rule_partition_processor.go +++ b/planner/core/rule_partition_processor.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/parser/ast" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" + "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/table" @@ -190,14 +191,27 @@ func (s *partitionProcessor) findUsedPartitions(ctx sessionctx.Context, tbl tabl } used = append(used, int(idx)) } - } else { - used = []int{FullRange} - break + continue + } + + // issue:#22619 + if col.RetType.Tp == mysql.TypeBit { + // maximum number of partitions is 8192 + if col.RetType.Flen > 0 && col.RetType.Flen < int(math.Log2(ddl.PartitionCountLimit)) { + // all possible hash values + maxUsedPartitions := 1 << col.RetType.Flen + if maxUsedPartitions < numPartitions { + for i := 0; i < maxUsedPartitions; i++ { + used = append(used, i) + } + continue + } + } } - } else { - used = []int{FullRange} - break } + + used = []int{FullRange} + break } } if len(partitionNames) > 0 && len(used) == 1 && used[0] == FullRange { diff --git a/planner/core/testdata/partition_pruner_in.json b/planner/core/testdata/partition_pruner_in.json index 376d60d83b5a1..9562379ac4574 100644 --- a/planner/core/testdata/partition_pruner_in.json +++ b/planner/core/testdata/partition_pruner_in.json @@ -28,7 +28,8 @@ "explain format = 'brief' select * from t8 where (a is null) or (a between 5 and 8)", "explain format = 'brief' select * from t8 where a between 5 and 12", "explain format = 'brief' select * from t8 where (a <= 10 and a >= 8) or (a <= 13 and a >= 11) or (a <= 16 and a >= 14)", - "explain format = 'brief' select * from t8 where a < 12 and a > 9" + "explain format = 'brief' select * from t8 where a < 12 and a > 9", + "explain format = 'brief' select * from t9" ] }, { diff --git a/planner/core/testdata/partition_pruner_out.json b/planner/core/testdata/partition_pruner_out.json index 31ca209396c87..bfea064683c39 100644 --- a/planner/core/testdata/partition_pruner_out.json +++ b/planner/core/testdata/partition_pruner_out.json @@ -218,6 +218,13 @@ "└─Selection 250.00 cop[tikv] gt(test_partition.t8.a, 9), lt(test_partition.t8.a, 12)", " └─TableFullScan 10000.00 cop[tikv] table:t8 keep order:false, stats:pseudo" ] + }, + { + "SQL": "explain format = 'brief' select * from t9", + "Result": [ + "TableReader 10000.00 root partition:p0,p1 data:TableFullScan", + "└─TableFullScan 10000.00 cop[tikv] table:t9 keep order:false, stats:pseudo" + ] } ] }, diff --git a/util/math/math.go b/util/math/math.go index 3a25178326261..b9d5bf3a94477 100644 --- a/util/math/math.go +++ b/util/math/math.go @@ -48,3 +48,8 @@ func StrLenOfInt64Fast(x int64) int { } return size + StrLenOfUint64Fast(uint64(Abs(x))) } + +// Log2 is used for export purposes +func Log2(x float64) float64 { + return math.Log2(x) +} diff --git a/util/ranger/testdata/ranger_suite_out.json b/util/ranger/testdata/ranger_suite_out.json index c652916186c16..f56a1567e6043 100644 --- a/util/ranger/testdata/ranger_suite_out.json +++ b/util/ranger/testdata/ranger_suite_out.json @@ -392,13 +392,11 @@ { "SQL": "select * from t;", "Plan": [ - "PartitionUnion_9 10004.00 root ", - "├─TableReader_11 1.00 root data:TableFullScan_10", - "│ └─TableFullScan_10 1.00 cop[tikv] table:t, partition:p0 keep order:false", - "├─TableReader_13 3.00 root data:TableFullScan_12", - "│ └─TableFullScan_12 3.00 cop[tikv] table:t, partition:p1 keep order:false", - "└─TableReader_15 10000.00 root data:TableFullScan_14", - " └─TableFullScan_14 10000.00 cop[tikv] table:t, partition:p2 keep order:false, stats:pseudo" + "PartitionUnion_8 4.00 root ", + "├─TableReader_10 1.00 root data:TableFullScan_9", + "│ └─TableFullScan_9 1.00 cop[tikv] table:t, partition:p0 keep order:false", + "└─TableReader_12 3.00 root data:TableFullScan_11", + " └─TableFullScan_11 3.00 cop[tikv] table:t, partition:p1 keep order:false" ], "Result": [ "\u0000 0", @@ -461,14 +459,12 @@ { "SQL": "select * from t where a < 1;", "Plan": [ - "Selection_10 8003.20 root lt(test.t.a, 1)", - "└─PartitionUnion_11 10004.00 root ", - " ├─TableReader_13 1.00 root data:TableFullScan_12", - " │ └─TableFullScan_12 1.00 cop[tikv] table:t, partition:p0 keep order:false", - " ├─TableReader_15 3.00 root data:TableFullScan_14", - " │ └─TableFullScan_14 3.00 cop[tikv] table:t, partition:p1 keep order:false", - " └─TableReader_17 10000.00 root data:TableFullScan_16", - " └─TableFullScan_16 10000.00 cop[tikv] table:t, partition:p2 keep order:false, stats:pseudo" + "Selection_9 3.20 root lt(test.t.a, 1)", + "└─PartitionUnion_10 4.00 root ", + " ├─TableReader_12 1.00 root data:TableFullScan_11", + " │ └─TableFullScan_11 1.00 cop[tikv] table:t, partition:p0 keep order:false", + " └─TableReader_14 3.00 root data:TableFullScan_13", + " └─TableFullScan_13 3.00 cop[tikv] table:t, partition:p1 keep order:false" ], "Result": [ "\u0000 0" @@ -477,14 +473,12 @@ { "SQL": "select * from t where a < 3;", "Plan": [ - "Selection_10 8003.20 root lt(test.t.a, 3)", - "└─PartitionUnion_11 10004.00 root ", - " ├─TableReader_13 1.00 root data:TableFullScan_12", - " │ └─TableFullScan_12 1.00 cop[tikv] table:t, partition:p0 keep order:false", - " ├─TableReader_15 3.00 root data:TableFullScan_14", - " │ └─TableFullScan_14 3.00 cop[tikv] table:t, partition:p1 keep order:false", - " └─TableReader_17 10000.00 root data:TableFullScan_16", - " └─TableFullScan_16 10000.00 cop[tikv] table:t, partition:p2 keep order:false, stats:pseudo" + "Selection_9 3.20 root lt(test.t.a, 3)", + "└─PartitionUnion_10 4.00 root ", + " ├─TableReader_12 1.00 root data:TableFullScan_11", + " │ └─TableFullScan_11 1.00 cop[tikv] table:t, partition:p0 keep order:false", + " └─TableReader_14 3.00 root data:TableFullScan_13", + " └─TableFullScan_13 3.00 cop[tikv] table:t, partition:p1 keep order:false" ], "Result": [ "\u0000 0", @@ -504,14 +498,12 @@ { "SQL": "select * from t where a > 0;", "Plan": [ - "Selection_10 8003.20 root gt(test.t.a, 0)", - "└─PartitionUnion_11 10004.00 root ", - " ├─TableReader_13 1.00 root data:TableFullScan_12", - " │ └─TableFullScan_12 1.00 cop[tikv] table:t, partition:p0 keep order:false", - " ├─TableReader_15 3.00 root data:TableFullScan_14", - " │ └─TableFullScan_14 3.00 cop[tikv] table:t, partition:p1 keep order:false", - " └─TableReader_17 10000.00 root data:TableFullScan_16", - " └─TableFullScan_16 10000.00 cop[tikv] table:t, partition:p2 keep order:false, stats:pseudo" + "Selection_9 3.20 root gt(test.t.a, 0)", + "└─PartitionUnion_10 4.00 root ", + " ├─TableReader_12 1.00 root data:TableFullScan_11", + " │ └─TableFullScan_11 1.00 cop[tikv] table:t, partition:p0 keep order:false", + " └─TableReader_14 3.00 root data:TableFullScan_13", + " └─TableFullScan_13 3.00 cop[tikv] table:t, partition:p1 keep order:false" ], "Result": [ "\u0001 -1", @@ -522,14 +514,12 @@ { "SQL": "select * from t where a > -1;", "Plan": [ - "Selection_10 8003.20 root gt(test.t.a, -1)", - "└─PartitionUnion_11 10004.00 root ", - " ├─TableReader_13 1.00 root data:TableFullScan_12", - " │ └─TableFullScan_12 1.00 cop[tikv] table:t, partition:p0 keep order:false", - " ├─TableReader_15 3.00 root data:TableFullScan_14", - " │ └─TableFullScan_14 3.00 cop[tikv] table:t, partition:p1 keep order:false", - " └─TableReader_17 10000.00 root data:TableFullScan_16", - " └─TableFullScan_16 10000.00 cop[tikv] table:t, partition:p2 keep order:false, stats:pseudo" + "Selection_9 3.20 root gt(test.t.a, -1)", + "└─PartitionUnion_10 4.00 root ", + " ├─TableReader_12 1.00 root data:TableFullScan_11", + " │ └─TableFullScan_11 1.00 cop[tikv] table:t, partition:p0 keep order:false", + " └─TableReader_14 3.00 root data:TableFullScan_13", + " └─TableFullScan_13 3.00 cop[tikv] table:t, partition:p1 keep order:false" ], "Result": [ "\u0000 0", @@ -541,14 +531,12 @@ { "SQL": "select * from t where a > 3;", "Plan": [ - "Selection_10 8003.20 root gt(test.t.a, 3)", - "└─PartitionUnion_11 10004.00 root ", - " ├─TableReader_13 1.00 root data:TableFullScan_12", - " │ └─TableFullScan_12 1.00 cop[tikv] table:t, partition:p0 keep order:false", - " ├─TableReader_15 3.00 root data:TableFullScan_14", - " │ └─TableFullScan_14 3.00 cop[tikv] table:t, partition:p1 keep order:false", - " └─TableReader_17 10000.00 root data:TableFullScan_16", - " └─TableFullScan_16 10000.00 cop[tikv] table:t, partition:p2 keep order:false, stats:pseudo" + "Selection_9 3.20 root gt(test.t.a, 3)", + "└─PartitionUnion_10 4.00 root ", + " ├─TableReader_12 1.00 root data:TableFullScan_11", + " │ └─TableFullScan_11 1.00 cop[tikv] table:t, partition:p0 keep order:false", + " └─TableReader_14 3.00 root data:TableFullScan_13", + " └─TableFullScan_13 3.00 cop[tikv] table:t, partition:p1 keep order:false" ], "Result": null } From bc4a38c9b94a7a270bbffac3e999aca1b90426b5 Mon Sep 17 00:00:00 2001 From: Howie Date: Thu, 6 May 2021 20:27:52 +0800 Subject: [PATCH 17/67] ddl: add partition compatibility for temporary table (#24406) --- ddl/db_partition_test.go | 29 +++++++++++++++++++++++++++++ ddl/error.go | 3 +++ ddl/table.go | 5 +++++ errors.toml | 5 +++++ 4 files changed, 42 insertions(+) diff --git a/ddl/db_partition_test.go b/ddl/db_partition_test.go index 59bd721536d59..b52644c151fd4 100644 --- a/ddl/db_partition_test.go +++ b/ddl/db_partition_test.go @@ -33,6 +33,7 @@ import ( "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/ddl/testutil" "github.com/pingcap/tidb/domain" + "github.com/pingcap/tidb/errno" tmysql "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" @@ -3385,3 +3386,31 @@ func (s *testIntegrationSuite7) TestPartitionListWithNewCollation(c *C) { str := tk.MustQuery(`desc select * from t11 where a = 'b';`).Rows()[0][3].(string) c.Assert(strings.Contains(str, "partition:p0"), IsTrue) } + +func (s *testIntegrationSuite7) TestAddTableWithPartition(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("use test;") + tk.MustExec("drop table if exists global_partition_table;") + tk.MustGetErrCode("create global temporary table global_partition_table (a int, b int) partition by hash(a) partitions 3 ON COMMIT DELETE ROWS;", errno.ErrPartitionNoTemporary) + tk.MustExec("drop table if exists global_partition_table;") + tk.MustExec("drop table if exists partition_table;") + _, err := tk.Exec("create table partition_table (a int, b int) partition by hash(a) partitions 3;") + c.Assert(err, IsNil) + tk.MustExec("drop table if exists partition_table;") + tk.MustExec("drop table if exists partition_range_table;") + tk.MustGetErrCode(`create global temporary table partition_range_table (c1 smallint(6) not null, c2 char(5) default null) partition by range ( c1 ) ( + partition p0 values less than (10), + partition p1 values less than (20), + partition p2 values less than (30), + partition p3 values less than (MAXVALUE) + ) ON COMMIT DELETE ROWS;`, errno.ErrPartitionNoTemporary) + tk.MustExec("drop table if exists partition_range_table;") + tk.MustExec("drop table if exists partition_list_table;") + tk.MustExec("set @@session.tidb_enable_list_partition = ON") + tk.MustGetErrCode(`create global temporary table partition_list_table (id int) partition by list (id) ( + partition p0 values in (1,2), + partition p1 values in (3,4), + partition p3 values in (5,null) + ) ON COMMIT DELETE ROWS;`, errno.ErrPartitionNoTemporary) + tk.MustExec("drop table if exists partition_list_table;") +} diff --git a/ddl/error.go b/ddl/error.go index 852d2218c6275..463c9c405a19e 100644 --- a/ddl/error.go +++ b/ddl/error.go @@ -274,4 +274,7 @@ var ( ErrUnknownEngine = dbterror.ClassDDL.NewStd(mysql.ErrUnknownStorageEngine) errExchangePartitionDisabled = dbterror.ClassDDL.NewStdErr(mysql.ErrUnsupportedDDLOperation, parser_mysql.Message("Exchange Partition is disabled, please set 'tidb_enable_exchange_partition' if you need to need to enable it", nil)) + + // ErrPartitionNoTemporary returns when partition at temporary mode + ErrPartitionNoTemporary = dbterror.ClassDDL.NewStd(mysql.ErrPartitionNoTemporary) ) diff --git a/ddl/table.go b/ddl/table.go index 668de3ac41c05..acd209a7bb0da 100644 --- a/ddl/table.go +++ b/ddl/table.go @@ -56,6 +56,11 @@ func onCreateTable(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error) job.State = model.JobStateCancelled return ver, errors.Trace(err) } + if tbInfo.Partition != nil && (tbInfo.TempTableType == model.TempTableGlobal || tbInfo.TempTableType == model.TempTableLocal) { + // unsupported ddl, cancel this job. + job.State = model.JobStateCancelled + return ver, errors.Trace(ErrPartitionNoTemporary) + } tbInfo.State = model.StateNone err := checkTableNotExists(d, t, schemaID, tbInfo.Name.L) diff --git a/errors.toml b/errors.toml index f09e33605293d..458af951629d8 100644 --- a/errors.toml +++ b/errors.toml @@ -261,6 +261,11 @@ error = ''' Duplicate partition name %-.192s ''' +["ddl:1562"] +error = ''' +Cannot create temporary table with partitions +''' + ["ddl:1563"] error = ''' Partition constant is out of partition function domain From 5bd601bdaea8f9ae1cd0a2f8fb477582cf93e3aa Mon Sep 17 00:00:00 2001 From: Lynn Date: Thu, 6 May 2021 21:01:52 +0800 Subject: [PATCH 18/67] ddl: disallow changing the default value of the primary key column to null (#24417) --- ddl/column_type_change_test.go | 14 +++++++++ ddl/ddl_api.go | 53 +++++++++++++++------------------- 2 files changed, 37 insertions(+), 30 deletions(-) diff --git a/ddl/column_type_change_test.go b/ddl/column_type_change_test.go index 68ee059f47305..16c0c6f38e308 100644 --- a/ddl/column_type_change_test.go +++ b/ddl/column_type_change_test.go @@ -1702,6 +1702,20 @@ func (s *testColumnTypeChangeSuite) TestChangingAttributeOfColumnWithFK(c *C) { tk.MustExec("drop table if exists orders, users") } +func (s *testColumnTypeChangeSuite) TestAlterPrimaryKeyToNull(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.Se.GetSessionVars().EnableChangeColumnType = true + + tk.MustExec("drop table if exists t, t1") + tk.MustExec("create table t(a int not null, b int not null, primary key(a, b));") + tk.MustGetErrCode("alter table t modify a bigint null;", mysql.ErrPrimaryCantHaveNull) + tk.MustGetErrCode("alter table t change column a a bigint null;", mysql.ErrPrimaryCantHaveNull) + tk.MustExec("create table t1(a int not null, b int not null, primary key(a));") + tk.MustGetErrCode("alter table t modify a bigint null;", mysql.ErrPrimaryCantHaveNull) + tk.MustGetErrCode("alter table t change column a a bigint null;", mysql.ErrPrimaryCantHaveNull) +} + // Close issue #23202 func (s *testColumnTypeChangeSuite) TestDDLExitWhenCancelMeetPanic(c *C) { tk := testkit.NewTestKit(c, s.store) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 138bb69756f52..0e1213e59c797 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -690,24 +690,7 @@ func columnDefToCol(ctx sessionctx.Context, offset int, colDef *ast.ColumnDef, o } } - processDefaultValue(col, hasDefaultValue, setOnUpdateNow) - - processColumnFlags(col) - - err = checkPriKeyConstraint(col, hasDefaultValue, hasNullFlag, outPriKeyConstraint) - if err != nil { - return nil, nil, errors.Trace(err) - } - err = checkColumnValueConstraint(col, col.Collate) - if err != nil { - return nil, nil, errors.Trace(err) - } - err = checkDefaultValue(ctx, col, hasDefaultValue) - if err != nil { - return nil, nil, errors.Trace(err) - } - err = checkColumnFieldLength(col) - if err != nil { + if err = processAndCheckDefaultValueAndColumn(ctx, col, outPriKeyConstraint, hasDefaultValue, setOnUpdateNow, hasNullFlag); err != nil { return nil, nil, errors.Trace(err) } return col, constraints, nil @@ -3675,6 +3658,7 @@ func processColumnOptions(ctx sessionctx.Context, col *table.Column, options []* var hasDefaultValue, setOnUpdateNow bool var err error + var hasNullFlag bool for _, opt := range options { switch opt.Tp { case ast.ColumnOptionDefaultValue: @@ -3690,6 +3674,7 @@ func processColumnOptions(ctx sessionctx.Context, col *table.Column, options []* case ast.ColumnOptionNotNull: col.Flag |= mysql.NotNullFlag case ast.ColumnOptionNull: + hasNullFlag = true col.Flag &= ^mysql.NotNullFlag case ast.ColumnOptionAutoIncrement: col.Flag |= mysql.AutoIncrementFlag @@ -3734,14 +3719,30 @@ func processColumnOptions(ctx sessionctx.Context, col *table.Column, options []* } } - processDefaultValue(col, hasDefaultValue, setOnUpdateNow) + if err = processAndCheckDefaultValueAndColumn(ctx, col, nil, hasDefaultValue, setOnUpdateNow, hasNullFlag); err != nil { + return errors.Trace(err) + } + return nil +} + +func processAndCheckDefaultValueAndColumn(ctx sessionctx.Context, col *table.Column, outPriKeyConstraint *ast.Constraint, hasDefaultValue, setOnUpdateNow, hasNullFlag bool) error { + processDefaultValue(col, hasDefaultValue, setOnUpdateNow) processColumnFlags(col) - if hasDefaultValue { - return errors.Trace(checkDefaultValue(ctx, col, true)) + err := checkPriKeyConstraint(col, hasDefaultValue, hasNullFlag, outPriKeyConstraint) + if err != nil { + return errors.Trace(err) + } + if err = checkColumnValueConstraint(col, col.Collate); err != nil { + return errors.Trace(err) + } + if err = checkDefaultValue(ctx, col, hasDefaultValue); err != nil { + return errors.Trace(err) + } + if err = checkColumnFieldLength(col); err != nil { + return errors.Trace(err) } - return nil } @@ -3854,10 +3855,6 @@ func (d *ddl) getModifiableColumnJob(ctx sessionctx.Context, ident ast.Ident, or return nil, errors.Trace(err) } - if err = checkColumnValueConstraint(newCol, newCol.Collate); err != nil { - return nil, errors.Trace(err) - } - if err = checkModifyTypes(ctx, &col.FieldType, &newCol.FieldType, isColumnWithIndex(col.Name.L, t.Meta().Indices)); err != nil { if strings.Contains(err.Error(), "Unsupported modifying collation") { colErrMsg := "Unsupported modifying collation of column '%s' from '%s' to '%s' when index is defined on it." @@ -3894,10 +3891,6 @@ func (d *ddl) getModifiableColumnJob(ctx sessionctx.Context, ident ast.Ident, or modifyColumnTp = mysql.TypeNull } - if err = checkColumnFieldLength(newCol); err != nil { - return nil, err - } - if err = checkColumnWithIndexConstraint(t.Meta(), col.ColumnInfo, newCol.ColumnInfo); err != nil { return nil, err } From a6ec081b82a5357a9688a98d4d3cca35c87eab75 Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Thu, 6 May 2021 21:15:52 +0800 Subject: [PATCH 19/67] planner: fix column pruning bug for Apply and Join (#24369) --- planner/core/integration_test.go | 29 ++++++++++ planner/core/rule_column_pruning.go | 26 +++++++++ .../testdata/integration_serial_suite_in.json | 6 +++ .../integration_serial_suite_out.json | 54 ++++++++++++++----- 4 files changed, 102 insertions(+), 13 deletions(-) diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index de79bb35b62e2..0a6a560802240 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -3114,6 +3114,35 @@ func (s *testIntegrationSuite) TestReorderSimplifiedOuterJoins(c *C) { } } +// Apply operator may got panic because empty Projection is eliminated. +func (s *testIntegrationSerialSuite) TestIssue23887(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t(a int, b int);") + tk.MustExec("insert into t values(1, 2), (3, 4);") + var input []string + var output []struct { + SQL string + Plan []string + Res []string + } + s.testData.GetTestCases(c, &input, &output) + for i, tt := range input { + s.testData.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + tt).Rows()) + output[i].Res = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Sort().Rows()) + }) + tk.MustQuery("explain format = 'brief' " + tt).Check(testkit.Rows(output[i].Plan...)) + tk.MustQuery(tt).Sort().Check(testkit.Rows(output[i].Res...)) + } + + tk.MustExec("drop table if exists t1;") + tk.MustExec("create table t1 (c1 int primary key, c2 int, c3 int, index c2 (c2));") + tk.MustQuery("select count(1) from (select count(1) from (select * from t1 where c3 = 100) k) k2;").Check(testkit.Rows("1")) +} + func (s *testIntegrationSerialSuite) TestDeleteStmt(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") diff --git a/planner/core/rule_column_pruning.go b/planner/core/rule_column_pruning.go index d070d793e1f8e..4b31853c138d0 100644 --- a/planner/core/rule_column_pruning.go +++ b/planner/core/rule_column_pruning.go @@ -314,11 +314,13 @@ func (p *LogicalJoin) PruneColumns(parentUsedCols []*expression.Column) error { if err != nil { return err } + addConstOneForEmptyProjection(p.children[0]) err = p.children[1].PruneColumns(rightCols) if err != nil { return err } + addConstOneForEmptyProjection(p.children[1]) p.mergeSchema() if p.JoinType == LeftOuterSemiJoin || p.JoinType == AntiLeftOuterSemiJoin { @@ -337,6 +339,7 @@ func (la *LogicalApply) PruneColumns(parentUsedCols []*expression.Column) error if err != nil { return err } + addConstOneForEmptyProjection(la.children[1]) la.CorCols = extractCorColumnsBySchema4LogicalPlan(la.children[1], la.children[0].Schema()) for _, col := range la.CorCols { @@ -347,6 +350,7 @@ func (la *LogicalApply) PruneColumns(parentUsedCols []*expression.Column) error if err != nil { return err } + addConstOneForEmptyProjection(la.children[0]) la.mergeSchema() return nil @@ -431,3 +435,25 @@ func (p *LogicalLimit) PruneColumns(parentUsedCols []*expression.Column) error { func (*columnPruner) name() string { return "column_prune" } + +// By add const one, we can avoid empty Projection is eliminated. +// Because in some cases, Projectoin cannot be eliminated even its output is empty. +func addConstOneForEmptyProjection(p LogicalPlan) { + proj, ok := p.(*LogicalProjection) + if !ok { + return + } + if proj.Schema().Len() != 0 { + return + } + + constOne := expression.NewOne() + proj.schema.Append(&expression.Column{ + UniqueID: proj.ctx.GetSessionVars().AllocPlanColumnID(), + RetType: constOne.GetType(), + }) + proj.Exprs = append(proj.Exprs, &expression.Constant{ + Value: constOne.Value, + RetType: constOne.GetType(), + }) +} diff --git a/planner/core/testdata/integration_serial_suite_in.json b/planner/core/testdata/integration_serial_suite_in.json index 81dfccf978aaa..9f9e6e31c3253 100644 --- a/planner/core/testdata/integration_serial_suite_in.json +++ b/planner/core/testdata/integration_serial_suite_in.json @@ -293,5 +293,11 @@ "desc format = 'brief' select * from t where a = 1 and b > 2 and b < 10 and d = 10 order by b,c limit 10", "desc format = 'brief' select * from t where a = 1 and b > 2 and b < 10 and d = 10 order by b desc, c desc limit 10" ] + }, + { + "name": "TestIssue23887", + "cases": [ + "select (2) in (select b from t) from (select t.a < (select t.a from t t1 limit 1) from t) t" + ] } ] diff --git a/planner/core/testdata/integration_serial_suite_out.json b/planner/core/testdata/integration_serial_suite_out.json index fcd6bfd7dee44..e05c7ef22e4cb 100644 --- a/planner/core/testdata/integration_serial_suite_out.json +++ b/planner/core/testdata/integration_serial_suite_out.json @@ -185,19 +185,21 @@ { "SQL": "explain format = 'verbose' select (2) in (select count(*) from t1) from (select t.b < (select t.b from t2 limit 1 ) from t3 t) t", "Plan": [ - "HashJoin_19 3.00 113.61 root CARTESIAN left outer semi join", - "├─Selection_38(Build) 0.80 11.18 root eq(2, Column#18)", - "│ └─StreamAgg_59 1.00 69.50 root funcs:count(Column#27)->Column#18", - "│ └─TableReader_60 1.00 5.17 root data:StreamAgg_43", - "│ └─StreamAgg_43 1.00 8.18 batchCop[tiflash] funcs:count(1)->Column#27", - "│ └─TableFullScan_58 3.00 60.50 batchCop[tiflash] table:t1 keep order:false", - "└─Apply_21(Probe) 3.00 82.03 root CARTESIAN left outer join", - " ├─TableReader_23(Build) 3.00 10.16 root data:TableFullScan_22", - " │ └─TableFullScan_22 3.00 128.00 cop[tikv] table:t keep order:false", - " └─Limit_27(Probe) 1.00 5.36 root offset:0, count:1", - " └─TableReader_33 1.00 5.36 root data:Limit_32", - " └─Limit_32 1.00 56.00 cop[tikv] offset:0, count:1", - " └─TableFullScan_30 1.00 56.00 cop[tikv] table:t2 keep order:false" + "HashJoin_19 3.00 133.41 root CARTESIAN left outer semi join", + "├─Selection_39(Build) 0.80 11.18 root eq(2, Column#18)", + "│ └─StreamAgg_60 1.00 69.50 root funcs:count(Column#32)->Column#18", + "│ └─TableReader_61 1.00 5.17 root data:StreamAgg_44", + "│ └─StreamAgg_44 1.00 8.18 batchCop[tiflash] funcs:count(1)->Column#32", + "│ └─TableFullScan_59 3.00 60.50 batchCop[tiflash] table:t1 keep order:false", + "└─Projection_20(Probe) 3.00 101.83 root 1->Column#28", + " └─Apply_22 3.00 82.03 root CARTESIAN left outer join", + " ├─TableReader_24(Build) 3.00 10.16 root data:TableFullScan_23", + " │ └─TableFullScan_23 3.00 128.00 cop[tikv] table:t keep order:false", + " └─Projection_27(Probe) 1.00 23.96 root 1->Column#26", + " └─Limit_28 1.00 5.36 root offset:0, count:1", + " └─TableReader_34 1.00 5.36 root data:Limit_33", + " └─Limit_33 1.00 56.00 cop[tikv] offset:0, count:1", + " └─TableFullScan_31 1.00 56.00 cop[tikv] table:t2 keep order:false" ] }, { @@ -2552,5 +2554,31 @@ ] } ] + }, + { + "Name": "TestIssue23887", + "Cases": [ + { + "SQL": "select (2) in (select b from t) from (select t.a < (select t.a from t t1 limit 1) from t) t", + "Plan": [ + "HashJoin 10000.00 root CARTESIAN left outer semi join, other cond:eq(2, test.t.b)", + "├─TableReader(Build) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + "└─Projection(Probe) 10000.00 root 1->Column#27", + " └─Apply 10000.00 root CARTESIAN left outer join", + " ├─TableReader(Build) 10000.00 root data:TableFullScan", + " │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + " └─Projection(Probe) 1.00 root 1->Column#25", + " └─Limit 1.00 root offset:0, count:1", + " └─TableReader 1.00 root data:Limit", + " └─Limit 1.00 cop[tikv] offset:0, count:1", + " └─TableFullScan 1.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Res": [ + "1", + "1" + ] + } + ] } ] From 289dcfefdd159345972f71cf42c9d6e594255290 Mon Sep 17 00:00:00 2001 From: Shenghui Wu <793703860@qq.com> Date: Thu, 6 May 2021 23:05:52 +0800 Subject: [PATCH 20/67] util: fix enum index range for in/not in clause. (#24431) --- expression/integration_test.go | 17 +++++++++++++++++ util/ranger/points.go | 18 ++++++++++++++++++ 2 files changed, 35 insertions(+) diff --git a/expression/integration_test.go b/expression/integration_test.go index 587b8903fb075..76ba37b49d4bd 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -9318,4 +9318,21 @@ func (s *testIntegrationSuite) TestEnumIndex(c *C) { result := tk.MustQuery("select * from t where " + cond).Sort().Rows() tk.MustQuery("select * from tidx where " + cond).Sort().Check(result) } + + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(e enum('d','c','b','a'), a int, index idx(e));") + tk.MustExec("insert into t values(1,1),(2,2),(3,3),(4,4);") + tk.MustQuery("select /*+ use_index(t, idx) */ * from t where e not in ('a','d') and a = 2;").Check( + testkit.Rows("c 2")) + + // issue 24419 + tk.MustExec("use test") + tk.MustExec("drop table if exists t02") + tk.MustExec("CREATE TABLE `t02` ( `COL1` enum('^YSQT0]V@9TFN>^WB6G?NG@S8>VYOM;BSC@64=ZISGS?O[JDFBI5M]QXJYQNSKU>NGAWLXS26LMTZ2YNN`XKIUGKY0IHDWV>E[BJJCABOKH1M^CB5E@DLS7Q88PWZTEAY]1ZQMN5NX[IFIYA983K:E4N77@FINM5HVGQCUCVNF5WLOOOEORAM=_JLMVFURMUASTVDBE','NL3V:J9LM4U5KUCVR;P','M5=T5FLQEZMPZAXH]4G:TSYYYVQ7O@4S6C3N8WPFKSP;SRD6VW@94BBH8XCT','P]I52Y46F?@RMOOF6;FWDTO`7FIT]R:]ELHD[CNLDSHC7FPBYOOJXLZSBV^5C^AAF6J5BCKE4V9==@H=4C]GMZXPNM','ECIQWH>?MK=ARGI0WVJNIBZFCFVJHFIUYJ:2?2WWZBNBWTPFNQPLLBFP9R_','E<_Y9OT@SOPYR72VIJVMBWIVPF@TTBZ@8ZPBZL=LXZF`WM4V2?K>AT','PZ@PR6XN28JL`B','ZOHBSCRMZPOI`IVTSEZAIDAF7DS@1TT20AP9','QLDIOY[Y:JZR@OL__I^@FBO=O_?WOOR:2BE:QJC','BI^TGJ_NEEXYKV1POHTOJQPGCPVR=TYZMGWABUQR07J8U::W4','N`ZN4P@9T[JW;FR6=FA4WP@APNPG[XQVIK4]F]2>EC>JEIOXC``;;?OHP') DEFAULT NULL, `COL2` tinyint DEFAULT NULL, `COL3` time DEFAULT NULL, KEY `U_M_COL4` (`COL1`,`COL2`), KEY `U_M_COL5` (`COL3`,`COL2`)) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin;") + tk.MustExec("insert into t02(col1, col2) values ('OFJHCEKCQGT:MXI7P3[YO4N0DF=2XJWJ4Z9Z;HQ8TMUTZV8YLQAHWJ4BDZHR3A', 39), ('OFJHCEKCQGT:MXI7P3[YO4N0DF=2XJWJ4Z9Z;HQ8TMUTZV8YLQAHWJ4BDZHR3A', 51), ('OFJHCEKCQGT:MXI7P3[YO4N0DF=2XJWJ4Z9Z;HQ8TMUTZV8YLQAHWJ4BDZHR3A', 55), ('OFJHCEKCQGT:MXI7P3[YO4N0DF=2XJWJ4Z9Z;HQ8TMUTZV8YLQAHWJ4BDZHR3A', -30), ('ZOHBSCRMZPOI`IVTSEZAIDAF7DS@1TT20AP9', -30);") + tk.MustQuery("select * from t02 where col1 not in (\"W1Rgd74pbJaGX47h1MPjpr0XSKJNCnwEleJ50Vbpl9EmbHJX6D6BXYKT2UAbl1uDw3ZGeYykhzG6Gld0wKdOiT4Gv5j9upHI0Q7vrXij4N9WNFJvB\", \"N`ZN4P@9T[JW;FR6=FA4WP@APNPG[XQVIK4]F]2>EC>JEIOXC``;;?OHP\") and col2 = -30;").Check( + testkit.Rows( + "OFJHCEKCQGT:MXI7P3[YO4N0DF=2XJWJ4Z9Z;HQ8TMUTZV8YLQAHWJ4BDZHR3A -30 ", + "ZOHBSCRMZPOI`IVTSEZAIDAF7DS@1TT20AP9 -30 ")) } diff --git a/util/ranger/points.go b/util/ranger/points.go index 19d7c18d9ecf7..d98b548dcbb7a 100644 --- a/util/ranger/points.go +++ b/util/ranger/points.go @@ -102,6 +102,9 @@ func (r *pointSorter) Less(i, j int) bool { } func rangePointLess(sc *stmtctx.StatementContext, a, b *point) (bool, error) { + if a.value.Kind() == types.KindMysqlEnum && b.value.Kind() == types.KindMysqlEnum { + return rangePointEnumLess(sc, a, b) + } cmp, err := a.value.CompareDatum(sc, &b.value) if cmp != 0 { return cmp < 0, nil @@ -109,6 +112,14 @@ func rangePointLess(sc *stmtctx.StatementContext, a, b *point) (bool, error) { return rangePointEqualValueLess(a, b), errors.Trace(err) } +func rangePointEnumLess(sc *stmtctx.StatementContext, a, b *point) (bool, error) { + cmp := types.CompareInt64(a.value.GetInt64(), b.value.GetInt64()) + if cmp != 0 { + return cmp < 0, nil + } + return rangePointEqualValueLess(a, b), nil +} + func rangePointEqualValueLess(a, b *point) bool { if a.start && b.start { return !a.excl && b.excl @@ -552,6 +563,13 @@ func (r *builder) buildFromIn(expr *expression.ScalarFunction) ([]*point, bool) if dt.Kind() == types.KindString || dt.Kind() == types.KindBinaryLiteral { dt.SetString(dt.GetString(), colCollate) } + if expr.GetArgs()[0].GetType().Tp == mysql.TypeEnum { + dt, err = dt.ConvertTo(r.sc, expr.GetArgs()[0].GetType()) + if err != nil { + // in (..., an impossible value (not valid enum), ...), the range is empty, so skip it. + continue + } + } if expr.GetArgs()[0].GetType().Tp == mysql.TypeYear { dt, err = dt.ConvertToMysqlYear(r.sc, expr.GetArgs()[0].GetType()) if err != nil { From ff689c08aa1ac3d06038be2332be548889286b98 Mon Sep 17 00:00:00 2001 From: Shirly Date: Fri, 7 May 2021 10:47:52 +0800 Subject: [PATCH 21/67] store/tikv: make tikv.ErrRegionUnavailable as a normal error (#24390) --- store/copr/coprocessor.go | 21 ++++++++++++--------- store/driver/txn/error.go | 6 ++++++ store/tikv/error/errcode.go | 2 -- store/tikv/error/error.go | 3 ++- 4 files changed, 20 insertions(+), 12 deletions(-) diff --git a/store/copr/coprocessor.go b/store/copr/coprocessor.go index fe255ffae05c1..d8d0bb1034e4f 100644 --- a/store/copr/coprocessor.go +++ b/store/copr/coprocessor.go @@ -35,6 +35,7 @@ import ( "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/kv" tidbmetrics "github.com/pingcap/tidb/metrics" + txndriver "github.com/pingcap/tidb/store/driver/txn" "github.com/pingcap/tidb/store/tikv" tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/store/tikv/logutil" @@ -273,7 +274,7 @@ type copIteratorWorker struct { respChan chan<- *copResponse finishCh <-chan struct{} vars *tikv.Variables - *tikv.ClientHelper + kvclient *tikv.ClientHelper memTracker *memory.Tracker @@ -402,7 +403,7 @@ func (it *copIterator) open(ctx context.Context, enabledRateLimitAction bool) { respChan: it.respChan, finishCh: it.finishCh, vars: it.vars, - ClientHelper: tikv.NewClientHelper(it.store.store, it.resolvedLocks), + kvclient: tikv.NewClientHelper(it.store.store, it.resolvedLocks), memTracker: it.memTracker, replicaReadSeed: it.replicaReadSeed, actionOnExceed: it.actionOnExceed, @@ -706,8 +707,8 @@ func (worker *copIteratorWorker) handleTaskOnce(bo *tikv.Backoffer, task *copTas }) req.StoreTp = getEndPointType(task.storeType) startTime := time.Now() - if worker.Stats == nil { - worker.Stats = make(map[tikvrpc.CmdType]*tikv.RPCRuntimeStats) + if worker.kvclient.Stats == nil { + worker.kvclient.Stats = make(map[tikvrpc.CmdType]*tikv.RPCRuntimeStats) } if worker.req.IsStaleness { req.EnableStaleRead() @@ -716,7 +717,8 @@ func (worker *copIteratorWorker) handleTaskOnce(bo *tikv.Backoffer, task *copTas if len(worker.req.MatchStoreLabels) > 0 { ops = append(ops, tikv.WithMatchLabels(worker.req.MatchStoreLabels)) } - resp, rpcCtx, storeAddr, err := worker.SendReqCtx(bo, req, task.region, tikv.ReadTimeoutMedium, getEndPointType(task.storeType), task.storeAddr, ops...) + resp, rpcCtx, storeAddr, err := worker.kvclient.SendReqCtx(bo, req, task.region, tikv.ReadTimeoutMedium, getEndPointType(task.storeType), task.storeAddr, ops...) + err = txndriver.ToTiDBErr(err) if err != nil { if task.storeType == kv.TiDB { err = worker.handleTiDBSendReqErr(err, task, ch) @@ -872,7 +874,8 @@ func (worker *copIteratorWorker) handleCopResponse(bo *tikv.Backoffer, rpcCtx *t if lockErr := resp.pbResp.GetLocked(); lockErr != nil { logutil.BgLogger().Debug("coprocessor encounters", zap.Stringer("lock", lockErr)) - msBeforeExpired, err1 := worker.ResolveLocks(bo, worker.req.StartTs, []*tikv.Lock{tikv.NewLock(lockErr)}) + msBeforeExpired, err1 := worker.kvclient.ResolveLocks(bo, worker.req.StartTs, []*tikv.Lock{tikv.NewLock(lockErr)}) + err1 = txndriver.ToTiDBErr(err1) if err1 != nil { return nil, errors.Trace(err1) } @@ -901,8 +904,8 @@ func (worker *copIteratorWorker) handleCopResponse(bo *tikv.Backoffer, rpcCtx *t if resp.detail == nil { resp.detail = new(CopRuntimeStats) } - resp.detail.Stats = worker.Stats - worker.Stats = nil + resp.detail.Stats = worker.kvclient.Stats + worker.kvclient.Stats = nil backoffTimes := bo.GetBackoffTimes() resp.detail.BackoffTime = time.Duration(bo.GetTotalSleep()) * time.Millisecond resp.detail.BackoffSleep = make(map[string]time.Duration, len(backoffTimes)) @@ -980,7 +983,7 @@ type CopRuntimeStats struct { func (worker *copIteratorWorker) handleTiDBSendReqErr(err error, task *copTask, ch chan<- *copResponse) error { errCode := errno.ErrUnknown errMsg := err.Error() - if terror.ErrorEqual(err, tikverr.ErrTiKVServerTimeout) { + if terror.ErrorEqual(err, txndriver.ErrTiKVServerTimeout) { errCode = errno.ErrTiKVServerTimeout errMsg = "TiDB server timeout, address is " + task.storeAddr } diff --git a/store/driver/txn/error.go b/store/driver/txn/error.go index d0248c47760a8..03e4b3c42d39f 100644 --- a/store/driver/txn/error.go +++ b/store/driver/txn/error.go @@ -53,6 +53,8 @@ var ( ErrTiFlashServerBusy = dbterror.ClassTiKV.NewStd(errno.ErrTiFlashServerBusy) // ErrPDServerTimeout is the error when pd server is timeout. ErrPDServerTimeout = dbterror.ClassTiKV.NewStd(errno.ErrPDServerTimeout) + // ErrRegionUnavailable is the error when region is not available. + ErrRegionUnavailable = dbterror.ClassTiKV.NewStd(errno.ErrRegionUnavailable) ) func genKeyExistsError(name string, value string, err error) error { @@ -235,6 +237,10 @@ func ToTiDBErr(err error) error { return ErrResolveLockTimeout } + if errors.ErrorEqual(err, tikverr.ErrRegionUnavailable) { + return ErrRegionUnavailable + } + return errors.Trace(originErr) } diff --git a/store/tikv/error/errcode.go b/store/tikv/error/errcode.go index 0b15b97edb8d7..070a49deb8b88 100644 --- a/store/tikv/error/errcode.go +++ b/store/tikv/error/errcode.go @@ -25,8 +25,6 @@ const ( CodeLockAcquireFailAndNoWaitSet = 3572 // TiKV/PD/TiFlash errors. - CodeRegionUnavailable = 9005 - CodeTiKVStoreLimit = 9008 CodeTiFlashServerTimeout = 9012 diff --git a/store/tikv/error/error.go b/store/tikv/error/error.go index 734851865820c..571cfbc55b898 100644 --- a/store/tikv/error/error.go +++ b/store/tikv/error/error.go @@ -46,6 +46,8 @@ var ( ErrTiKVServerBusy = errors.New("tikv server busy") // ErrTiFlashServerBusy is the error that tiflash server is busy. ErrTiFlashServerBusy = errors.New("tiflash server busy") + // ErrRegionUnavailable is the error when region is not available. + ErrRegionUnavailable = errors.New("region unavailable") ) // MismatchClusterID represents the message that the cluster ID of the PD client does not match the PD. @@ -54,7 +56,6 @@ const MismatchClusterID = "mismatch cluster id" // error instances. var ( ErrTiFlashServerTimeout = dbterror.ClassTiKV.NewStd(CodeTiFlashServerTimeout) - ErrRegionUnavailable = dbterror.ClassTiKV.NewStd(CodeRegionUnavailable) ErrQueryInterrupted = dbterror.ClassTiKV.NewStd(CodeQueryInterrupted) ErrLockAcquireFailAndNoWaitSet = dbterror.ClassTiKV.NewStd(CodeLockAcquireFailAndNoWaitSet) ErrLockWaitTimeout = dbterror.ClassTiKV.NewStd(CodeLockWaitTimeout) From 437a5f62518a8f55f9598843f57713020cf29901 Mon Sep 17 00:00:00 2001 From: wjHuang Date: Fri, 7 May 2021 11:03:53 +0800 Subject: [PATCH 22/67] *: fix some typos (#24418) --- planner/core/plan_test.go | 2 +- util/logutil/hex.go | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/planner/core/plan_test.go b/planner/core/plan_test.go index a7cb71136470b..53f63f25fbc18 100644 --- a/planner/core/plan_test.go +++ b/planner/core/plan_test.go @@ -6,7 +6,7 @@ // // http://www.apache.org/licenses/LICENSE-2.0 // -// Unless required by aprettyPrintlicable law or agreed to in writing, software +// Unless required by applicable law or agreed to in writing, software // distributed under the License is distributed on an "AS IS" BASIS, // See the License for the specific language governing permissions and // limitations under the License. diff --git a/util/logutil/hex.go b/util/logutil/hex.go index e8ca2faeccefe..1bff4ec591404 100644 --- a/util/logutil/hex.go +++ b/util/logutil/hex.go @@ -6,7 +6,7 @@ // // http://www.apache.org/licenses/LICENSE-2.0 // -// Unless required by aprettyPrintlicable law or agreed to in writing, software +// Unless required by applicable law or agreed to in writing, software // distributed under the License is distributed on an "AS IS" BASIS, // See the License for the specific language governing permissions and // limitations under the License. From 3e0947fd21ddb0b83ace1f1d5fdeb48d277846bc Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Fri, 7 May 2021 15:20:41 +0800 Subject: [PATCH 23/67] planner: add some test cases about partition table dynamic-mode and plan-cache (#24422) --- planner/core/prepare_test.go | 145 +++++++++++++++++++++++++++++++++++ 1 file changed, 145 insertions(+) diff --git a/planner/core/prepare_test.go b/planner/core/prepare_test.go index 686d67b00a9e0..537f1e06468bd 100644 --- a/planner/core/prepare_test.go +++ b/planner/core/prepare_test.go @@ -17,7 +17,9 @@ import ( "context" "fmt" "math" + "math/rand" "strconv" + "strings" "time" . "github.com/pingcap/check" @@ -31,6 +33,7 @@ import ( "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/util/hint" + "github.com/pingcap/tidb/util/israce" "github.com/pingcap/tidb/util/kvcache" "github.com/pingcap/tidb/util/testkit" "github.com/pingcap/tidb/util/testleak" @@ -1193,3 +1196,145 @@ func (s *testPlanSerialSuite) TestIssue23671(c *C) { tk.MustQuery("execute s1 using @a, @b, @c").Check(testkit.Rows("1 1", "2 2")) tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) } + +func (s *testPlanSerialSuite) TestPartitionTable(c *C) { + if israce.RaceEnabled { + c.Skip("exhaustive types test, skip race test") + } + + // enable plan cache + store, dom, err := newStoreWithBootstrap() + c.Assert(err, IsNil) + tk := testkit.NewTestKit(c, store) + orgEnable := core.PreparedPlanCacheEnabled() + defer func() { + dom.Close() + err = store.Close() + c.Assert(err, IsNil) + core.SetPreparedPlanCache(orgEnable) + }() + core.SetPreparedPlanCache(true) + tk.Se, err = session.CreateSession4TestWithOpt(store, &session.Opt{ + PreparedPlanCache: kvcache.NewSimpleLRUCache(100, 0.1, math.MaxUint64), + }) + c.Assert(err, IsNil) + + // enable partition table dynamic mode + tk.MustExec("create database test_plan_cache") + tk.MustExec("use test_plan_cache") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + + type testcase struct { + t1Create string + t2Create string + rowGener func() string + varGener func() string + query string + } + randDateTime := func() string { + return fmt.Sprintf("%v-%v-%v %v:%v:%v", + 1950+rand.Intn(100), 1+rand.Intn(12), 1+rand.Intn(28), // date + rand.Intn(24), rand.Intn(60), rand.Intn(60)) // time + } + randDate := func() string { + return fmt.Sprintf("%v-%v-%v", 1950+rand.Intn(100), 1+rand.Intn(12), 1+rand.Intn(28)) + } + testcases := []testcase{ + { // hash partition + int + "create table t1(a int, b int) partition by hash(a) partitions 20", + "create table t2(a int, b int)", + func() string { return fmt.Sprintf("(%v, %v)", rand.Intn(100000000), rand.Intn(100000000)) }, + func() string { return fmt.Sprintf("%v", rand.Intn(100000000)) }, + `select * from %v where a > ?`, + }, + { // range partition + int + `create table t1(a int, b int) partition by range(a) ( + partition p0 values less than (20000000), + partition p1 values less than (40000000), + partition p2 values less than (60000000), + partition p3 values less than (80000000), + partition p4 values less than (100000000))`, + `create table t2(a int, b int)`, + func() string { return fmt.Sprintf("(%v, %v)", rand.Intn(100000000), rand.Intn(100000000)) }, + func() string { return fmt.Sprintf("%v", rand.Intn(100000000)) }, + `select * from %v where a > ?`, + }, + { // range partition + varchar + `create table t1(a varchar(10), b varchar(10)) partition by range columns(a) ( + partition p0 values less than ('200'), + partition p1 values less than ('400'), + partition p2 values less than ('600'), + partition p3 values less than ('800'), + partition p4 values less than ('9999'))`, + `create table t2(a varchar(10), b varchar(10))`, + func() string { return fmt.Sprintf(`("%v", "%v")`, rand.Intn(1000), rand.Intn(1000)) }, + func() string { return fmt.Sprintf(`"%v"`, rand.Intn(1000)) }, + `select * from %v where a > ?`, + }, + { // range partition + datetime + `create table t1(a datetime, b datetime) partition by range columns(a) ( + partition p0 values less than ('1970-01-01 00:00:00'), + partition p1 values less than ('1990-01-01 00:00:00'), + partition p2 values less than ('2010-01-01 00:00:00'), + partition p3 values less than ('2030-01-01 00:00:00'), + partition p4 values less than ('2060-01-01 00:00:00'))`, + `create table t2(a datetime, b datetime)`, + func() string { return fmt.Sprintf(`("%v", "%v")`, randDateTime(), randDateTime()) }, + func() string { return fmt.Sprintf(`"%v"`, randDateTime()) }, + `select * from %v where a > ?`, + }, + { // range partition + date + `create table t1(a date, b date) partition by range columns(a) ( + partition p0 values less than ('1970-01-01'), + partition p1 values less than ('1990-01-01'), + partition p2 values less than ('2010-01-01'), + partition p3 values less than ('2030-01-01'), + partition p4 values less than ('2060-01-01'))`, + `create table t2(a date, b date)`, + func() string { return fmt.Sprintf(`("%v", "%v")`, randDate(), randDate()) }, + func() string { return fmt.Sprintf(`"%v"`, randDate()) }, + `select * from %v where a > ?`, + }, + { // list partition + int + `create table t1(a int, b int) partition by list(a) ( + partition p0 values in (0, 1, 2, 3, 4), + partition p1 values in (5, 6, 7, 8, 9), + partition p2 values in (10, 11, 12, 13, 14), + partition p3 values in (15, 16, 17, 18, 19))`, + `create table t2(a int, b int)`, + func() string { return fmt.Sprintf("(%v, %v)", rand.Intn(20), rand.Intn(20)) }, + func() string { return fmt.Sprintf("%v", rand.Intn(20)) }, + `select * from %v where a > ?`, + }, + } + for _, tc := range testcases { + // create tables and insert some records + tk.MustExec("drop table if exists t1") + tk.MustExec("drop table if exists t2") + tk.MustExec(tc.t1Create) + tk.MustExec(tc.t2Create) + vals := make([]string, 0, 2048) + for i := 0; i < 2048; i++ { + vals = append(vals, tc.rowGener()) + } + tk.MustExec(fmt.Sprintf("insert into t1 values %s", strings.Join(vals, ","))) + tk.MustExec(fmt.Sprintf("insert into t2 values %s", strings.Join(vals, ","))) + + // the first query, @last_plan_from_cache should be zero + tk.MustExec(fmt.Sprintf(`prepare stmt1 from "%s"`, fmt.Sprintf(tc.query, "t1"))) + tk.MustExec(fmt.Sprintf(`prepare stmt2 from "%s"`, fmt.Sprintf(tc.query, "t2"))) + tk.MustExec(fmt.Sprintf("set @a=%v", tc.varGener())) + result1 := tk.MustQuery("execute stmt1 using @a").Sort().Rows() + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) + tk.MustQuery("execute stmt2 using @a").Sort().Check(result1) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) + + for i := 0; i < 100; i++ { + tk.MustExec(fmt.Sprintf("set @a=%v", tc.varGener())) + result1 := tk.MustQuery("execute stmt1 using @a").Sort().Rows() + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1")) + tk.MustQuery("execute stmt2 using @a").Sort().Check(result1) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1")) + } + } +} From 4bde135acddf1f8968f8954d2cbc82779256bd95 Mon Sep 17 00:00:00 2001 From: Lynn Date: Fri, 7 May 2021 16:06:42 +0800 Subject: [PATCH 24/67] ddl: generated columns to be disallowed more completely for `column type change` operation (need to change the data) (#24320) --- ddl/column.go | 5 +++++ ddl/db_change_test.go | 20 ++++++++++++++++++++ ddl/ddl_api.go | 19 +++++++------------ ddl/failtest/fail_db_test.go | 2 ++ ddl/generated_column.go | 27 +++++++++++++++++++++++++++ 5 files changed, 61 insertions(+), 12 deletions(-) diff --git a/ddl/column.go b/ddl/column.go index 805d1d2e73d1e..8311834602ae5 100644 --- a/ddl/column.go +++ b/ddl/column.go @@ -828,6 +828,11 @@ func (w *worker) onModifyColumn(d *ddlCtx, t *meta.Meta, job *model.Job) (ver in return w.doModifyColumn(d, t, job, dbInfo, tblInfo, jobParam.newCol, oldCol, jobParam.pos) } + if err = isGeneratedRelatedColumn(tblInfo, jobParam.newCol, oldCol); err != nil { + job.State = model.JobStateCancelled + return ver, errors.Trace(err) + } + if jobParam.changingCol == nil { changingColPos := &ast.ColumnPosition{Tp: ast.ColumnPositionNone} newColName := model.NewCIStr(genChangingColumnUniqueName(tblInfo, oldCol)) diff --git a/ddl/db_change_test.go b/ddl/db_change_test.go index de9d8eb504866..7c3a0f9ad970f 100644 --- a/ddl/db_change_test.go +++ b/ddl/db_change_test.go @@ -1056,6 +1056,26 @@ func (s *testStateChangeSuite) TestParallelAlterModifyColumn(c *C) { s.testControlParallelExecSQL(c, sql, sql, f) } +func (s *testStateChangeSuite) TestParallelAddGeneratedColumnAndAlterModifyColumn(c *C) { + _, err := s.se.Execute(context.Background(), "set global tidb_enable_change_column_type = 1") + c.Assert(err, IsNil) + defer func() { + _, err = s.se.Execute(context.Background(), "set global tidb_enable_change_column_type = 0") + c.Assert(err, IsNil) + }() + domain.GetDomain(s.se).GetGlobalVarsCache().Disable() + + sql1 := "ALTER TABLE t ADD COLUMN f INT GENERATED ALWAYS AS(a+1);" + sql2 := "ALTER TABLE t MODIFY COLUMN a tinyint;" + f := func(c *C, err1, err2 error) { + c.Assert(err1, IsNil) + c.Assert(err2.Error(), Equals, "[ddl:8200]Unsupported modify column: tidb_enable_change_column_type is true, oldCol is a dependent column 'a' for generated column") + _, err := s.se.Execute(context.Background(), "select * from t") + c.Assert(err, IsNil) + } + s.testControlParallelExecSQL(c, sql1, sql2, f) +} + func (s *testStateChangeSuite) TestParallelAlterModifyColumnAndAddPK(c *C) { _, err := s.se.Execute(context.Background(), "set global tidb_enable_change_column_type = 1") c.Assert(err, IsNil) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 0e1213e59c797..b03b4ca66f536 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -3863,11 +3863,10 @@ func (d *ddl) getModifiableColumnJob(ctx sessionctx.Context, ident ast.Ident, or return nil, errors.Trace(err) } if ctx.GetSessionVars().EnableChangeColumnType && needChangeColumnData(col.ColumnInfo, newCol.ColumnInfo) { - if newCol.IsGenerated() || col.IsGenerated() { - // TODO: Make it compatible with MySQL error. - msg := fmt.Sprintf("tidb_enable_change_column_type is true, newCol IsGenerated %v, oldCol IsGenerated %v", newCol.IsGenerated(), col.IsGenerated()) - return nil, errUnsupportedModifyColumn.GenWithStackByArgs(msg) - } else if t.Meta().Partition != nil { + if err = isGeneratedRelatedColumn(t.Meta(), newCol.ColumnInfo, col.ColumnInfo); err != nil { + return nil, errors.Trace(err) + } + if t.Meta().Partition != nil { return nil, errUnsupportedModifyColumn.GenWithStackByArgs("tidb_enable_change_column_type is true, table is partition table") } } @@ -5357,14 +5356,10 @@ func (d *ddl) DropIndex(ctx sessionctx.Context, ti ast.Ident, indexName model.CI } func isDroppableColumn(tblInfo *model.TableInfo, colName model.CIStr) error { - // Check whether there are other columns depend on this column or not. - for _, col := range tblInfo.Columns { - for dep := range col.Dependences { - if dep == colName.L { - return errDependentByGeneratedColumn.GenWithStackByArgs(dep) - } - } + if ok, dep := hasDependentByGeneratedColumn(tblInfo, colName); ok { + return errDependentByGeneratedColumn.GenWithStackByArgs(dep) } + if len(tblInfo.Columns) == 1 { return ErrCantRemoveAllFields.GenWithStack("can't drop only column %s in table %s", colName, tblInfo.Name) diff --git a/ddl/failtest/fail_db_test.go b/ddl/failtest/fail_db_test.go index 78d6ad9decfd6..8ccb7a5adb594 100644 --- a/ddl/failtest/fail_db_test.go +++ b/ddl/failtest/fail_db_test.go @@ -524,6 +524,8 @@ func (s *testFailDBSuite) TestModifyColumn(c *C) { c.Assert(err.Error(), Equals, "[ddl:8200]Unsupported modify column: tidb_enable_change_column_type is true, newCol IsGenerated false, oldCol IsGenerated true") _, err = tk.Exec("alter table t2 modify column a mediumint generated always as(id+1) stored") c.Assert(err.Error(), Equals, "[ddl:8200]Unsupported modify column: tidb_enable_change_column_type is true, newCol IsGenerated true, oldCol IsGenerated false") + _, err = tk.Exec("alter table t2 modify column a mediumint") + c.Assert(err.Error(), Equals, "[ddl:8200]Unsupported modify column: tidb_enable_change_column_type is true, oldCol is a dependent column 'a' for generated column") // Test multiple rows of data. tk.MustExec("create table t3(a int not null default 1, b int default 2, c int not null default 0, primary key(c), index idx(b), index idx1(a), index idx2(b, c))") diff --git a/ddl/generated_column.go b/ddl/generated_column.go index 0b33ad2a90b14..657a27ec3db4f 100644 --- a/ddl/generated_column.go +++ b/ddl/generated_column.go @@ -14,6 +14,8 @@ package ddl import ( + "fmt" + "github.com/pingcap/errors" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/model" @@ -137,6 +139,31 @@ func findColumnNamesInExpr(expr ast.ExprNode) []*ast.ColumnName { return c.cols } +// hasDependentByGeneratedColumn checks whether there are other columns depend on this column or not. +func hasDependentByGeneratedColumn(tblInfo *model.TableInfo, colName model.CIStr) (bool, string) { + for _, col := range tblInfo.Columns { + for dep := range col.Dependences { + if dep == colName.L { + return true, dep + } + } + } + return false, "" +} + +func isGeneratedRelatedColumn(tblInfo *model.TableInfo, newCol, col *model.ColumnInfo) error { + if newCol.IsGenerated() || col.IsGenerated() { + // TODO: Make it compatible with MySQL error. + msg := fmt.Sprintf("tidb_enable_change_column_type is true, newCol IsGenerated %v, oldCol IsGenerated %v", newCol.IsGenerated(), col.IsGenerated()) + return errUnsupportedModifyColumn.GenWithStackByArgs(msg) + } + if ok, dep := hasDependentByGeneratedColumn(tblInfo, col.Name); ok { + msg := fmt.Sprintf("tidb_enable_change_column_type is true, oldCol is a dependent column '%s' for generated column", dep) + return errUnsupportedModifyColumn.GenWithStackByArgs(msg) + } + return nil +} + type generatedColumnChecker struct { cols []*ast.ColumnName } From 9fe539c5a141735daece115700cf41a34447e5d1 Mon Sep 17 00:00:00 2001 From: Arenatlx <314806019@qq.com> Date: Fri, 7 May 2021 17:48:42 +0800 Subject: [PATCH 25/67] ddl: fix alter int to bit will panic in backfill index (#24263) --- ddl/column.go | 7 ++- ddl/column_type_change_test.go | 31 ++++++++++++ table/tables/tables.go | 11 +++-- util/rowDecoder/decoder.go | 89 ++++++++++++++++++++++++---------- 4 files changed, 106 insertions(+), 32 deletions(-) diff --git a/ddl/column.go b/ddl/column.go index 8311834602ae5..18c23b4d9c45a 100644 --- a/ddl/column.go +++ b/ddl/column.go @@ -1261,7 +1261,7 @@ func (w *updateColumnWorker) fetchRowColVals(txn kv.Transaction, taskRange reorg } func (w *updateColumnWorker) getRowRecord(handle kv.Handle, recordKey []byte, rawRow []byte) error { - _, err := w.rowDecoder.DecodeAndEvalRowWithMap(w.sessCtx, handle, rawRow, time.UTC, timeutil.SystemLocation(), w.rowMap) + _, err := w.rowDecoder.DecodeTheExistedColumnMap(w.sessCtx, handle, rawRow, time.UTC, w.rowMap) if err != nil { return errors.Trace(errCantDecodeRecord.GenWithStackByArgs("column", err)) } @@ -1299,6 +1299,11 @@ func (w *updateColumnWorker) getRowRecord(handle kv.Handle, recordKey []byte, ra }) w.rowMap[w.newColInfo.ID] = newColVal + _, err = w.rowDecoder.EvalRemainedExprColumnMap(w.sessCtx, timeutil.SystemLocation(), w.rowMap) + if err != nil { + return errors.Trace(err) + } + newColumnIDs := make([]int64, 0, len(w.rowMap)) newRow := make([]types.Datum, 0, len(w.rowMap)) for colID, val := range w.rowMap { diff --git a/ddl/column_type_change_test.go b/ddl/column_type_change_test.go index 16c0c6f38e308..992631b4bd97b 100644 --- a/ddl/column_type_change_test.go +++ b/ddl/column_type_change_test.go @@ -1766,3 +1766,34 @@ func (s *testColumnTypeChangeSuite) TestDDLExitWhenCancelMeetPanic(c *C) { c.Assert(job.ErrorCount, Equals, int64(4)) c.Assert(job.Error.Error(), Equals, "[ddl:-1]panic in handling DDL logic and error count beyond the limitation 3, cancelled") } + +// Close issue #24253 +func (s *testColumnTypeChangeSuite) TestChangeIntToBitWillPanicInBackfillIndexes(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + // Enable column change variable. + tk.Se.GetSessionVars().EnableChangeColumnType = true + + tk.MustExec("drop table if exists t") + tk.MustExec("CREATE TABLE `t` (" + + " `a` int(11) DEFAULT NULL," + + " `b` varchar(10) DEFAULT NULL," + + " `c` decimal(10,2) DEFAULT NULL," + + " KEY `idx1` (`a`)," + + " UNIQUE KEY `idx2` (`a`)," + + " KEY `idx3` (`a`,`b`)," + + " KEY `idx4` (`a`,`b`,`c`)" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin") + tk.MustExec("insert into t values(19,1,1),(17,2,2)") + tk.MustExec("alter table t modify a bit(5) not null") + tk.MustQuery("show create table t").Check(testkit.Rows("t CREATE TABLE `t` (\n" + + " `a` bit(5) NOT NULL,\n" + + " `b` varchar(10) DEFAULT NULL,\n" + + " `c` decimal(10,2) DEFAULT NULL,\n" + + " KEY `idx1` (`a`),\n" + + " UNIQUE KEY `idx2` (`a`),\n" + + " KEY `idx3` (`a`,`b`),\n" + + " KEY `idx4` (`a`,`b`,`c`)\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin")) + tk.MustQuery("select * from t").Check(testkit.Rows("\x13 1 1.00", "\x11 2 2.00")) +} diff --git a/table/tables/tables.go b/table/tables/tables.go index 7da187c785b4c..8fd3cca9e2657 100644 --- a/table/tables/tables.go +++ b/table/tables/tables.go @@ -978,15 +978,16 @@ func DecodeRawRowData(ctx sessionctx.Context, meta *model.TableInfo, h kv.Handle } // GetChangingColVal gets the changing column value when executing "modify/change column" statement. +// For statement like update-where, it will fetch the old row out and insert it into kv again. +// Since update statement can see the writable columns, it is responsible for the casting relative column / get the fault value here. +// old row : a-b-[nil] +// new row : a-b-[a'/default] +// Thus the writable new row is corresponding to Write-Only constraints. func GetChangingColVal(ctx sessionctx.Context, cols []*table.Column, col *table.Column, rowMap map[int64]types.Datum, defaultVals []types.Datum) (_ types.Datum, isDefaultVal bool, err error) { relativeCol := cols[col.ChangeStateInfo.DependencyColumnOffset] idxColumnVal, ok := rowMap[relativeCol.ID] if ok { - // It needs cast values here when filling back column or index values in "modify/change column" statement. - if ctx.GetSessionVars().StmtCtx.IsDDLJobInQueue { - return idxColumnVal, false, nil - } - idxColumnVal, err := table.CastValue(ctx, rowMap[relativeCol.ID], col.ColumnInfo, false, false) + idxColumnVal, err = table.CastValue(ctx, idxColumnVal, col.ColumnInfo, false, false) // TODO: Consider sql_mode and the error msg(encounter this error check whether to rollback). if err != nil { return idxColumnVal, false, errors.Trace(err) diff --git a/util/rowDecoder/decoder.go b/util/rowDecoder/decoder.go index 111481ad7b40d..3fd63c05fae96 100644 --- a/util/rowDecoder/decoder.go +++ b/util/rowDecoder/decoder.go @@ -57,14 +57,8 @@ func NewRowDecoder(tbl table.Table, cols []*table.Column, decodeColMap map[int64 tps := make([]*types.FieldType, len(cols)) for _, col := range cols { - if col.ChangeStateInfo == nil { - tps[col.Offset] = &col.FieldType - } else { - // Since changing column in the mutRow will be set with relative column's old value in the process of column-type-change, - // we should set fieldType as the relative column does. Otherwise it may get a panic, take change json to int as an example, - // setting json value to a int type column in mutRow will panic because it lacks of offset array. - tps[col.Offset] = &cols[col.ChangeStateInfo.DependencyColumnOffset].FieldType - } + // Even for changing column in column type change, we target field type uniformly. + tps[col.Offset] = &col.FieldType } var pkCols []int64 switch { @@ -119,6 +113,66 @@ func (rd *RowDecoder) DecodeAndEvalRowWithMap(ctx sessionctx.Context, handle kv. } rd.mutRow.SetValue(colInfo.Offset, val.GetValue()) } + return rd.EvalRemainedExprColumnMap(ctx, sysLoc, row) +} + +// BuildFullDecodeColMap builds a map that contains [columnID -> struct{*table.Column, expression.Expression}] from all columns. +func BuildFullDecodeColMap(cols []*table.Column, schema *expression.Schema) map[int64]Column { + decodeColMap := make(map[int64]Column, len(cols)) + for _, col := range cols { + decodeColMap[col.ID] = Column{ + Col: col, + GenExpr: schema.Columns[col.Offset].VirtualExpr, + } + } + return decodeColMap +} + +// CurrentRowWithDefaultVal returns current decoding row with default column values set properly. +// Please make sure calling DecodeAndEvalRowWithMap first. +func (rd *RowDecoder) CurrentRowWithDefaultVal() chunk.Row { + return rd.mutRow.ToRow() +} + +// DecodeTheExistedColumnMap is used by ddl column-type-change first column reorg stage. +// In the function, we only decode the existed column in the row and fill the default value. +// For changing column, we shouldn't cast it here, because we will do a unified cast operation latter. +// For generated column, we didn't cast it here too, because the eval process will depend on the changing column. +func (rd *RowDecoder) DecodeTheExistedColumnMap(ctx sessionctx.Context, handle kv.Handle, b []byte, decodeLoc *time.Location, row map[int64]types.Datum) (map[int64]types.Datum, error) { + var err error + if rowcodec.IsNewFormat(b) { + row, err = tablecodec.DecodeRowWithMapNew(b, rd.colTypes, decodeLoc, row) + } else { + row, err = tablecodec.DecodeRowWithMap(b, rd.colTypes, decodeLoc, row) + } + if err != nil { + return nil, err + } + row, err = tablecodec.DecodeHandleToDatumMap(handle, rd.pkCols, rd.colTypes, decodeLoc, row) + if err != nil { + return nil, err + } + for _, dCol := range rd.colMap { + colInfo := dCol.Col.ColumnInfo + val, ok := row[colInfo.ID] + if ok || dCol.GenExpr != nil || dCol.Col.ChangeStateInfo != nil { + rd.mutRow.SetValue(colInfo.Offset, val.GetValue()) + continue + } + // Get the default value of the column in the generated column expression. + val, err = tables.GetColDefaultValue(ctx, dCol.Col, rd.defaultVals) + if err != nil { + return nil, err + } + rd.mutRow.SetValue(colInfo.Offset, val.GetValue()) + } + // return the existed column map here. + return row, nil +} + +// EvalRemainedExprColumnMap is used by ddl column-type-change first column reorg stage. +// It is always called after DecodeTheExistedColumnMap to finish the generated column evaluation. +func (rd *RowDecoder) EvalRemainedExprColumnMap(ctx sessionctx.Context, sysLoc *time.Location, row map[int64]types.Datum) (map[int64]types.Datum, error) { keys := make([]int, 0, len(rd.colMap)) ids := make(map[int]int, len(rd.colMap)) for k, col := range rd.colMap { @@ -155,23 +209,6 @@ func (rd *RowDecoder) DecodeAndEvalRowWithMap(ctx sessionctx.Context, handle kv. row[int64(ids[id])] = val } + // return the existed and evaluated column map here. return row, nil } - -// BuildFullDecodeColMap builds a map that contains [columnID -> struct{*table.Column, expression.Expression}] from all columns. -func BuildFullDecodeColMap(cols []*table.Column, schema *expression.Schema) map[int64]Column { - decodeColMap := make(map[int64]Column, len(cols)) - for _, col := range cols { - decodeColMap[col.ID] = Column{ - Col: col, - GenExpr: schema.Columns[col.Offset].VirtualExpr, - } - } - return decodeColMap -} - -// CurrentRowWithDefaultVal returns current decoding row with default column values set properly. -// Please make sure calling DecodeAndEvalRowWithMap first. -func (rd *RowDecoder) CurrentRowWithDefaultVal() chunk.Row { - return rd.mutRow.ToRow() -} From a329ce6cff1da79a138720edbb908fe9b7868757 Mon Sep 17 00:00:00 2001 From: MyonKeminta <9948422+MyonKeminta@users.noreply.github.com> Date: Fri, 7 May 2021 18:42:41 +0800 Subject: [PATCH 26/67] docs: Add design document about Lock View (#24375) --- docs/design/2021-04-26-lock-view.md | 196 ++++++++++++++++++++++++++++ 1 file changed, 196 insertions(+) create mode 100644 docs/design/2021-04-26-lock-view.md diff --git a/docs/design/2021-04-26-lock-view.md b/docs/design/2021-04-26-lock-view.md new file mode 100644 index 0000000000000..56d16e8e86194 --- /dev/null +++ b/docs/design/2021-04-26-lock-view.md @@ -0,0 +1,196 @@ +# TiDB Design Documents + +- Author(s): [longfangsong](https://github.com/longfangsong), [MyonKeminta](http://github.com/MyonKeminta) +- Last updated: May 6, 2021 +- Discussion PR: N/A +- Tracking Issue: https://github.com/pingcap/tidb/issues/24199 + +## Table of Contents + +* [Introduction](#introduction) +* [Motivation or Background](#motivation-or-background) +* [Detailed Design](#detailed-design) +* [Test Design](#test-design) + * [Functional Tests](#functional-tests) + * [Scenario Tests](#scenario-tests) + * [Compatibility Tests](#compatibility-tests) + * [Benchmark Tests](#benchmark-tests) +* [Impacts & Risks](#impacts--risks) +* [Investigation & Alternatives](#investigation--alternatives) +* [Unresolved Questions](#unresolved-questions) + +## Introduction + +This document describes the design of the feature Lock View, which provides tools to analyze problems about transaction's lock waiting, lock contentions and deadlocks. + +## Motivation or Background + +Currently, it's very hard to analyze lock contentions and deadlocks for transactions. One may need to enable general log, try to reproduce the problem, and try to analyze the log to find the cause, which is very difficult and inconvenient. We also found that this way of analyzing is not feasible in some scenarios. It's highly required to provide some better approach to analyze these kinds of problems. + +## Detailed Design + +Several tables will be provided in `information_schema`. Some tables has both local version (fetches data on the current TiDB node) and global version (fetches data among the whole cluster), and the global version's table name has the `"CLUSTER_"` prefix. + +### Table `(CLUSTER_)TIDB_TRX` + +| Field | Type | Comment | +|------------|------------|---------| +|`TRX_ID` | `unsigned bigint` | The transaction ID (aka. start ts) | +|`TRX_STARTED`|`time`| Human readable start time of the transaction | +|`DIGEST`|`text`| The digest of the current executing SQL statement | +|`SQLS` | `text` | A list of all executed SQL statements' digests | +|`STATE`| `enum('Running', 'Lock waiting', 'Committing', 'RollingBack')`| The state of the transaction | +| `WAITING_START_TIME` | `time` | The elapsed time since the start of the current lock waiting (if any) | +| `SCOPE` | `enum('Global', 'Local')` | The scope of the transaction | +| `ISOLATION_LEVEL` | `enum('RR', 'RC')` | | +| `AUTOCOMMIT` | `bool` | | +| `SESSION_ID` | `unsigned bigint` | | +| `USER` | `varchar` | | +| `DB` | `varchar` | | +| `SET_COUNT` | `int` | Modified keys of the current transaction | +| `LOCKED_COUNT` | `int` | Locked keys of the current transaction | +| `MEM_BUFFER_SIZE` | `int` | Size occupied by the transaction's membuffer | + +* Life span of rows: + * Create on first writing or locking operation in a transaction + * Remove after the transaction is done +* Collecting, storing and querying: + * All these information can be collected on TiDB side. Since the amount of concurrent transactions won't be too large, and it doesn't need to be persisted, so it's ok to implement it as a memory table. For querying among the cluster, just register the table under `infoschema/cluster.go` and write the global table name with the local one. + * As the simplest way of implementing, most information can be passed with a similar way like `ProcessInfo`, or even directly passed via the `ProcessInfo` struct. +* Permission: + * `PROCESS` privilege is needed to access the full content of this table. For users without `PROCESS` permission, only transactions started by the current user will be shown, and others will be filtered out, which is similar to the `processlist` table. + +### Table `DATA_LOCK_WAITS` + +| Field | Type | Comment | +|------------|------------|---------| +| `HASH` | `bigint` | The hash of the lock in TiKV's LockManager | +| `KEY` | `varchar` | The key that's being waiting on | +| `TRX_ID` | `unsigned bigint` | The current transaction that's waiting for the lock | +| `SQL_DIGEST` | `text` | The digest of the SQL that's trying to acquire the lock | +| `CURRENT_HOLDING_TRX_ID` | `unsigned bigint` | The transaction that's holding the lock and blocks the current transaction | + +* Life span of rows: + * Created on a lock come into LockManager + * Removed after a lock leave LockManager +* Collecting, storing and querying: + * All these will be collected on TiKV LockManager, and will need a new RPC entry for TiDB to query. LockManager won't store the un-hashed key or SQL_DIGEST for now, so we need to modify it. +* The SQL Digest of the transaction that's currently holding the lock may be helpful, but it's hard to implement under the current architecture. So it won't be included in the first version of the feature. +* Permission: + * `PROCESS` privilege is needed to access this table. + +### Table `(CLUSTER_)DEAD_LOCK` + +| Field | Type | Comment | +|------------|------------|---------| +| `DEADLOCK_ID` | `int` | There needs multiple rows to represent information of a single deadlock event. This field is used to distinguish different events. | +| `OCCUR_TIME` | `time` | The physical time when the deadlock occurs | +| `TRY_LOCK_TRX_ID` | `unsigned bigint` | The transaction ID (start ts) of the transaction that's trying to acquire the lock | +| `CURRENT_SQL_DIGEST` | `text` | The SQL that's being blocked | +| `KEY` | `varchar` | The key that's being locked, but locked by another transaction in the deadlock event | +| `SQLS` | `text` | A list of the digest of SQL statements that the transaction has executed | +| `TRX_HOLDING_LOCK` | `unsigned bigint` | The transaction that's currently holding the lock. There will be another record in the table with the same `DEADLOCK_ID` for that transaction. | + +* Life span of rows: + * Create after TiDB receive a deadlock error + * FIFO,clean the oldest after buffer is full +* Collecting, storing and querying: + * All of these information can be collected on TiDB side. It just need to add the information to the table when receives deadlock error from TiKV. The information of other transactions involved in the deadlock circle needed to be fetched from elsewhere (the `TIDB_TRX` table) when handling the deadlock error. + * Currently there are no much information in the deadlock error (it doesn't has the SQLs and keys' information), which needs to be improved. +* Permission: + * `PROCESS` privilege is needed to access this table. + +### Protocol + +To pass necessary information between TiDB and TiKV to make this feature possible, there needs some additional information carried in the protocol defined in kvproto. + +deadlockpb: + +```diff + message WaitForEntry { + ... ++ bytes key = ...; ++ bytes resource_group_tag = ...; + } + + message DeadlockResponse { + ... ++ repeated WaitiForEntry wait_chain = ...; + } +``` + +kvrpcpb: + +```diff + message Context { + ... ++ bytes resource_group_tag = ...; + } + + message Deadlock { + ... ++ repeated deadlock.WaitForEntry wait_chain = ...; + } + ++ message GetLockWaitInfoRequest { ++ Context context = 1; ++ } ++ ++ message GetLockWaitInfoResponse { ++ errorpb.Error region_error = 1; ++ string error = 2; ++ repeated deadlock.WaitForEntry entries = 3; ++ } +``` + +A field `resource_group_tag` will be added to `Context`. The SQL digest (and maybe more information) will be serialized and carried in this field. This field is expected to be reused by another feature named *Top SQL* which wants to carry SQL digest and plan to most transactional requests. + +A new KV RPC `GetLockWait` will be added to allow getting the lock waiting status from TiKV. This is a store-level (instead of region level) request, like `UnsafeDestroyRange`, and those Green GC related RPCs. The request can carry some filtering options to filter out those information the user don't care about. But the current memory table implementation only allow TiDB to scan the whole table and then filter it. This may need further optimization in the future. + +The locking key and `resource_group_tag` that comes from the `Context` of the pessimistic lock request is added to the deadlock detect request, and the wait chain is added to the deadlock detect response. + +The wait chain will be added to the `Deadlock` error which is returned by the `PessimisticLock` request, so that when deadlock happens, the full wait chain information can be passed to TiDB. + +## Compatibility + +This feature is not expected to be incompatible with other features. During upgrading, when there are different versions of TiDB nodes exists at the same time, it's possible that the `CLUSTER_` prefixed tables may encounter errors. But since this feature is typically used by user manually, this shouldn't be a severe problem. So we don't need to care much about that. + +## Test Design + +### Functional Tests + +* Querying the tables defined above gives correct result. + +### Scenario Tests + +* In a scenario where there's lock contention, this feature helps locating the problem. +* In a scenario where some a SQL is blocked by another transaction, this feature helps locating the problem. +* In a scenario where deadlock happens, this feature helps finding how the deadlock is formed. + +### Compatibility Tests + +- N/A + +### Benchmark Tests + +* The feature shouldn't cause any obvious performance regression (< 2%) on normal scenarios. +* Accessing these tables shouldn't increase latency of concurrent normal queries. + +## Impacts & Risks + +* To be investigated + +## Investigation & Alternatives + +* MySQL provides `data_locks` and `data_lock_waits` tables. +* Oracle provides `v$lock` view. +* CRDB provides `crdb_internal.node_transaction_statistics` that shows rich information for transactions. + +## Unresolved Questions + +* Since lock waiting on TiKV may timeout and retry, it's possible that in a single query to `DATA_LOCK_WAIT` table doesn't shows all (logical) lock waiting. +* Information about internal transactions may not be collected in our first version of implementation. +* Since TiDB need to query transaction information after it receives the deadlock error, the transactions' status may be changed during that time. As a result the information in `(CLUSTER_)DEAD_LOCK` table can't be promised to be accurate and complete. +* Statistics about transaction conflicts is still not enough. +* Historical information of `TIDB_TRX` and `DATA_LOCK_WAITS` is not kept, which possibly makes it still difficult to investigate some kind of problems. +* The SQL digest that's holding lock and blocking the current transaction is hard to retrieve and is not included in the current design. From 358c7975b171f347789f8d9d27f7799e1cb2840f Mon Sep 17 00:00:00 2001 From: tangenta Date: Fri, 7 May 2021 18:50:41 +0800 Subject: [PATCH 27/67] test: fix unstable TestIssue20658 (#24425) --- executor/aggregate_test.go | 19 ++++++++++++------- 1 file changed, 12 insertions(+), 7 deletions(-) diff --git a/executor/aggregate_test.go b/executor/aggregate_test.go index 56a7a22f4bdd7..94820a028123d 100644 --- a/executor/aggregate_test.go +++ b/executor/aggregate_test.go @@ -1307,6 +1307,7 @@ func (s *testSuiteAgg) TestIssue20658(c *C) { for _, sql := range sqls { var expected [][]interface{} for _, con := range concurrencies { + comment := Commentf("sql: %s; concurrency: %d", sql, con) tk.MustExec(fmt.Sprintf("set @@tidb_streamagg_concurrency=%d;", con)) if con == 1 { expected = tk.MustQuery(sql).Sort().Rows() @@ -1320,16 +1321,20 @@ func (s *testSuiteAgg) TestIssue20658(c *C) { break } } - c.Assert(ok, Equals, true) + c.Assert(ok, Equals, true, comment) rows := tk.MustQuery(sql).Sort().Rows() - c.Assert(len(rows), Equals, len(expected)) + c.Assert(len(rows), Equals, len(expected), comment) for i := range rows { - v1, err := strconv.ParseFloat(rows[i][0].(string), 64) - c.Assert(err, IsNil) - v2, err := strconv.ParseFloat(expected[i][0].(string), 64) - c.Assert(err, IsNil) - c.Assert(math.Abs(v1-v2), Less, 1e-3) + rowStr, expStr := rows[i][0].(string), expected[i][0].(string) + if rowStr == "" && expStr == "" { + continue + } + v1, err := strconv.ParseFloat(rowStr, 64) + c.Assert(err, IsNil, comment) + v2, err := strconv.ParseFloat(expStr, 64) + c.Assert(err, IsNil, comment) + c.Assert(math.Abs(v1-v2), Less, 1e-3, comment) } } } From 5d1958b82b7f3e4f729cf02cc49db9bb4c9cd155 Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Fri, 7 May 2021 20:08:41 +0800 Subject: [PATCH 28/67] planner: add some test cases about partition-table dynamic mode with global-stats and SQL binding (#24430) --- executor/partition_table_test.go | 83 ++++++++++++++++++++++++++++++++ 1 file changed, 83 insertions(+) diff --git a/executor/partition_table_test.go b/executor/partition_table_test.go index b1ff32e15f729..ee7d807ef4e8f 100644 --- a/executor/partition_table_test.go +++ b/executor/partition_table_test.go @@ -14,10 +14,15 @@ package executor_test import ( + "fmt" + "math/rand" + "strings" + . "github.com/pingcap/check" "github.com/pingcap/parser/model" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/util/israce" "github.com/pingcap/tidb/util/testkit" ) @@ -222,6 +227,84 @@ func (s *partitionTableSuite) TestPartitionInfoDisable(c *C) { tk.MustQuery("select * from t_info_null where (date = '2020-10-02' or date = '2020-10-06') and app = 'xxx' and media = '19003006'").Check(testkit.Rows()) } +func (s *partitionTableSuite) TestGlobalStatsAndSQLBinding(c *C) { + if israce.RaceEnabled { + c.Skip("exhaustive types test, skip race test") + } + + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create database test_global_stats") + tk.MustExec("use test_global_stats") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + + // hash and range and list partition + tk.MustExec("create table thash(a int, b int, key(a)) partition by hash(a) partitions 4") + tk.MustExec(`create table trange(a int, b int, key(a)) partition by range(a) ( + partition p0 values less than (200), + partition p1 values less than (400), + partition p2 values less than (600), + partition p3 values less than (800), + partition p4 values less than (1001))`) + tk.MustExec(`create table tlist(a int, b int, key(a)) partition by list (a) ( + partition p0 values in (0, 1, 2, 3, 4, 5, 6, 7, 8, 9), + partition p0 values in (10, 11, 12, 13, 14, 15, 16, 17, 18, 19), + partition p0 values in (20, 21, 22, 23, 24, 25, 26, 27, 28, 29), + partition p0 values in (30, 31, 32, 33, 34, 35, 36, 37, 38, 39), + partition p0 values in (40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50))`) + + // construct some special data distribution + vals := make([]string, 0, 1000) + listVals := make([]string, 0, 1000) + for i := 0; i < 1000; i++ { + if i < 10 { + // for hash and range partition, 1% of records are in [0, 100) + vals = append(vals, fmt.Sprintf("(%v, %v)", rand.Intn(100), rand.Intn(100))) + // for list partition, 1% of records are equal to 0 + listVals = append(listVals, "(0, 0)") + } else { + vals = append(vals, fmt.Sprintf("(%v, %v)", 100+rand.Intn(900), 100+rand.Intn(900))) + listVals = append(listVals, fmt.Sprintf("(%v, %v)", 1+rand.Intn(50), 1+rand.Intn(50))) + } + } + tk.MustExec("insert into thash values " + strings.Join(vals, ",")) + tk.MustExec("insert into trange values " + strings.Join(vals, ",")) + tk.MustExec("insert into tlist values " + strings.Join(listVals, ",")) + + // before analyzing, the planner will choose TableScan to access the 1% of records + c.Assert(tk.HasPlan("select * from thash where a<100", "TableFullScan"), IsTrue) + c.Assert(tk.HasPlan("select * from trange where a<100", "TableFullScan"), IsTrue) + c.Assert(tk.HasPlan("select * from tlist where a<1", "TableFullScan"), IsTrue) + + tk.MustExec("analyze table thash") + tk.MustExec("analyze table trange") + tk.MustExec("analyze table tlist") + + // after analyzing, the planner will use the Index(a) + tk.MustIndexLookup("select * from thash where a<100") + tk.MustIndexLookup("select * from trange where a<100") + tk.MustIndexLookup("select * from tlist where a<1") + + // create SQL bindings + tk.MustExec("create session binding for select * from thash where a<100 using select * from thash ignore index(a) where a<100") + tk.MustExec("create session binding for select * from trange where a<100 using select * from trange ignore index(a) where a<100") + tk.MustExec("create session binding for select * from tlist where a<100 using select * from tlist ignore index(a) where a<100") + + // use TableScan again since the Index(a) is ignored + c.Assert(tk.HasPlan("select * from thash where a<100", "TableFullScan"), IsTrue) + c.Assert(tk.HasPlan("select * from trange where a<100", "TableFullScan"), IsTrue) + c.Assert(tk.HasPlan("select * from tlist where a<1", "TableFullScan"), IsTrue) + + // drop SQL bindings + tk.MustExec("drop session binding for select * from thash where a<100") + tk.MustExec("drop session binding for select * from trange where a<100") + tk.MustExec("drop session binding for select * from tlist where a<100") + + // use Index(a) again + tk.MustIndexLookup("select * from thash where a<100") + tk.MustIndexLookup("select * from trange where a<100") + tk.MustIndexLookup("select * from tlist where a<1") +} + func (s *globalIndexSuite) TestGlobalIndexScan(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) tk.MustExec("drop table if exists p") From 4127c6b19a1489e35a3659224888c40ff7b09c99 Mon Sep 17 00:00:00 2001 From: tangenta Date: Fri, 7 May 2021 20:18:51 +0800 Subject: [PATCH 29/67] server: add MVCC API for secondary index with common handle (#24436) --- docs/tidb_http_api.md | 30 +++++++++++++++++++++ server/http_handler.go | 52 ++++++++++++++++++------------------- server/http_handler_test.go | 14 +++++++--- server/http_status.go | 3 ++- 4 files changed, 68 insertions(+), 31 deletions(-) diff --git a/docs/tidb_http_api.md b/docs/tidb_http_api.md index d948146b58b83..220c9479aef10 100644 --- a/docs/tidb_http_api.md +++ b/docs/tidb_http_api.md @@ -289,6 +289,36 @@ timezone.* $curl http://127.0.0.1:10080/mvcc/index/test(p1)/t1/idx/1\?a\=A ``` + If the handle is clustered, also specify the primary key column values in the query string + + ```shell + $curl http://{TiDBIP}:10080/mvcc/index/{db}/{table}/{index}?${c1}={v1}&${c2}=${v2} + ``` + + ```shell + $curl http://127.0.0.1:10080/mvcc/index/test/t/idx\?a\=1.1\&b\=111\&c\=1 + { + "key": "74800000000000003B5F69800000000000000203800000000000000105BFF199999999999A013131310000000000FA", + "region_id": 59, + "value": { + "info": { + "writes": [ + { + "start_ts": 424752858505150464, + "commit_ts": 424752858506461184, + "short_value": "AH0B" + } + ], + "values": [ + { + "start_ts": 424752858505150464, + "value": "AH0B" + } + ] + } + } + } + 1. Scatter regions of the specified table, add a `scatter-range` scheduler for the PD and the range is same as the table range. ```shell diff --git a/server/http_handler.go b/server/http_handler.go index 9ac2de71474ae..67babd1f05e8d 100644 --- a/server/http_handler.go +++ b/server/http_handler.go @@ -157,7 +157,7 @@ func (t *tikvHandlerTool) getRegionIDByKey(encodedKey []byte) (uint64, error) { return keyLocation.Region.GetID(), nil } -func (t *tikvHandlerTool) getMvccByHandle(tb table.PhysicalTable, params map[string]string, values url.Values) (*mvccKV, error) { +func (t *tikvHandlerTool) getHandle(tb table.PhysicalTable, params map[string]string, values url.Values) (kv.Handle, error) { var handle kv.Handle if intHandleStr, ok := params[pHandle]; ok { if tb.Meta().IsCommonHandle { @@ -196,16 +196,7 @@ func (t *tikvHandlerTool) getMvccByHandle(tb table.PhysicalTable, params map[str return nil, errors.Trace(err) } } - encodedKey := tablecodec.EncodeRecordKey(tb.RecordPrefix(), handle) - data, err := t.GetMvccByEncodedKey(encodedKey) - if err != nil { - return nil, err - } - regionID, err := t.getRegionIDByKey(encodedKey) - if err != nil { - return nil, err - } - return &mvccKV{Key: strings.ToUpper(hex.EncodeToString(encodedKey)), Value: data, RegionID: regionID}, err + return handle, nil } func (t *tikvHandlerTool) getMvccByStartTs(startTS uint64, startKey, endKey kv.Key) (*mvccKV, error) { @@ -272,7 +263,7 @@ func (t *tikvHandlerTool) getMvccByStartTs(startTS uint64, startKey, endKey kv.K } } -func (t *tikvHandlerTool) getMvccByIdxValue(idx table.Index, values url.Values, idxCols []*model.ColumnInfo, handleStr string) (*mvccKV, error) { +func (t *tikvHandlerTool) getMvccByIdxValue(idx table.Index, values url.Values, idxCols []*model.ColumnInfo, handle kv.Handle) (*mvccKV, error) { sc := new(stmtctx.StatementContext) // HTTP request is not a database session, set timezone to UTC directly here. // See https://github.com/pingcap/tidb/blob/master/docs/tidb_http_api.md for more details. @@ -281,11 +272,7 @@ func (t *tikvHandlerTool) getMvccByIdxValue(idx table.Index, values url.Values, if err != nil { return nil, errors.Trace(err) } - handle, err := strconv.ParseInt(handleStr, 10, 64) - if err != nil { - return nil, errors.Trace(err) - } - encodedKey, _, err := idx.GenIndexKey(sc, idxRow, kv.IntHandle(handle), nil) + encodedKey, _, err := idx.GenIndexKey(sc, idxRow, handle, nil) if err != nil { return nil, errors.Trace(err) } @@ -473,11 +460,10 @@ type mvccTxnHandler struct { } const ( - opMvccGetByHex = "hex" - opMvccGetByKey = "key" - opMvccGetByIdx = "idx" - opMvccGetByTxn = "txn" - opMvccGetByClusteredKey = "cls_key" + opMvccGetByHex = "hex" + opMvccGetByKey = "key" + opMvccGetByIdx = "idx" + opMvccGetByTxn = "txn" ) // ServeHTTP handles request of list a database or table's schemas. @@ -1583,7 +1569,7 @@ func (h mvccTxnHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) { switch h.op { case opMvccGetByHex: data, err = h.handleMvccGetByHex(params) - case opMvccGetByIdx, opMvccGetByKey, opMvccGetByClusteredKey: + case opMvccGetByIdx, opMvccGetByKey: if req.URL == nil { err = errors.BadRequestf("Invalid URL") break @@ -1627,12 +1613,15 @@ func extractTableAndPartitionName(str string) (string, string) { func (h mvccTxnHandler) handleMvccGetByIdx(params map[string]string, values url.Values) (interface{}, error) { dbName := params[pDBName] tableName := params[pTableName] - handleStr := params[pHandle] t, err := h.getTable(dbName, tableName) if err != nil { return nil, errors.Trace(err) } + handle, err := h.getHandle(t, params, values) + if err != nil { + return nil, errors.Trace(err) + } var idxCols []*model.ColumnInfo var idx table.Index @@ -1648,7 +1637,7 @@ func (h mvccTxnHandler) handleMvccGetByIdx(params map[string]string, values url. if idx == nil { return nil, errors.NotFoundf("Index %s not found!", params[pIndexName]) } - return h.getMvccByIdxValue(idx, values, idxCols, handleStr) + return h.getMvccByIdxValue(idx, values, idxCols, handle) } func (h mvccTxnHandler) handleMvccGetByKey(params map[string]string, values url.Values) (interface{}, error) { @@ -1658,10 +1647,21 @@ func (h mvccTxnHandler) handleMvccGetByKey(params map[string]string, values url. if err != nil { return nil, errors.Trace(err) } - resp, err := h.getMvccByHandle(tb, params, values) + handle, err := h.getHandle(tb, params, values) + if err != nil { + return nil, err + } + + encodedKey := tablecodec.EncodeRecordKey(tb.RecordPrefix(), handle) + data, err := h.GetMvccByEncodedKey(encodedKey) + if err != nil { + return nil, err + } + regionID, err := h.getRegionIDByKey(encodedKey) if err != nil { return nil, err } + resp := &mvccKV{Key: strings.ToUpper(hex.EncodeToString(encodedKey)), Value: data, RegionID: regionID} if len(values.Get("decode")) == 0 { return resp, nil } diff --git a/server/http_handler_test.go b/server/http_handler_test.go index 9d7fbb15d69f7..a7495b987f24a 100644 --- a/server/http_handler_test.go +++ b/server/http_handler_test.go @@ -274,8 +274,8 @@ func (ts *HTTPHandlerTestSuite) TestRegionsAPIForClusterIndex(c *C) { frameCnt++ } } - // Primary index is as the record frame, so frame count is 1. - c.Assert(frameCnt, Equals, 1) + // frameCnt = clustered primary key + secondary index(idx) = 2. + c.Assert(frameCnt, Equals, 2) c.Assert(resp.Body.Close(), IsNil) } } @@ -294,9 +294,11 @@ func (ts *HTTPHandlerTestSuite) TestRangesAPI(c *C) { err = decoder.Decode(&data) c.Assert(err, IsNil) c.Assert(data.TableName, Equals, "t") - c.Assert(len(data.Indices), Equals, 1) + c.Assert(len(data.Indices), Equals, 2) _, ok := data.Indices["PRIMARY"] c.Assert(ok, IsTrue) + _, ok = data.Indices["idx"] + c.Assert(ok, IsTrue) } func (ts *HTTPHandlerTestSuite) regionContainsTable(c *C, regionID uint64, tableID int64) bool { @@ -552,7 +554,7 @@ partition by range (a) err = txn2.Commit() c.Assert(err, IsNil) dbt.mustExec("drop table if exists t") - dbt.mustExec("create table t (a double, b varchar(20), c int, primary key(a,b) clustered)") + dbt.mustExec("create table t (a double, b varchar(20), c int, primary key(a,b) clustered, key idx(c))") dbt.mustExec("insert into t values(1.1,'111',1),(2.2,'222',2)") } @@ -956,6 +958,10 @@ func (ts *HTTPHandlerTestSuite) TestGetIndexMVCC(c *C) { c.Assert(err, IsNil) decodeKeyMvcc(resp.Body, c, true) + resp, err = ts.fetchStatus("/mvcc/index/tidb/t/idx?a=1.1&b=111&c=1") + c.Assert(err, IsNil) + decodeKeyMvcc(resp.Body, c, true) + // tests for wrong key resp, err = ts.fetchStatus("/mvcc/index/tidb/test/idx1/5?a=5&b=1") c.Assert(err, IsNil) diff --git a/server/http_status.go b/server/http_status.go index 034e71fcfd7b2..b385ea0c45890 100644 --- a/server/http_status.go +++ b/server/http_status.go @@ -153,10 +153,11 @@ func (s *Server) startHTTPServer() { } // HTTP path for get MVCC info - router.Handle("/mvcc/key/{db}/{table}", mvccTxnHandler{tikvHandlerTool, opMvccGetByClusteredKey}) + router.Handle("/mvcc/key/{db}/{table}", mvccTxnHandler{tikvHandlerTool, opMvccGetByKey}) router.Handle("/mvcc/key/{db}/{table}/{handle}", mvccTxnHandler{tikvHandlerTool, opMvccGetByKey}) router.Handle("/mvcc/txn/{startTS}/{db}/{table}", mvccTxnHandler{tikvHandlerTool, opMvccGetByTxn}) router.Handle("/mvcc/hex/{hexKey}", mvccTxnHandler{tikvHandlerTool, opMvccGetByHex}) + router.Handle("/mvcc/index/{db}/{table}/{index}", mvccTxnHandler{tikvHandlerTool, opMvccGetByIdx}) router.Handle("/mvcc/index/{db}/{table}/{index}/{handle}", mvccTxnHandler{tikvHandlerTool, opMvccGetByIdx}) // HTTP path for generate metric profile. From 075df515e4620255a423ab2a9d6b23ab3e4279a1 Mon Sep 17 00:00:00 2001 From: Morgan Tocker Date: Fri, 7 May 2021 09:12:41 -0600 Subject: [PATCH 30/67] executor, session, variable: Move deprecation, synonyms, scope validation to sysvar struct (#24280) --- executor/set.go | 34 +--- session/session.go | 39 +++-- session/session_test.go | 39 ++++- sessionctx/variable/error.go | 2 + sessionctx/variable/mock_globalaccessor.go | 5 + sessionctx/variable/noop.go | 4 +- sessionctx/variable/session.go | 6 +- sessionctx/variable/sysvar.go | 181 +++++++++++++++------ sessionctx/variable/sysvar_test.go | 59 +++++++ sessionctx/variable/varsutil.go | 28 ++-- sessionctx/variable/varsutil_test.go | 40 +++-- 11 files changed, 308 insertions(+), 129 deletions(-) diff --git a/executor/set.go b/executor/set.go index 83dc63d0e9762..1223f973f5e39 100644 --- a/executor/set.go +++ b/executor/set.go @@ -94,44 +94,22 @@ func (e *SetExecutor) Next(ctx context.Context, req *chunk.Chunk) error { continue } - syns := e.getSynonyms(name) - // Set system variable - for _, n := range syns { - err := e.setSysVariable(n, v) - if err != nil { - return err - } + if err := e.setSysVariable(name, v); err != nil { + return err } } return nil } -func (e *SetExecutor) getSynonyms(varName string) []string { - synonyms, ok := variable.SynonymsSysVariables[varName] - if ok { - return synonyms - } - - synonyms = []string{varName} - return synonyms -} - func (e *SetExecutor) setSysVariable(name string, v *expression.VarAssignment) error { sessionVars := e.ctx.GetSessionVars() sysVar := variable.GetSysVar(name) if sysVar == nil { return variable.ErrUnknownSystemVar.GenWithStackByArgs(name) } - if sysVar.Scope == variable.ScopeNone { - return errors.Errorf("Variable '%s' is a read only variable", name) - } var valStr string var err error if v.IsGlobal { - // Set global scope system variable. - if sysVar.Scope&variable.ScopeGlobal == 0 { - return errors.Errorf("Variable '%s' is a SESSION variable and can't be used with SET GLOBAL", name) - } valStr, err = e.getVarValue(v, sysVar) if err != nil { return err @@ -152,10 +130,6 @@ func (e *SetExecutor) setSysVariable(name string, v *expression.VarAssignment) e } logutil.BgLogger().Info("set global var", zap.Uint64("conn", sessionVars.ConnectionID), zap.String("name", name), zap.String("val", valStr)) } else { - // Set session scope system variable. - if sysVar.Scope&variable.ScopeSession == 0 { - return errors.Errorf("Variable '%s' is a GLOBAL variable and should be set with SET GLOBAL", name) - } valStr, err = e.getVarValue(v, nil) if err != nil { return err @@ -186,6 +160,10 @@ func (e *SetExecutor) setSysVariable(name string, v *expression.VarAssignment) e logutil.BgLogger().Debug("set session var", zap.Uint64("conn", sessionVars.ConnectionID), zap.String("name", name), zap.String("val", valStr)) } + // These are server instance scoped variables, and have special semantics. + // i.e. after SET SESSION, other users sessions will reflect the new value. + // TODO: in future these could be better managed as a post-set hook. + valStrToBoolStr := variable.BoolToOnOff(variable.TiDBOptOn(valStr)) switch name { diff --git a/session/session.go b/session/session.go index 8ca49b6a9e79d..13df91510f61a 100644 --- a/session/session.go +++ b/session/session.go @@ -999,29 +999,42 @@ func (s *session) GetGlobalSysVar(name string) (string, error) { } // SetGlobalSysVar implements GlobalVarAccessor.SetGlobalSysVar interface. -func (s *session) SetGlobalSysVar(name, value string) error { - if name == variable.TiDBSlowLogMasking { - name = variable.TiDBRedactLog +func (s *session) SetGlobalSysVar(name, value string) (err error) { + sv := variable.GetSysVar(name) + if sv == nil { + return variable.ErrUnknownSystemVar.GenWithStackByArgs(name) + } + if value, err = sv.Validate(s.sessionVars, value, variable.ScopeGlobal); err != nil { + return err + } + if err = sv.SetGlobalFromHook(s.sessionVars, value, false); err != nil { + return err } + + return s.updateGlobalSysVar(sv, value) +} + +// SetGlobalSysVarOnly updates the sysvar, but does not call the validation function or update aliases. +// This is helpful to prevent duplicate warnings being appended from aliases, or recursion. +func (s *session) SetGlobalSysVarOnly(name, value string) (err error) { sv := variable.GetSysVar(name) if sv == nil { return variable.ErrUnknownSystemVar.GenWithStackByArgs(name) } - var sVal string - var err error - sVal, err = sv.Validate(s.sessionVars, value, variable.ScopeGlobal) - if err != nil { + if err = sv.SetGlobalFromHook(s.sessionVars, value, true); err != nil { return err } - name = strings.ToLower(name) + return s.updateGlobalSysVar(sv, value) +} + +func (s *session) updateGlobalSysVar(sv *variable.SysVar, value string) error { // update mysql.tidb if required. - if s.varFromTiDBTable(name) { - if err = s.setTiDBTableValue(name, sVal); err != nil { + if s.varFromTiDBTable(sv.Name) { + if err := s.setTiDBTableValue(sv.Name, value); err != nil { return err } } - variable.CheckDeprecationSetSystemVar(s.sessionVars, name) - stmt, err := s.ParseWithParams(context.TODO(), "REPLACE %n.%n VALUES (%?, %?)", mysql.SystemDB, mysql.GlobalVariablesTable, name, sVal) + stmt, err := s.ParseWithParams(context.TODO(), "REPLACE %n.%n VALUES (%?, %?)", mysql.SystemDB, mysql.GlobalVariablesTable, sv.Name, value) if err != nil { return err } @@ -2635,7 +2648,7 @@ func (s *session) loadCommonGlobalVariablesIfNeeded() error { // `collation_server` is related to `character_set_server`, set `character_set_server` will also set `collation_server`. // We have to make sure we set the `collation_server` with right value. if _, ok := vars.GetSystemVar(varName); !ok || varName == variable.CollationServer { - err = variable.SetSessionSystemVar(s.sessionVars, varName, varVal) + err = vars.SetSystemVar(varName, varVal) if err != nil { return err } diff --git a/session/session_test.go b/session/session_test.go index a3a283eb09848..84442a8a16956 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -3746,7 +3746,7 @@ func (s *testSessionSuite2) TestMemoryUsageAlarmVariable(c *C) { err = tk.ExecToErr("set @@session.tidb_memory_usage_alarm_ratio=-1") c.Assert(err.Error(), Equals, "[variable:1231]Variable 'tidb_memory_usage_alarm_ratio' can't be set to the value of '-1'") err = tk.ExecToErr("set @@global.tidb_memory_usage_alarm_ratio=0.8") - c.Assert(err.Error(), Equals, "Variable 'tidb_memory_usage_alarm_ratio' is a SESSION variable and can't be used with SET GLOBAL") + c.Assert(err.Error(), Equals, "[variable:1228]Variable 'tidb_memory_usage_alarm_ratio' is a SESSION variable and can't be used with SET GLOBAL") } func (s *testSessionSuite2) TestSelectLockInShare(c *C) { @@ -4125,6 +4125,43 @@ func (s *testSessionSerialSuite) TestRemovedSysVars(c *C) { c.Assert(err.Error(), Equals, "[variable:1193]Unknown system variable 'bogus_var'") } +func (s *testSessionSerialSuite) TestCorrectScopeError(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + + variable.RegisterSysVar(&variable.SysVar{Scope: variable.ScopeNone, Name: "sv_none", Value: "acdc"}) + variable.RegisterSysVar(&variable.SysVar{Scope: variable.ScopeGlobal, Name: "sv_global", Value: "acdc"}) + variable.RegisterSysVar(&variable.SysVar{Scope: variable.ScopeSession, Name: "sv_session", Value: "acdc"}) + variable.RegisterSysVar(&variable.SysVar{Scope: variable.ScopeGlobal | variable.ScopeSession, Name: "sv_both", Value: "acdc"}) + + // check set behavior + + // none + _, err := tk.Exec("SET sv_none='acdc'") + c.Assert(err.Error(), Equals, "[variable:1238]Variable 'sv_none' is a read only variable") + _, err = tk.Exec("SET GLOBAL sv_none='acdc'") + c.Assert(err.Error(), Equals, "[variable:1238]Variable 'sv_none' is a read only variable") + + // global + tk.MustExec("SET GLOBAL sv_global='acdc'") + _, err = tk.Exec("SET sv_global='acdc'") + c.Assert(err.Error(), Equals, "[variable:1229]Variable 'sv_global' is a GLOBAL variable and should be set with SET GLOBAL") + + // session + _, err = tk.Exec("SET GLOBAL sv_session='acdc'") + c.Assert(err.Error(), Equals, "[variable:1228]Variable 'sv_session' is a SESSION variable and can't be used with SET GLOBAL") + tk.MustExec("SET sv_session='acdc'") + + // both + tk.MustExec("SET GLOBAL sv_both='acdc'") + tk.MustExec("SET sv_both='acdc'") + + // unregister + variable.UnregisterSysVar("sv_none") + variable.UnregisterSysVar("sv_global") + variable.UnregisterSysVar("sv_session") + variable.UnregisterSysVar("sv_both") +} + func (s *testSessionSerialSuite) TestTiKVSystemVars(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) diff --git a/sessionctx/variable/error.go b/sessionctx/variable/error.go index c5e14eeac83a2..c3029447b00b7 100644 --- a/sessionctx/variable/error.go +++ b/sessionctx/variable/error.go @@ -34,6 +34,8 @@ var ( ErrMaxPreparedStmtCountReached = dbterror.ClassVariable.NewStd(mysql.ErrMaxPreparedStmtCountReached) ErrUnsupportedIsolationLevel = dbterror.ClassVariable.NewStd(mysql.ErrUnsupportedIsolationLevel) errUnknownSystemVariable = dbterror.ClassVariable.NewStd(mysql.ErrUnknownSystemVariable) + errGlobalVariable = dbterror.ClassVariable.NewStd(mysql.ErrGlobalVariable) + errLocalVariable = dbterror.ClassVariable.NewStd(mysql.ErrLocalVariable) errValueNotSupportedWhen = dbterror.ClassVariable.NewStdErr(mysql.ErrNotSupportedYet, pmysql.Message("%s = OFF is not supported when %s = ON", nil)) // ErrFunctionsNoopImpl is an error to say the behavior is protected by the tidb_enable_noop_functions sysvar. // This is copied from expression.ErrFunctionsNoopImpl to prevent circular dependencies. diff --git a/sessionctx/variable/mock_globalaccessor.go b/sessionctx/variable/mock_globalaccessor.go index b2b47c8185dbf..525ce9fa9efff 100644 --- a/sessionctx/variable/mock_globalaccessor.go +++ b/sessionctx/variable/mock_globalaccessor.go @@ -35,3 +35,8 @@ func (m *MockGlobalAccessor) GetGlobalSysVar(name string) (string, error) { func (m *MockGlobalAccessor) SetGlobalSysVar(name string, value string) error { panic("not supported") } + +// SetGlobalSysVarOnly implements GlobalVarAccessor.SetGlobalSysVarOnly interface. +func (m *MockGlobalAccessor) SetGlobalSysVarOnly(name string, value string) error { + panic("not supported") +} diff --git a/sessionctx/variable/noop.go b/sessionctx/variable/noop.go index 4cc78799918f2..a3382fc1159d1 100644 --- a/sessionctx/variable/noop.go +++ b/sessionctx/variable/noop.go @@ -26,10 +26,10 @@ var noopSysVars = []*SysVar{ // It is unsafe to pretend that any variation of "read only" is enabled when the server // does not support it. It is possible that these features will be supported in future, // but until then... - {Scope: ScopeGlobal | ScopeSession, Name: TxReadOnly, Value: Off, Type: TypeBool, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + {Scope: ScopeGlobal | ScopeSession, Name: TxReadOnly, Value: Off, Type: TypeBool, Aliases: []string{TransactionReadOnly}, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { return checkReadOnly(vars, normalizedValue, originalValue, scope, false) }}, - {Scope: ScopeGlobal | ScopeSession, Name: TransactionReadOnly, Value: Off, Type: TypeBool, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + {Scope: ScopeGlobal | ScopeSession, Name: TransactionReadOnly, Value: Off, Type: TypeBool, Aliases: []string{TxReadOnly}, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { return checkReadOnly(vars, normalizedValue, originalValue, scope, false) }}, {Scope: ScopeGlobal, Name: OfflineMode, Value: Off, Type: TypeBool, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index a154d3d451b06..1554f3c429d65 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -1377,11 +1377,7 @@ func (s *SessionVars) ClearStmtVars() { // i.e. oN / on / 1 => ON func (s *SessionVars) SetSystemVar(name string, val string) error { sv := GetSysVar(name) - if err := sv.SetSessionFromHook(s, val); err != nil { - return err - } - s.systems[name] = val - return nil + return sv.SetSessionFromHook(s, val) } // GetReadableTxnMode returns the session variable TxnMode but rewrites it to "OPTIMISTIC" when it's empty. diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 751f0e70f0469..c17238c9ae9c5 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -106,24 +106,81 @@ type SysVar struct { AllowEmptyAll bool // AllowAutoValue means that the special value "-1" is permitted, even when outside of range. AllowAutoValue bool - // Validation is a callback after the type validation has been performed + // Validation is a callback after the type validation has been performed, but before the Set function Validation func(*SessionVars, string, string, ScopeFlag) (string, error) - // SetSession is called after validation + // SetSession is called after validation but before updating systems[]. It also doubles as an Init function + // and will be called on all variables in builtinGlobalVariable, regardless of their scope. SetSession func(*SessionVars, string) error + // SetGlobal is called after validation + SetGlobal func(*SessionVars, string) error // IsHintUpdatable indicate whether it's updatable via SET_VAR() hint (optional) IsHintUpdatable bool // Hidden means that it still responds to SET but doesn't show up in SHOW VARIABLES Hidden bool + // Aliases is a list of sysvars that should also be updated when this sysvar is updated. + // Updating aliases calls the SET function of the aliases, but does not update their aliases (preventing SET recursion) + Aliases []string } // SetSessionFromHook calls the SetSession func if it exists. func (sv *SysVar) SetSessionFromHook(s *SessionVars, val string) error { if sv.SetSession != nil { - return sv.SetSession(s, val) + if err := sv.SetSession(s, val); err != nil { + return err + } + } + s.systems[sv.Name] = val + + // Call the Set function on all the aliases for this sysVar + // Skipping the validation function, and not calling aliases of + // aliases. By skipping the validation function it means that things + // like duplicate warnings should not appear. + + if sv.Aliases != nil { + for _, aliasName := range sv.Aliases { + aliasSv := GetSysVar(aliasName) + if aliasSv.SetSession != nil { + if err := aliasSv.SetSession(s, val); err != nil { + return err + } + } + s.systems[aliasSv.Name] = val + } + } + return nil +} + +// SetGlobalFromHook calls the SetGlobal func if it exists. +func (sv *SysVar) SetGlobalFromHook(s *SessionVars, val string, skipAliases bool) error { + if sv.SetGlobal != nil { + return sv.SetGlobal(s, val) + } + + // Call the SetGlobalSysVarOnly function on all the aliases for this sysVar + // which skips the validation function and when SetGlobalFromHook is called again + // it will be with skipAliases=true. This helps break recursion because + // most aliases are reciprocal. + + if !skipAliases && sv.Aliases != nil { + for _, aliasName := range sv.Aliases { + if err := s.GlobalVarsAccessor.SetGlobalSysVarOnly(aliasName, val); err != nil { + return err + } + } } return nil } +// HasSessionScope returns true if the scope for the sysVar includes session. +func (sv *SysVar) HasSessionScope() bool { + return sv.Scope&ScopeSession != 0 +} + +// HasGlobalScope returns true if the scope for the sysVar includes global. +func (sv *SysVar) HasGlobalScope() bool { + return sv.Scope&ScopeGlobal != 0 +} + // Validate checks if system variable satisfies specific restriction. func (sv *SysVar) Validate(vars *SessionVars, value string, scope ScopeFlag) (string, error) { // Normalize the value and apply validation based on type. @@ -141,10 +198,17 @@ func (sv *SysVar) Validate(vars *SessionVars, value string, scope ScopeFlag) (st // validateFromType provides automatic validation based on the SysVar's type func (sv *SysVar) validateFromType(vars *SessionVars, value string, scope ScopeFlag) (string, error) { - // Some sysvars are read-only. Attempting to set should always fail. + // Check that the scope is correct and return the appropriate error message. if sv.ReadOnly || sv.Scope == ScopeNone { - return value, ErrIncorrectScope.GenWithStackByArgs(sv.Name, "read only") + return value, ErrIncorrectScope.FastGenByArgs(sv.Name, "read only") } + if scope == ScopeGlobal && !sv.HasGlobalScope() { + return value, errLocalVariable.FastGenByArgs(sv.Name) + } + if scope == ScopeSession && !sv.HasSessionScope() { + return value, errGlobalVariable.FastGenByArgs(sv.Name) + } + // The string "DEFAULT" is a special keyword in MySQL, which restores // the compiled sysvar value. In which case we can skip further validation. if strings.EqualFold(value, "DEFAULT") { @@ -434,7 +498,6 @@ func init() { for _, v := range noopSysVars { RegisterSysVar(v) } - initSynonymsSysVariables() } var defaultSysVars = []*SysVar{ @@ -543,27 +606,14 @@ var defaultSysVars = []*SysVar{ return checkCharacterSet(normalizedValue, "") }}, {Scope: ScopeNone, Name: VersionComment, Value: "TiDB Server (Apache License 2.0) " + versioninfo.TiDBEdition + " Edition, MySQL 5.7 compatible"}, - {Scope: ScopeGlobal | ScopeSession, Name: TxnIsolation, Value: "REPEATABLE-READ", Type: TypeEnum, PossibleValues: []string{"READ-UNCOMMITTED", "READ-COMMITTED", "REPEATABLE-READ", "SERIALIZABLE"}, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { - if normalizedValue == "SERIALIZABLE" || normalizedValue == "READ-UNCOMMITTED" { - if skipIsolationLevelCheck, err := GetSessionSystemVar(vars, TiDBSkipIsolationLevelCheck); err != nil { - return normalizedValue, err - } else if !TiDBOptOn(skipIsolationLevelCheck) { - return normalizedValue, ErrUnsupportedIsolationLevel.GenWithStackByArgs(normalizedValue) - } - } - return normalizedValue, nil + {Scope: ScopeGlobal | ScopeSession, Name: TxnIsolation, Value: "REPEATABLE-READ", Type: TypeEnum, Aliases: []string{TransactionIsolation}, PossibleValues: []string{"READ-UNCOMMITTED", "READ-COMMITTED", "REPEATABLE-READ", "SERIALIZABLE"}, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + // MySQL appends a warning here for tx_isolation is deprecated + // TiDB doesn't currently, but may in future. It is still commonly used by applications + // So it might be noisy to do so. + return checkIsolationLevel(vars, normalizedValue, originalValue, scope) }}, - {Scope: ScopeGlobal | ScopeSession, Name: TransactionIsolation, Value: "REPEATABLE-READ", Type: TypeEnum, PossibleValues: []string{"READ-UNCOMMITTED", "READ-COMMITTED", "REPEATABLE-READ", "SERIALIZABLE"}, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { - if normalizedValue == "SERIALIZABLE" || normalizedValue == "READ-UNCOMMITTED" { - returnErr := ErrUnsupportedIsolationLevel.GenWithStackByArgs(normalizedValue) - if skipIsolationLevelCheck, err := GetSessionSystemVar(vars, TiDBSkipIsolationLevelCheck); err != nil { - return normalizedValue, err - } else if !TiDBOptOn(skipIsolationLevelCheck) { - return normalizedValue, returnErr - } - vars.StmtCtx.AppendWarning(returnErr) - } - return normalizedValue, nil + {Scope: ScopeGlobal | ScopeSession, Name: TransactionIsolation, Value: "REPEATABLE-READ", Type: TypeEnum, Aliases: []string{TxnIsolation}, PossibleValues: []string{"READ-UNCOMMITTED", "READ-COMMITTED", "REPEATABLE-READ", "SERIALIZABLE"}, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + return checkIsolationLevel(vars, normalizedValue, originalValue, scope) }}, {Scope: ScopeGlobal | ScopeSession, Name: CollationConnection, Value: mysql.DefaultCollationName, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { return checkCollation(vars, normalizedValue, originalValue, scope) @@ -785,10 +835,16 @@ var defaultSysVars = []*SysVar{ {Scope: ScopeGlobal | ScopeSession, Name: TiDBIndexLookupConcurrency, Value: strconv.Itoa(DefIndexLookupConcurrency), Type: TypeInt, MinValue: 1, MaxValue: math.MaxInt64, AllowAutoValue: true, SetSession: func(s *SessionVars, val string) error { s.indexLookupConcurrency = tidbOptPositiveInt32(val, ConcurrencyUnset) return nil + }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + appendDeprecationWarning(vars, TiDBIndexLookupConcurrency, TiDBExecutorConcurrency) + return normalizedValue, nil }}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBIndexLookupJoinConcurrency, Value: strconv.Itoa(DefIndexLookupJoinConcurrency), Type: TypeInt, MinValue: 1, MaxValue: math.MaxInt64, AllowAutoValue: true, SetSession: func(s *SessionVars, val string) error { s.indexLookupJoinConcurrency = tidbOptPositiveInt32(val, ConcurrencyUnset) return nil + }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + appendDeprecationWarning(vars, TiDBIndexLookupJoinConcurrency, TiDBExecutorConcurrency) + return normalizedValue, nil }}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBIndexSerialScanConcurrency, Value: strconv.Itoa(DefIndexSerialScanConcurrency), Type: TypeUnsigned, MinValue: 1, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { s.indexSerialScanConcurrency = tidbOptPositiveInt32(val, DefIndexSerialScanConcurrency) @@ -853,26 +909,44 @@ var defaultSysVars = []*SysVar{ {Scope: ScopeSession, Name: TIDBMemQuotaHashJoin, Value: strconv.FormatInt(DefTiDBMemQuotaHashJoin, 10), Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { s.MemQuotaHashJoin = tidbOptInt64(val, DefTiDBMemQuotaHashJoin) return nil + }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + appendDeprecationWarning(vars, TIDBMemQuotaHashJoin, TIDBMemQuotaQuery) + return normalizedValue, nil }}, {Scope: ScopeSession, Name: TIDBMemQuotaMergeJoin, Value: strconv.FormatInt(DefTiDBMemQuotaMergeJoin, 10), Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { s.MemQuotaMergeJoin = tidbOptInt64(val, DefTiDBMemQuotaMergeJoin) return nil + }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + appendDeprecationWarning(vars, TIDBMemQuotaMergeJoin, TIDBMemQuotaQuery) + return normalizedValue, nil }}, {Scope: ScopeSession, Name: TIDBMemQuotaSort, Value: strconv.FormatInt(DefTiDBMemQuotaSort, 10), Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { s.MemQuotaSort = tidbOptInt64(val, DefTiDBMemQuotaSort) return nil + }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + appendDeprecationWarning(vars, TIDBMemQuotaSort, TIDBMemQuotaQuery) + return normalizedValue, nil }}, {Scope: ScopeSession, Name: TIDBMemQuotaTopn, Value: strconv.FormatInt(DefTiDBMemQuotaTopn, 10), Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { s.MemQuotaTopn = tidbOptInt64(val, DefTiDBMemQuotaTopn) return nil + }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + appendDeprecationWarning(vars, TIDBMemQuotaTopn, TIDBMemQuotaQuery) + return normalizedValue, nil }}, {Scope: ScopeSession, Name: TIDBMemQuotaIndexLookupReader, Value: strconv.FormatInt(DefTiDBMemQuotaIndexLookupReader, 10), Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { s.MemQuotaIndexLookupReader = tidbOptInt64(val, DefTiDBMemQuotaIndexLookupReader) return nil + }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + appendDeprecationWarning(vars, TIDBMemQuotaIndexLookupReader, TIDBMemQuotaQuery) + return normalizedValue, nil }}, {Scope: ScopeSession, Name: TIDBMemQuotaIndexLookupJoin, Value: strconv.FormatInt(DefTiDBMemQuotaIndexLookupJoin, 10), Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { s.MemQuotaIndexLookupJoin = tidbOptInt64(val, DefTiDBMemQuotaIndexLookupJoin) return nil + }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + appendDeprecationWarning(vars, TIDBMemQuotaIndexLookupJoin, TIDBMemQuotaQuery) + return normalizedValue, nil }}, {Scope: ScopeSession, Name: TiDBEnableStreaming, Value: Off, Type: TypeBool, SetSession: func(s *SessionVars, val string) error { s.EnableStreaming = TiDBOptOn(val) @@ -883,14 +957,7 @@ var defaultSysVars = []*SysVar{ return nil }}, {Scope: ScopeSession, Name: TxnIsolationOneShot, Value: "", Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { - if normalizedValue == "SERIALIZABLE" || normalizedValue == "READ-UNCOMMITTED" { - if skipIsolationLevelCheck, err := GetSessionSystemVar(vars, TiDBSkipIsolationLevelCheck); err != nil { - return normalizedValue, err - } else if !TiDBOptOn(skipIsolationLevelCheck) { - return normalizedValue, ErrUnsupportedIsolationLevel.GenWithStackByArgs(normalizedValue) - } - } - return normalizedValue, nil + return checkIsolationLevel(vars, normalizedValue, originalValue, scope) }, SetSession: func(s *SessionVars, val string) error { s.txnIsolationLevelOneShot.state = oneShotSet s.txnIsolationLevelOneShot.value = val @@ -907,30 +974,52 @@ var defaultSysVars = []*SysVar{ {Scope: ScopeGlobal | ScopeSession, Name: TiDBHashJoinConcurrency, Value: strconv.Itoa(DefTiDBHashJoinConcurrency), Type: TypeInt, MinValue: 1, MaxValue: math.MaxInt64, AllowAutoValue: true, SetSession: func(s *SessionVars, val string) error { s.hashJoinConcurrency = tidbOptPositiveInt32(val, ConcurrencyUnset) return nil + }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + appendDeprecationWarning(vars, TiDBHashJoinConcurrency, TiDBExecutorConcurrency) + return normalizedValue, nil }}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBProjectionConcurrency, Value: strconv.Itoa(DefTiDBProjectionConcurrency), Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { s.projectionConcurrency = tidbOptPositiveInt32(val, ConcurrencyUnset) return nil + }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + appendDeprecationWarning(vars, TiDBProjectionConcurrency, TiDBExecutorConcurrency) + return normalizedValue, nil }}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBHashAggPartialConcurrency, Value: strconv.Itoa(DefTiDBHashAggPartialConcurrency), Type: TypeInt, MinValue: 1, MaxValue: math.MaxInt64, AllowAutoValue: true, SetSession: func(s *SessionVars, val string) error { s.hashAggPartialConcurrency = tidbOptPositiveInt32(val, ConcurrencyUnset) return nil + }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + appendDeprecationWarning(vars, TiDBHashAggPartialConcurrency, TiDBExecutorConcurrency) + return normalizedValue, nil }}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBHashAggFinalConcurrency, Value: strconv.Itoa(DefTiDBHashAggFinalConcurrency), Type: TypeInt, MinValue: 1, MaxValue: math.MaxInt64, AllowAutoValue: true, SetSession: func(s *SessionVars, val string) error { s.hashAggFinalConcurrency = tidbOptPositiveInt32(val, ConcurrencyUnset) return nil + }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + appendDeprecationWarning(vars, TiDBHashAggFinalConcurrency, TiDBExecutorConcurrency) + return normalizedValue, nil }}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBWindowConcurrency, Value: strconv.Itoa(DefTiDBWindowConcurrency), Type: TypeInt, MinValue: 1, MaxValue: math.MaxInt64, AllowAutoValue: true, SetSession: func(s *SessionVars, val string) error { s.windowConcurrency = tidbOptPositiveInt32(val, ConcurrencyUnset) return nil + }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + appendDeprecationWarning(vars, TiDBWindowConcurrency, TiDBExecutorConcurrency) + return normalizedValue, nil }}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBMergeJoinConcurrency, Value: strconv.Itoa(DefTiDBMergeJoinConcurrency), Type: TypeInt, MinValue: 1, MaxValue: math.MaxInt64, AllowAutoValue: true, SetSession: func(s *SessionVars, val string) error { s.mergeJoinConcurrency = tidbOptPositiveInt32(val, ConcurrencyUnset) return nil + }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + appendDeprecationWarning(vars, TiDBMergeJoinConcurrency, TiDBExecutorConcurrency) + return normalizedValue, nil }}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBStreamAggConcurrency, Value: strconv.Itoa(DefTiDBStreamAggConcurrency), Type: TypeInt, MinValue: 1, MaxValue: math.MaxInt64, AllowAutoValue: true, SetSession: func(s *SessionVars, val string) error { s.streamAggConcurrency = tidbOptPositiveInt32(val, ConcurrencyUnset) return nil + }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + appendDeprecationWarning(vars, TiDBStreamAggConcurrency, TiDBExecutorConcurrency) + return normalizedValue, nil }}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableParallelApply, Value: BoolToOnOff(DefTiDBEnableParallelApply), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { s.EnableParallelApply = TiDBOptOn(val) @@ -1287,11 +1376,11 @@ var defaultSysVars = []*SysVar{ s.PartitionPruneMode.Store(strings.ToLower(strings.TrimSpace(val))) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBSlowLogMasking, Value: BoolToOnOff(DefTiDBRedactLog), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { - // TiDBSlowLogMasking is deprecated and a alias of TiDBRedactLog. - return s.SetSystemVar(TiDBRedactLog, val) + {Scope: ScopeGlobal | ScopeSession, Name: TiDBSlowLogMasking, Value: BoolToOnOff(DefTiDBRedactLog), Aliases: []string{TiDBRedactLog}, Type: TypeBool, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + appendDeprecationWarning(vars, TiDBSlowLogMasking, TiDBRedactLog) + return normalizedValue, nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBRedactLog, Value: BoolToOnOff(DefTiDBRedactLog), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeGlobal | ScopeSession, Name: TiDBRedactLog, Value: BoolToOnOff(DefTiDBRedactLog), Aliases: []string{TiDBSlowLogMasking}, Type: TypeBool, SetSession: func(s *SessionVars, val string) error { s.EnableRedactLog = TiDBOptOn(val) errors.RedactLogEnabled.Store(s.EnableRedactLog) return nil @@ -1367,20 +1456,6 @@ var defaultSysVars = []*SysVar{ // It's initialized in init() in feedback.go to solve import cycle. var FeedbackProbability *atomic2.Float64 -// SynonymsSysVariables is synonyms of system variables. -var SynonymsSysVariables = map[string][]string{} - -func addSynonymsSysVariables(synonyms ...string) { - for _, s := range synonyms { - SynonymsSysVariables[s] = synonyms - } -} - -func initSynonymsSysVariables() { - addSynonymsSysVariables(TxnIsolation, TransactionIsolation) - addSynonymsSysVariables(TxReadOnly, TransactionReadOnly) -} - // SetNamesVariables is the system variable names related to set names statements. var SetNamesVariables = []string{ CharacterSetClient, @@ -1661,4 +1736,6 @@ type GlobalVarAccessor interface { GetGlobalSysVar(name string) (string, error) // SetGlobalSysVar sets the global system variable name to value. SetGlobalSysVar(name string, value string) error + // SetGlobalSysVarOnly sets the global system variable without calling the validation function or updating aliases. + SetGlobalSysVarOnly(name string, value string) error } diff --git a/sessionctx/variable/sysvar_test.go b/sessionctx/variable/sysvar_test.go index d954c753e0088..31e44a88ad83f 100644 --- a/sessionctx/variable/sysvar_test.go +++ b/sessionctx/variable/sysvar_test.go @@ -168,3 +168,62 @@ func (*testSysVarSuite) TestEnumValidation(c *C) { c.Assert(err, IsNil) c.Assert(val, Equals, "AUTO") } + +func (*testSysVarSuite) TestSynonyms(c *C) { + sysVar := GetSysVar(TxnIsolation) + c.Assert(sysVar, NotNil) + + vars := NewSessionVars() + + // It does not permit SERIALIZABLE by default. + _, err := sysVar.Validate(vars, "SERIALIZABLE", ScopeSession) + c.Assert(err, NotNil) + c.Assert(err.Error(), Equals, "[variable:8048]The isolation level 'SERIALIZABLE' is not supported. Set tidb_skip_isolation_level_check=1 to skip this error") + + // Enable Skip isolation check + c.Assert(GetSysVar(TiDBSkipIsolationLevelCheck).SetSessionFromHook(vars, "ON"), IsNil) + + // Serializable is now permitted. + _, err = sysVar.Validate(vars, "SERIALIZABLE", ScopeSession) + c.Assert(err, IsNil) + + // Currently TiDB returns a warning because of SERIALIZABLE, but in future + // it may also return a warning because TxnIsolation is deprecated. + + warn := vars.StmtCtx.GetWarnings()[0].Err + c.Assert(warn.Error(), Equals, "[variable:8048]The isolation level 'SERIALIZABLE' is not supported. Set tidb_skip_isolation_level_check=1 to skip this error") + + c.Assert(sysVar.SetSessionFromHook(vars, "SERIALIZABLE"), IsNil) + + // When we set TxnIsolation, it also updates TransactionIsolation. + c.Assert(vars.systems[TxnIsolation], Equals, "SERIALIZABLE") + c.Assert(vars.systems[TransactionIsolation], Equals, vars.systems[TxnIsolation]) +} + +func (*testSysVarSuite) TestDeprecation(c *C) { + sysVar := GetSysVar(TiDBIndexLookupConcurrency) + c.Assert(sysVar, NotNil) + + vars := NewSessionVars() + + _, err := sysVar.Validate(vars, "1234", ScopeSession) + c.Assert(err, IsNil) + + // There was no error but there is a deprecation warning. + warn := vars.StmtCtx.GetWarnings()[0].Err + c.Assert(warn.Error(), Equals, "[variable:1287]'tidb_index_lookup_concurrency' is deprecated and will be removed in a future release. Please use tidb_executor_concurrency instead") +} + +func (*testSysVarSuite) TestScope(c *C) { + sv := SysVar{Scope: ScopeGlobal | ScopeSession, Name: "mynewsysvar", Value: On, Type: TypeEnum, PossibleValues: []string{"OFF", "ON", "AUTO"}} + c.Assert(sv.HasSessionScope(), IsTrue) + c.Assert(sv.HasGlobalScope(), IsTrue) + + sv = SysVar{Scope: ScopeGlobal, Name: "mynewsysvar", Value: On, Type: TypeEnum, PossibleValues: []string{"OFF", "ON", "AUTO"}} + c.Assert(sv.HasSessionScope(), IsFalse) + c.Assert(sv.HasGlobalScope(), IsTrue) + + sv = SysVar{Scope: ScopeNone, Name: "mynewsysvar", Value: On, Type: TypeEnum, PossibleValues: []string{"OFF", "ON", "AUTO"}} + c.Assert(sv.HasSessionScope(), IsFalse) + c.Assert(sv.HasGlobalScope(), IsFalse) +} diff --git a/sessionctx/variable/varsutil.go b/sessionctx/variable/varsutil.go index 0204b58a3e13b..bb9f0e1841e54 100644 --- a/sessionctx/variable/varsutil.go +++ b/sessionctx/variable/varsutil.go @@ -149,6 +149,17 @@ func checkReadOnly(vars *SessionVars, normalizedValue string, originalValue stri return normalizedValue, nil } +func checkIsolationLevel(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + if normalizedValue == "SERIALIZABLE" || normalizedValue == "READ-UNCOMMITTED" { + returnErr := ErrUnsupportedIsolationLevel.GenWithStackByArgs(normalizedValue) + if !TiDBOptOn(vars.systems[TiDBSkipIsolationLevelCheck]) { + return normalizedValue, ErrUnsupportedIsolationLevel.GenWithStackByArgs(normalizedValue) + } + vars.StmtCtx.AppendWarning(returnErr) + } + return normalizedValue, nil +} + // GetSessionSystemVar gets a system variable. // If it is a session only variable, use the default value defined in code. // Returns error if there is no such variable. @@ -284,7 +295,6 @@ func SetSessionSystemVar(vars *SessionVars, name string, value string) error { if err != nil { return err } - CheckDeprecationSetSystemVar(vars, name) return vars.SetSystemVar(name, sVal) } @@ -299,7 +309,6 @@ func SetStmtVar(vars *SessionVars, name string, value string) error { if err != nil { return err } - CheckDeprecationSetSystemVar(vars, name) return vars.SetStmtVar(name, sVal) } @@ -329,18 +338,9 @@ const ( maxChunkSizeLowerBound = 32 ) -// CheckDeprecationSetSystemVar checks if the system variable is deprecated. -func CheckDeprecationSetSystemVar(s *SessionVars, name string) { - switch name { - case TiDBIndexLookupConcurrency, TiDBIndexLookupJoinConcurrency, - TiDBHashJoinConcurrency, TiDBHashAggPartialConcurrency, TiDBHashAggFinalConcurrency, - TiDBProjectionConcurrency, TiDBWindowConcurrency, TiDBMergeJoinConcurrency, TiDBStreamAggConcurrency: - s.StmtCtx.AppendWarning(errWarnDeprecatedSyntax.FastGenByArgs(name, TiDBExecutorConcurrency)) - case TIDBMemQuotaHashJoin, TIDBMemQuotaMergeJoin, - TIDBMemQuotaSort, TIDBMemQuotaTopn, - TIDBMemQuotaIndexLookupReader, TIDBMemQuotaIndexLookupJoin: - s.StmtCtx.AppendWarning(errWarnDeprecatedSyntax.FastGenByArgs(name, TIDBMemQuotaQuery)) - } +// appendDeprecationWarning adds a warning that the item is deprecated. +func appendDeprecationWarning(s *SessionVars, name, replacement string) { + s.StmtCtx.AppendWarning(errWarnDeprecatedSyntax.FastGenByArgs(name, replacement)) } // TiDBOptOn could be used for all tidb session variable options, we use "ON"/1 to turn on those options. diff --git a/sessionctx/variable/varsutil_test.go b/sessionctx/variable/varsutil_test.go index f844ef1c57757..5757eaeeac403 100644 --- a/sessionctx/variable/varsutil_test.go +++ b/sessionctx/variable/varsutil_test.go @@ -277,13 +277,8 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) { c.Assert(err, IsNil) c.Assert(v.OptimizerSelectivityLevel, Equals, 1) - err = SetSessionSystemVar(v, TiDBDDLReorgWorkerCount, "-1") - c.Assert(terror.ErrorEqual(err, ErrWrongValueForVar), IsTrue) - - max := int64(maxDDLReorgWorkerCount) + 1 - err = SetSessionSystemVar(v, TiDBDDLReorgWorkerCount, strconv.FormatInt(max, 10)) - c.Assert(err, NotNil) - c.Assert(terror.ErrorEqual(err, ErrWrongValueForVar), IsTrue) + err = SetSessionSystemVar(v, TiDBDDLReorgWorkerCount, "4") // wrong scope global only + c.Assert(terror.ErrorEqual(err, errGlobalVariable), IsTrue) err = SetSessionSystemVar(v, TiDBRetryLimit, "3") c.Assert(err, IsNil) @@ -564,9 +559,6 @@ func (s *testVarsutilSuite) TestValidate(c *C) { {TiDBEnableTablePartition, "OFF", false}, {TiDBEnableTablePartition, "AUTO", false}, {TiDBEnableTablePartition, "UN", true}, - {TiDBEnableListTablePartition, "ON", false}, - {TiDBEnableListTablePartition, "OFF", false}, - {TiDBEnableListTablePartition, "list", true}, {TiDBOptCorrelationExpFactor, "a", true}, {TiDBOptCorrelationExpFactor, "-10", true}, {TiDBOptCorrelationThreshold, "a", true}, @@ -602,10 +594,6 @@ func (s *testVarsutilSuite) TestValidate(c *C) { {TiDBTxnMode, "pessimistic", false}, {TiDBTxnMode, "optimistic", false}, {TiDBTxnMode, "", false}, - {TiDBIsolationReadEngines, "", true}, - {TiDBIsolationReadEngines, "tikv", false}, - {TiDBIsolationReadEngines, "TiKV,tiflash", false}, - {TiDBIsolationReadEngines, " tikv, tiflash ", false}, {TiDBShardAllocateStep, "ad", true}, {TiDBShardAllocateStep, "-123", false}, {TiDBShardAllocateStep, "128", false}, @@ -629,6 +617,30 @@ func (s *testVarsutilSuite) TestValidate(c *C) { } } + // Test session scoped vars. + tests = []struct { + key string + value string + error bool + }{ + {TiDBEnableListTablePartition, "ON", false}, + {TiDBEnableListTablePartition, "OFF", false}, + {TiDBEnableListTablePartition, "list", true}, + {TiDBIsolationReadEngines, "", true}, + {TiDBIsolationReadEngines, "tikv", false}, + {TiDBIsolationReadEngines, "TiKV,tiflash", false}, + {TiDBIsolationReadEngines, " tikv, tiflash ", false}, + } + + for _, t := range tests { + _, err := GetSysVar(t.key).Validate(v, t.value, ScopeSession) + if t.error { + c.Assert(err, NotNil, Commentf("%v got err=%v", t, err)) + } else { + c.Assert(err, IsNil, Commentf("%v got err=%v", t, err)) + } + } + } func (s *testVarsutilSuite) TestValidateStmtSummary(c *C) { From bd011d3c9567c506d8d4343ade03edf77fcd5b56 Mon Sep 17 00:00:00 2001 From: Shirly Date: Sat, 8 May 2021 10:54:44 +0800 Subject: [PATCH 31/67] store/tikv: make tikv.ErrTiFlashServerTimeout as a normal error instead of dberror (#24385) --- server/conn.go | 8 ++-- server/conn_stmt.go | 4 +- store/copr/batch_coprocessor.go | 23 ++++++------ store/copr/batch_request_sender.go | 6 +-- store/copr/coprocessor.go | 30 +++++++-------- store/copr/coprocessor_test.go | 4 +- store/copr/mpp.go | 21 ++++++----- store/copr/store.go | 60 ++++++++++++++++++++++++++++++ store/driver/txn/error.go | 7 +++- store/tikv/error/errcode.go | 2 - store/tikv/error/error.go | 3 +- 11 files changed, 117 insertions(+), 51 deletions(-) diff --git a/server/conn.go b/server/conn.go index dc785280c144c..29c87bd0dfd86 100644 --- a/server/conn.go +++ b/server/conn.go @@ -76,7 +76,7 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" - tikverr "github.com/pingcap/tidb/store/tikv/error" + txndriver "github.com/pingcap/tidb/store/driver/txn" "github.com/pingcap/tidb/store/tikv/util" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/util/arena" @@ -1569,7 +1569,7 @@ func (cc *clientConn) handleQuery(ctx context.Context, sql string) (err error) { retryable, err = cc.handleStmt(ctx, stmt, parserWarns, i == len(stmts)-1) if err != nil { _, allowTiFlashFallback := cc.ctx.GetSessionVars().AllowFallbackToTiKV[kv.TiFlash] - if allowTiFlashFallback && errors.ErrorEqual(err, tikverr.ErrTiFlashServerTimeout) && retryable { + if allowTiFlashFallback && errors.ErrorEqual(err, txndriver.ErrTiFlashServerTimeout) && retryable { // When the TiFlash server seems down, we append a warning to remind the user to check the status of the TiFlash // server and fallback to TiKV. warns := append(parserWarns, stmtctx.SQLWarn{Level: stmtctx.WarnLevelError, Err: err}) @@ -1870,10 +1870,10 @@ func (cc *clientConn) writeChunks(ctx context.Context, rs ResultSet, binary bool failpoint.Inject("fetchNextErr", func(value failpoint.Value) { switch value.(string) { case "firstNext": - failpoint.Return(firstNext, tikverr.ErrTiFlashServerTimeout) + failpoint.Return(firstNext, txndriver.ErrTiFlashServerTimeout) case "secondNext": if !firstNext { - failpoint.Return(firstNext, tikverr.ErrTiFlashServerTimeout) + failpoint.Return(firstNext, txndriver.ErrTiFlashServerTimeout) } } }) diff --git a/server/conn_stmt.go b/server/conn_stmt.go index 63a9376e198dd..242b0df80fc83 100644 --- a/server/conn_stmt.go +++ b/server/conn_stmt.go @@ -50,7 +50,7 @@ import ( "github.com/pingcap/tidb/metrics" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx/stmtctx" - tikverr "github.com/pingcap/tidb/store/tikv/error" + txndriver "github.com/pingcap/tidb/store/driver/txn" "github.com/pingcap/tidb/store/tikv/util" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/execdetails" @@ -198,7 +198,7 @@ func (cc *clientConn) handleStmtExecute(ctx context.Context, data []byte) (err e ctx = context.WithValue(ctx, util.ExecDetailsKey, &util.ExecDetails{}) retryable, err := cc.executePreparedStmtAndWriteResult(ctx, stmt, args, useCursor) _, allowTiFlashFallback := cc.ctx.GetSessionVars().AllowFallbackToTiKV[kv.TiFlash] - if allowTiFlashFallback && err != nil && errors.ErrorEqual(err, tikverr.ErrTiFlashServerTimeout) && retryable { + if allowTiFlashFallback && err != nil && errors.ErrorEqual(err, txndriver.ErrTiFlashServerTimeout) && retryable { // When the TiFlash server seems down, we append a warning to remind the user to check the status of the TiFlash // server and fallback to TiKV. prevErr := err diff --git a/store/copr/batch_coprocessor.go b/store/copr/batch_coprocessor.go index 8a4336e0a06b6..5506c9d497ac2 100644 --- a/store/copr/batch_coprocessor.go +++ b/store/copr/batch_coprocessor.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/tidb/kv" + txndriver "github.com/pingcap/tidb/store/driver/txn" "github.com/pingcap/tidb/store/tikv" tikverr "github.com/pingcap/tidb/store/tikv/error" tikvstore "github.com/pingcap/tidb/store/tikv/kv" @@ -99,7 +100,7 @@ type copTaskAndRPCContext struct { ctx *tikv.RPCContext } -func buildBatchCopTasks(bo *tikv.Backoffer, cache *tikv.RegionCache, ranges *tikv.KeyRanges, storeType kv.StoreType) ([]*batchCopTask, error) { +func buildBatchCopTasks(bo *backoffer, cache *tikv.RegionCache, ranges *tikv.KeyRanges, storeType kv.StoreType) ([]*batchCopTask, error) { start := time.Now() const cmdType = tikvrpc.CmdBatchCop rangesLen := ranges.Len() @@ -114,7 +115,7 @@ func buildBatchCopTasks(bo *tikv.Backoffer, cache *tikv.RegionCache, ranges *tik }) } - err := tikv.SplitKeyRanges(bo, cache, ranges, appendTask) + err := tikv.SplitKeyRanges(bo.TiKVBackoffer(), cache, ranges, appendTask) if err != nil { return nil, errors.Trace(err) } @@ -124,7 +125,7 @@ func buildBatchCopTasks(bo *tikv.Backoffer, cache *tikv.RegionCache, ranges *tik storeTaskMap := make(map[string]*batchCopTask) needRetry := false for _, task := range tasks { - rpcCtx, err := cache.GetTiFlashRPCContext(bo, task.region, false) + rpcCtx, err := cache.GetTiFlashRPCContext(bo.TiKVBackoffer(), task.region, false) if err != nil { return nil, errors.Trace(err) } @@ -176,7 +177,7 @@ func (c *CopClient) sendBatch(ctx context.Context, req *kv.Request, vars *tikv.V return copErrorResponse{errors.New("batch coprocessor cannot prove keep order or desc property")} } ctx = context.WithValue(ctx, tikv.TxnStartKey, req.StartTs) - bo := tikv.NewBackofferWithVars(ctx, copBuildTaskMaxBackoff, vars) + bo := newBackofferWithVars(ctx, copBuildTaskMaxBackoff, vars) ranges := toTiKVKeyRanges(req.KeyRanges) tasks, err := buildBatchCopTasks(bo, c.store.GetRegionCache(), ranges, req.StoreType) if err != nil { @@ -227,7 +228,7 @@ func (b *batchCopIterator) run(ctx context.Context) { // We run workers for every batch cop. for _, task := range b.tasks { b.wg.Add(1) - bo := tikv.NewBackofferWithVars(ctx, copNextMaxBackoff, b.vars) + bo := newBackofferWithVars(ctx, copNextMaxBackoff, b.vars) go b.handleTask(ctx, bo, task) } b.wg.Wait() @@ -297,7 +298,7 @@ func (b *batchCopIterator) Close() error { return nil } -func (b *batchCopIterator) handleTask(ctx context.Context, bo *tikv.Backoffer, task *batchCopTask) { +func (b *batchCopIterator) handleTask(ctx context.Context, bo *backoffer, task *batchCopTask) { tasks := []*batchCopTask{task} for idx := 0; idx < len(tasks); idx++ { ret, err := b.handleTaskOnce(ctx, bo, tasks[idx]) @@ -312,7 +313,7 @@ func (b *batchCopIterator) handleTask(ctx context.Context, bo *tikv.Backoffer, t } // Merge all ranges and request again. -func (b *batchCopIterator) retryBatchCopTask(ctx context.Context, bo *tikv.Backoffer, batchTask *batchCopTask) ([]*batchCopTask, error) { +func (b *batchCopIterator) retryBatchCopTask(ctx context.Context, bo *backoffer, batchTask *batchCopTask) ([]*batchCopTask, error) { var ranges []tikvstore.KeyRange for _, taskCtx := range batchTask.copTasks { taskCtx.task.ranges.Do(func(ran *tikvstore.KeyRange) { @@ -322,7 +323,7 @@ func (b *batchCopIterator) retryBatchCopTask(ctx context.Context, bo *tikv.Backo return buildBatchCopTasks(bo, b.store.GetRegionCache(), tikv.NewKeyRanges(ranges), b.req.StoreType) } -func (b *batchCopIterator) handleTaskOnce(ctx context.Context, bo *tikv.Backoffer, task *batchCopTask) ([]*batchCopTask, error) { +func (b *batchCopIterator) handleTaskOnce(ctx context.Context, bo *backoffer, task *batchCopTask) ([]*batchCopTask, error) { sender := NewRegionBatchRequestSender(b.store.GetRegionCache(), b.store.GetTiKVClient()) var regionInfos = make([]*coprocessor.RegionInfo, 0, len(task.copTasks)) for _, task := range task.copTasks { @@ -367,7 +368,7 @@ func (b *batchCopIterator) handleTaskOnce(ctx context.Context, bo *tikv.Backoffe return nil, b.handleStreamedBatchCopResponse(ctx, bo, resp.Resp.(*tikvrpc.BatchCopStreamResponse), task) } -func (b *batchCopIterator) handleStreamedBatchCopResponse(ctx context.Context, bo *tikv.Backoffer, response *tikvrpc.BatchCopStreamResponse, task *batchCopTask) (err error) { +func (b *batchCopIterator) handleStreamedBatchCopResponse(ctx context.Context, bo *backoffer, response *tikvrpc.BatchCopStreamResponse, task *batchCopTask) (err error) { defer response.Close() resp := response.BatchResponse if resp == nil { @@ -395,12 +396,12 @@ func (b *batchCopIterator) handleStreamedBatchCopResponse(ctx context.Context, b } else { logutil.BgLogger().Info("stream unknown error", zap.Error(err)) } - return tikverr.ErrTiFlashServerTimeout + return txndriver.ErrTiFlashServerTimeout } } } -func (b *batchCopIterator) handleBatchCopResponse(bo *tikv.Backoffer, response *coprocessor.BatchResponse, task *batchCopTask) (err error) { +func (b *batchCopIterator) handleBatchCopResponse(bo *backoffer, response *coprocessor.BatchResponse, task *batchCopTask) (err error) { if otherErr := response.GetOtherError(); otherErr != "" { err = errors.Errorf("other error: %s", otherErr) logutil.BgLogger().Warn("other error", diff --git a/store/copr/batch_request_sender.go b/store/copr/batch_request_sender.go index 6a48d27fbb16a..bc2206b9780cb 100644 --- a/store/copr/batch_request_sender.go +++ b/store/copr/batch_request_sender.go @@ -38,7 +38,7 @@ func NewRegionBatchRequestSender(cache *tikv.RegionCache, client tikv.Client) *R } } -func (ss *RegionBatchRequestSender) sendStreamReqToAddr(bo *tikv.Backoffer, ctxs []copTaskAndRPCContext, req *tikvrpc.Request, timout time.Duration) (resp *tikvrpc.Response, retry bool, cancel func(), err error) { +func (ss *RegionBatchRequestSender) sendStreamReqToAddr(bo *backoffer, ctxs []copTaskAndRPCContext, req *tikvrpc.Request, timout time.Duration) (resp *tikvrpc.Response, retry bool, cancel func(), err error) { // use the first ctx to send request, because every ctx has same address. cancel = func() {} rpcCtx := ctxs[0].ctx @@ -67,7 +67,7 @@ func (ss *RegionBatchRequestSender) sendStreamReqToAddr(bo *tikv.Backoffer, ctxs return } -func (ss *RegionBatchRequestSender) onSendFail(bo *tikv.Backoffer, ctxs []copTaskAndRPCContext, err error) error { +func (ss *RegionBatchRequestSender) onSendFail(bo *backoffer, ctxs []copTaskAndRPCContext, err error) error { // If it failed because the context is cancelled by ourself, don't retry. if errors.Cause(err) == context.Canceled || status.Code(errors.Cause(err)) == codes.Canceled { return errors.Trace(err) @@ -78,7 +78,7 @@ func (ss *RegionBatchRequestSender) onSendFail(bo *tikv.Backoffer, ctxs []copTas for _, failedCtx := range ctxs { ctx := failedCtx.ctx if ctx.Meta != nil { - ss.GetRegionCache().OnSendFail(bo, ctx, ss.NeedReloadRegion(ctx), err) + ss.GetRegionCache().OnSendFail(bo.TiKVBackoffer(), ctx, ss.NeedReloadRegion(ctx), err) } } diff --git a/store/copr/coprocessor.go b/store/copr/coprocessor.go index d8d0bb1034e4f..cb799edb16f70 100644 --- a/store/copr/coprocessor.go +++ b/store/copr/coprocessor.go @@ -74,7 +74,7 @@ func (c *CopClient) Send(ctx context.Context, req *kv.Request, variables interfa return c.sendBatch(ctx, req, vars) } ctx = context.WithValue(ctx, tikv.TxnStartKey, req.StartTs) - bo := tikv.NewBackofferWithVars(ctx, copBuildTaskMaxBackoff, vars) + bo := newBackofferWithVars(ctx, copBuildTaskMaxBackoff, vars) ranges := toTiKVKeyRanges(req.KeyRanges) tasks, err := buildCopTasks(bo, c.store.GetRegionCache(), ranges, req) if err != nil { @@ -145,7 +145,7 @@ func (r *copTask) String() string { // rangesPerTask limits the length of the ranges slice sent in one copTask. const rangesPerTask = 25000 -func buildCopTasks(bo *tikv.Backoffer, cache *tikv.RegionCache, ranges *tikv.KeyRanges, req *kv.Request) ([]*copTask, error) { +func buildCopTasks(bo *backoffer, cache *tikv.RegionCache, ranges *tikv.KeyRanges, req *kv.Request) ([]*copTask, error) { start := time.Now() cmdType := tikvrpc.CmdCop if req.Streaming { @@ -177,7 +177,7 @@ func buildCopTasks(bo *tikv.Backoffer, cache *tikv.RegionCache, ranges *tikv.Key } } - err := tikv.SplitKeyRanges(bo, cache, ranges, appendTask) + err := tikv.SplitKeyRanges(bo.TiKVBackoffer(), cache, ranges, appendTask) if err != nil { return nil, errors.Trace(err) } @@ -606,12 +606,12 @@ func (it *copIterator) Next(ctx context.Context) (kv.ResultSubset, error) { // Associate each region with an independent backoffer. In this way, when multiple regions are // unavailable, TiDB can execute very quickly without blocking -func chooseBackoffer(ctx context.Context, backoffermap map[uint64]*tikv.Backoffer, task *copTask, worker *copIteratorWorker) *tikv.Backoffer { +func chooseBackoffer(ctx context.Context, backoffermap map[uint64]*backoffer, task *copTask, worker *copIteratorWorker) *backoffer { bo, ok := backoffermap[task.region.GetID()] if ok { return bo } - newbo := tikv.NewBackofferWithVars(ctx, copNextMaxBackoff, worker.vars) + newbo := newBackofferWithVars(ctx, copNextMaxBackoff, worker.vars) backoffermap[task.region.GetID()] = newbo return newbo } @@ -630,7 +630,7 @@ func (worker *copIteratorWorker) handleTask(ctx context.Context, task *copTask, } }() remainTasks := []*copTask{task} - backoffermap := make(map[uint64]*tikv.Backoffer) + backoffermap := make(map[uint64]*backoffer) for len(remainTasks) > 0 { curTask := remainTasks[0] bo := chooseBackoffer(ctx, backoffermap, curTask, worker) @@ -658,7 +658,7 @@ func (worker *copIteratorWorker) handleTask(ctx context.Context, task *copTask, // handleTaskOnce handles single copTask, successful results are send to channel. // If error happened, returns error. If region split or meet lock, returns the remain tasks. -func (worker *copIteratorWorker) handleTaskOnce(bo *tikv.Backoffer, task *copTask, ch chan<- *copResponse) ([]*copTask, error) { +func (worker *copIteratorWorker) handleTaskOnce(bo *backoffer, task *copTask, ch chan<- *copResponse) ([]*copTask, error) { failpoint.Inject("handleTaskOnceError", func(val failpoint.Value) { if val.(bool) { failpoint.Return(nil, errors.New("mock handleTaskOnce error")) @@ -717,7 +717,7 @@ func (worker *copIteratorWorker) handleTaskOnce(bo *tikv.Backoffer, task *copTas if len(worker.req.MatchStoreLabels) > 0 { ops = append(ops, tikv.WithMatchLabels(worker.req.MatchStoreLabels)) } - resp, rpcCtx, storeAddr, err := worker.kvclient.SendReqCtx(bo, req, task.region, tikv.ReadTimeoutMedium, getEndPointType(task.storeType), task.storeAddr, ops...) + resp, rpcCtx, storeAddr, err := worker.kvclient.SendReqCtx(bo.TiKVBackoffer(), req, task.region, tikv.ReadTimeoutMedium, getEndPointType(task.storeType), task.storeAddr, ops...) err = txndriver.ToTiDBErr(err) if err != nil { if task.storeType == kv.TiDB { @@ -748,10 +748,10 @@ const ( minLogKVProcessTime = 100 ) -func (worker *copIteratorWorker) logTimeCopTask(costTime time.Duration, task *copTask, bo *tikv.Backoffer, resp *tikvrpc.Response) { +func (worker *copIteratorWorker) logTimeCopTask(costTime time.Duration, task *copTask, bo *backoffer, resp *tikvrpc.Response) { logStr := fmt.Sprintf("[TIME_COP_PROCESS] resp_time:%s txnStartTS:%d region_id:%d store_addr:%s", costTime, worker.req.StartTs, task.region.GetID(), task.storeAddr) if bo.GetTotalSleep() > minLogBackoffTime { - backoffTypes := strings.Replace(fmt.Sprintf("%v", bo.GetTypes()), " ", ",", -1) + backoffTypes := strings.Replace(fmt.Sprintf("%v", bo.TiKVBackoffer().GetTypes()), " ", ",", -1) logStr += fmt.Sprintf(" backoff_ms:%d backoff_types:%s", bo.GetTotalSleep(), backoffTypes) } var detailV2 *kvrpcpb.ExecDetailsV2 @@ -810,7 +810,7 @@ func appendScanDetail(logStr string, columnFamily string, scanInfo *kvrpcpb.Scan return logStr } -func (worker *copIteratorWorker) handleCopStreamResult(bo *tikv.Backoffer, rpcCtx *tikv.RPCContext, stream *tikvrpc.CopStreamResponse, task *copTask, ch chan<- *copResponse, costTime time.Duration) ([]*copTask, error) { +func (worker *copIteratorWorker) handleCopStreamResult(bo *backoffer, rpcCtx *tikv.RPCContext, stream *tikvrpc.CopStreamResponse, task *copTask, ch chan<- *copResponse, costTime time.Duration) ([]*copTask, error) { defer stream.Close() var resp *coprocessor.Response var lastRange *coprocessor.KeyRange @@ -856,7 +856,7 @@ func (worker *copIteratorWorker) handleCopStreamResult(bo *tikv.Backoffer, rpcCt // returns more tasks when that happens, or handles the response if no error. // if we're handling streaming coprocessor response, lastRange is the range of last // successful response, otherwise it's nil. -func (worker *copIteratorWorker) handleCopResponse(bo *tikv.Backoffer, rpcCtx *tikv.RPCContext, resp *copResponse, cacheKey []byte, cacheValue *coprCacheValue, task *copTask, ch chan<- *copResponse, lastRange *coprocessor.KeyRange, costTime time.Duration) ([]*copTask, error) { +func (worker *copIteratorWorker) handleCopResponse(bo *backoffer, rpcCtx *tikv.RPCContext, resp *copResponse, cacheKey []byte, cacheValue *coprCacheValue, task *copTask, ch chan<- *copResponse, lastRange *coprocessor.KeyRange, costTime time.Duration) ([]*copTask, error) { if regionErr := resp.pbResp.GetRegionError(); regionErr != nil { if rpcCtx != nil && task.storeType == kv.TiDB { resp.err = errors.Errorf("error: %v", regionErr) @@ -874,7 +874,7 @@ func (worker *copIteratorWorker) handleCopResponse(bo *tikv.Backoffer, rpcCtx *t if lockErr := resp.pbResp.GetLocked(); lockErr != nil { logutil.BgLogger().Debug("coprocessor encounters", zap.Stringer("lock", lockErr)) - msBeforeExpired, err1 := worker.kvclient.ResolveLocks(bo, worker.req.StartTs, []*tikv.Lock{tikv.NewLock(lockErr)}) + msBeforeExpired, err1 := worker.kvclient.ResolveLocks(bo.TiKVBackoffer(), worker.req.StartTs, []*tikv.Lock{tikv.NewLock(lockErr)}) err1 = txndriver.ToTiDBErr(err1) if err1 != nil { return nil, errors.Trace(err1) @@ -987,7 +987,7 @@ func (worker *copIteratorWorker) handleTiDBSendReqErr(err error, task *copTask, errCode = errno.ErrTiKVServerTimeout errMsg = "TiDB server timeout, address is " + task.storeAddr } - if terror.ErrorEqual(err, tikverr.ErrTiFlashServerTimeout) { + if terror.ErrorEqual(err, txndriver.ErrTiFlashServerTimeout) { errCode = errno.ErrTiFlashServerTimeout errMsg = "TiDB server timeout, address is " + task.storeAddr } @@ -1013,7 +1013,7 @@ func (worker *copIteratorWorker) handleTiDBSendReqErr(err error, task *copTask, return nil } -func (worker *copIteratorWorker) buildCopTasksFromRemain(bo *tikv.Backoffer, lastRange *coprocessor.KeyRange, task *copTask) ([]*copTask, error) { +func (worker *copIteratorWorker) buildCopTasksFromRemain(bo *backoffer, lastRange *coprocessor.KeyRange, task *copTask) ([]*copTask, error) { remainedRanges := task.ranges if worker.req.Streaming && lastRange != nil { remainedRanges = worker.calculateRemain(task.ranges, lastRange, worker.req.Desc) diff --git a/store/copr/coprocessor_test.go b/store/copr/coprocessor_test.go index 73950f80390af..d7a6d52c5b4bb 100644 --- a/store/copr/coprocessor_test.go +++ b/store/copr/coprocessor_test.go @@ -43,7 +43,7 @@ func (s *testCoprocessorSuite) TestBuildTasks(c *C) { cache := tikv.NewRegionCache(pdCli) defer cache.Close() - bo := tikv.NewBackofferWithVars(context.Background(), 3000, nil) + bo := newBackofferWithVars(context.Background(), 3000, nil) req := &kv.Request{} flashReq := &kv.Request{} @@ -212,7 +212,7 @@ func (s *testCoprocessorSuite) TestRebuild(c *C) { pdCli := &tikv.CodecPDClient{Client: mocktikv.NewPDClient(cluster)} cache := tikv.NewRegionCache(pdCli) defer cache.Close() - bo := tikv.NewBackofferWithVars(context.Background(), 3000, nil) + bo := newBackofferWithVars(context.Background(), 3000, nil) req := &kv.Request{} tasks, err := buildCopTasks(bo, cache, buildCopRanges("a", "z"), req) diff --git a/store/copr/mpp.go b/store/copr/mpp.go index 3058f64849986..3ea07e744f9b9 100644 --- a/store/copr/mpp.go +++ b/store/copr/mpp.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/mpp" "github.com/pingcap/tidb/kv" + txndriver "github.com/pingcap/tidb/store/driver/txn" "github.com/pingcap/tidb/store/tikv" tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/store/tikv/logutil" @@ -56,7 +57,7 @@ func (c *MPPClient) selectAllTiFlashStore() []kv.MPPTaskMeta { // ConstructMPPTasks receives ScheduleRequest, which are actually collects of kv ranges. We allocates MPPTaskMeta for them and returns. func (c *MPPClient) ConstructMPPTasks(ctx context.Context, req *kv.MPPBuildTasksRequest) ([]kv.MPPTaskMeta, error) { ctx = context.WithValue(ctx, tikv.TxnStartKey, req.StartTS) - bo := tikv.NewBackofferWithVars(ctx, copBuildTaskMaxBackoff, nil) + bo := newBackofferWithVars(ctx, copBuildTaskMaxBackoff, nil) if req.KeyRanges == nil { return c.selectAllTiFlashStore(), nil } @@ -152,7 +153,7 @@ func (m *mppIterator) run(ctx context.Context) { break } m.wg.Add(1) - bo := tikv.NewBackoffer(ctx, copNextMaxBackoff) + bo := newBackoffer(ctx, copNextMaxBackoff) go m.handleDispatchReq(ctx, bo, task) } m.wg.Wait() @@ -176,7 +177,7 @@ func (m *mppIterator) sendToRespCh(resp *mppResponse) (exit bool) { // TODO:: Consider that which way is better: // - dispatch all tasks at once, and connect tasks at second. // - dispatch tasks and establish connection at the same time. -func (m *mppIterator) handleDispatchReq(ctx context.Context, bo *tikv.Backoffer, req *kv.MPPDispatchRequest) { +func (m *mppIterator) handleDispatchReq(ctx context.Context, bo *backoffer, req *kv.MPPDispatchRequest) { defer func() { m.wg.Done() }() @@ -225,7 +226,7 @@ func (m *mppIterator) handleDispatchReq(ctx context.Context, bo *tikv.Backoffer, if sender.GetRPCError() != nil { logutil.BgLogger().Error("mpp dispatch meet io error", zap.String("error", sender.GetRPCError().Error())) // we return timeout to trigger tikv's fallback - m.sendError(tikverr.ErrTiFlashServerTimeout) + m.sendError(txndriver.ErrTiFlashServerTimeout) return } } else { @@ -235,7 +236,7 @@ func (m *mppIterator) handleDispatchReq(ctx context.Context, bo *tikv.Backoffer, if err != nil { logutil.BgLogger().Error("mpp dispatch meet error", zap.String("error", err.Error())) // we return timeout to trigger tikv's fallback - m.sendError(tikverr.ErrTiFlashServerTimeout) + m.sendError(txndriver.ErrTiFlashServerTimeout) return } @@ -255,7 +256,7 @@ func (m *mppIterator) handleDispatchReq(ctx context.Context, bo *tikv.Backoffer, failpoint.Inject("mppNonRootTaskError", func(val failpoint.Value) { if val.(bool) && !req.IsRoot { time.Sleep(1 * time.Second) - m.sendError(tikverr.ErrTiFlashServerTimeout) + m.sendError(txndriver.ErrTiFlashServerTimeout) return } }) @@ -299,7 +300,7 @@ func (m *mppIterator) cancelMppTasks() { } } -func (m *mppIterator) establishMPPConns(bo *tikv.Backoffer, req *kv.MPPDispatchRequest, taskMeta *mpp.TaskMeta) { +func (m *mppIterator) establishMPPConns(bo *backoffer, req *kv.MPPDispatchRequest, taskMeta *mpp.TaskMeta) { connReq := &mpp.EstablishMPPConnectionRequest{ SenderMeta: taskMeta, ReceiverMeta: &mpp.TaskMeta{ @@ -318,7 +319,7 @@ func (m *mppIterator) establishMPPConns(bo *tikv.Backoffer, req *kv.MPPDispatchR if err != nil { logutil.BgLogger().Error("establish mpp connection meet error", zap.String("error", err.Error())) // we return timeout to trigger tikv's fallback - m.sendError(tikverr.ErrTiFlashServerTimeout) + m.sendError(txndriver.ErrTiFlashServerTimeout) return } @@ -350,7 +351,7 @@ func (m *mppIterator) establishMPPConns(bo *tikv.Backoffer, req *kv.MPPDispatchR logutil.BgLogger().Info("stream unknown error", zap.Error(err)) } } - m.sendError(tikverr.ErrTiFlashServerTimeout) + m.sendError(txndriver.ErrTiFlashServerTimeout) return } } @@ -366,7 +367,7 @@ func (m *mppIterator) Close() error { return nil } -func (m *mppIterator) handleMPPStreamResponse(bo *tikv.Backoffer, response *mpp.MPPDataPacket, req *kv.MPPDispatchRequest) (err error) { +func (m *mppIterator) handleMPPStreamResponse(bo *backoffer, response *mpp.MPPDataPacket, req *kv.MPPDispatchRequest) (err error) { if response.Error != nil { err = errors.Errorf("other error for mpp stream: %s", response.Error.Msg) logutil.BgLogger().Warn("other error", diff --git a/store/copr/store.go b/store/copr/store.go index 8dd6825de06fb..2cc10ee7bad38 100644 --- a/store/copr/store.go +++ b/store/copr/store.go @@ -121,3 +121,63 @@ func getEndPointType(t kv.StoreType) tikvrpc.EndpointType { return tikvrpc.TiKV } } + +// backoffer wraps tikv.Backoffer and converts the error which returns by the functions of tikv.Backoffer to tidb error. +type backoffer struct { + b *tikv.Backoffer +} + +// newBackofferWithVars creates a Backoffer with maximum sleep time(in ms) and kv.Variables. +func newBackofferWithVars(ctx context.Context, maxSleep int, vars *kv.Variables) *backoffer { + b := tikv.NewBackofferWithVars(ctx, maxSleep, vars) + return &backoffer{b: b} +} + +func newBackoffer(ctx context.Context, maxSleep int) *backoffer { + b := tikv.NewBackoffer(ctx, maxSleep) + return &backoffer{b: b} +} + +// TiKVBackoffer returns tikv.Backoffer. +func (b *backoffer) TiKVBackoffer() *tikv.Backoffer { + return b.b +} + +// Backoff sleeps a while base on the backoffType and records the error message. +// It returns a retryable error if total sleep time exceeds maxSleep. +func (b *backoffer) Backoff(typ tikv.BackoffType, err error) error { + e := b.b.Backoff(typ, err) + return txndriver.ToTiDBErr(e) +} + +// BackoffWithMaxSleep sleeps a while base on the backoffType and records the error message +// and never sleep more than maxSleepMs for each sleep. +func (b *backoffer) BackoffWithMaxSleep(typ tikv.BackoffType, maxSleepMs int, err error) error { + e := b.b.BackoffWithMaxSleep(typ, maxSleepMs, err) + return txndriver.ToTiDBErr(e) +} + +// GetBackoffTimes returns a map contains backoff time count by type. +func (b *backoffer) GetBackoffTimes() map[tikv.BackoffType]int { + return b.b.GetBackoffTimes() +} + +// GetCtx returns the binded context. +func (b *backoffer) GetCtx() context.Context { + return b.b.GetCtx() +} + +// GetVars returns the binded vars. +func (b *backoffer) GetVars() *tikv.Variables { + return b.b.GetVars() +} + +// GetBackoffSleepMS returns a map contains backoff sleep time by type. +func (b *backoffer) GetBackoffSleepMS() map[tikv.BackoffType]int { + return b.b.GetBackoffSleepMS() +} + +// GetTotalSleep returns total sleep time. +func (b *backoffer) GetTotalSleep() int { + return b.b.GetTotalSleep() +} diff --git a/store/driver/txn/error.go b/store/driver/txn/error.go index 03e4b3c42d39f..6794a016a1060 100644 --- a/store/driver/txn/error.go +++ b/store/driver/txn/error.go @@ -39,7 +39,8 @@ import ( // tikv error instance var ( // ErrTiKVServerTimeout is the error when tikv server is timeout. - ErrTiKVServerTimeout = dbterror.ClassTiKV.NewStd(errno.ErrTiKVServerTimeout) + ErrTiKVServerTimeout = dbterror.ClassTiKV.NewStd(errno.ErrTiKVServerTimeout) + ErrTiFlashServerTimeout = dbterror.ClassTiKV.NewStd(errno.ErrTiFlashServerTimeout) // ErrGCTooEarly is the error that GC life time is shorter than transaction duration ErrGCTooEarly = dbterror.ClassTiKV.NewStd(errno.ErrGCTooEarly) // ErrTiKVStaleCommand is the error that the command is stale in tikv. @@ -213,6 +214,10 @@ func ToTiDBErr(err error) error { return ErrPDServerTimeout.GenWithStackByArgs(e.Error()) } + if errors.ErrorEqual(err, tikverr.ErrTiFlashServerTimeout) { + return ErrTiFlashServerTimeout + } + if errors.ErrorEqual(err, tikverr.ErrTiKVServerBusy) { return ErrTiKVServerBusy } diff --git a/store/tikv/error/errcode.go b/store/tikv/error/errcode.go index 070a49deb8b88..10cc2292be7e7 100644 --- a/store/tikv/error/errcode.go +++ b/store/tikv/error/errcode.go @@ -26,6 +26,4 @@ const ( // TiKV/PD/TiFlash errors. CodeTiKVStoreLimit = 9008 - - CodeTiFlashServerTimeout = 9012 ) diff --git a/store/tikv/error/error.go b/store/tikv/error/error.go index 571cfbc55b898..a4f213ab09a02 100644 --- a/store/tikv/error/error.go +++ b/store/tikv/error/error.go @@ -36,6 +36,8 @@ var ( ErrInvalidTxn = errors.New("invalid transaction") // ErrTiKVServerTimeout is the error when tikv server is timeout. ErrTiKVServerTimeout = errors.New("tikv server timeout") + // ErrTiFlashServerTimeout is the error when tiflash server is timeout. + ErrTiFlashServerTimeout = errors.New("tiflash server timeout") // ErrTiKVStaleCommand is the error that the command is stale in tikv. ErrTiKVStaleCommand = errors.New("tikv stale command") // ErrTiKVMaxTimestampNotSynced is the error that tikv's max timestamp is not synced. @@ -55,7 +57,6 @@ const MismatchClusterID = "mismatch cluster id" // error instances. var ( - ErrTiFlashServerTimeout = dbterror.ClassTiKV.NewStd(CodeTiFlashServerTimeout) ErrQueryInterrupted = dbterror.ClassTiKV.NewStd(CodeQueryInterrupted) ErrLockAcquireFailAndNoWaitSet = dbterror.ClassTiKV.NewStd(CodeLockAcquireFailAndNoWaitSet) ErrLockWaitTimeout = dbterror.ClassTiKV.NewStd(CodeLockWaitTimeout) From 82ea46d652970bb564a43d3effc04334f3bf9d0b Mon Sep 17 00:00:00 2001 From: Morgan Tocker Date: Fri, 7 May 2021 21:18:43 -0600 Subject: [PATCH 32/67] sessionctx/variable: Improve sysvar test coverage, collation normalization (#24472) --- sessionctx/variable/sysvar_test.go | 91 ++++++++++++++++++++++++++++++ sessionctx/variable/varsutil.go | 5 +- 2 files changed, 94 insertions(+), 2 deletions(-) diff --git a/sessionctx/variable/sysvar_test.go b/sessionctx/variable/sysvar_test.go index 31e44a88ad83f..4029072412977 100644 --- a/sessionctx/variable/sysvar_test.go +++ b/sessionctx/variable/sysvar_test.go @@ -15,6 +15,7 @@ package variable import ( "fmt" + "strings" "testing" . "github.com/pingcap/check" @@ -227,3 +228,93 @@ func (*testSysVarSuite) TestScope(c *C) { c.Assert(sv.HasSessionScope(), IsFalse) c.Assert(sv.HasGlobalScope(), IsFalse) } + +func (*testSysVarSuite) TestBuiltInCase(c *C) { + // All Sysvars should have lower case names. + // This tests builtins. + for name := range GetSysVars() { + c.Assert(name, Equals, strings.ToLower(name)) + } +} + +func (*testSysVarSuite) TestSQLSelectLimit(c *C) { + sv := GetSysVar(SQLSelectLimit) + vars := NewSessionVars() + val, err := sv.Validate(vars, "-10", ScopeSession) + c.Assert(err, IsNil) // it has autoconvert out of range. + c.Assert(val, Equals, "0") + + val, err = sv.Validate(vars, "9999", ScopeSession) + c.Assert(err, IsNil) + c.Assert(val, Equals, "9999") + + c.Assert(sv.SetSessionFromHook(vars, "9999"), IsNil) // sets + c.Assert(vars.SelectLimit, Equals, uint64(9999)) +} + +func (*testSysVarSuite) TestSQLModeVar(c *C) { + sv := GetSysVar(SQLModeVar) + vars := NewSessionVars() + val, err := sv.Validate(vars, "strict_trans_tabLES ", ScopeSession) + c.Assert(err, IsNil) + c.Assert(val, Equals, "STRICT_TRANS_TABLES") + + _, err = sv.Validate(vars, "strict_trans_tabLES,nonsense_option", ScopeSession) + c.Assert(err.Error(), Equals, "ERROR 1231 (42000): Variable 'sql_mode' can't be set to the value of 'NONSENSE_OPTION'") + + val, err = sv.Validate(vars, "ONLY_FULL_GROUP_BY,STRICT_TRANS_TABLES,NO_ZERO_IN_DATE,NO_ZERO_DATE,ERROR_FOR_DIVISION_BY_ZERO,NO_AUTO_CREATE_USER,NO_ENGINE_SUBSTITUTION", ScopeSession) + c.Assert(err, IsNil) + c.Assert(val, Equals, "ONLY_FULL_GROUP_BY,STRICT_TRANS_TABLES,NO_ZERO_IN_DATE,NO_ZERO_DATE,ERROR_FOR_DIVISION_BY_ZERO,NO_AUTO_CREATE_USER,NO_ENGINE_SUBSTITUTION") + + c.Assert(sv.SetSessionFromHook(vars, val), IsNil) // sets to strict from above + c.Assert(vars.StrictSQLMode, IsTrue) + + sqlMode, err := mysql.GetSQLMode(val) + c.Assert(err, IsNil) + c.Assert(vars.SQLMode, Equals, sqlMode) + + // Set it to non strict. + val, err = sv.Validate(vars, "ERROR_FOR_DIVISION_BY_ZERO,NO_AUTO_CREATE_USER,NO_ENGINE_SUBSTITUTION", ScopeSession) + c.Assert(err, IsNil) + c.Assert(val, Equals, "ERROR_FOR_DIVISION_BY_ZERO,NO_AUTO_CREATE_USER,NO_ENGINE_SUBSTITUTION") + + c.Assert(sv.SetSessionFromHook(vars, val), IsNil) // sets to non-strict from above + c.Assert(vars.StrictSQLMode, IsFalse) + sqlMode, err = mysql.GetSQLMode(val) + c.Assert(err, IsNil) + c.Assert(vars.SQLMode, Equals, sqlMode) +} + +func (*testSysVarSuite) TestMaxExecutionTime(c *C) { + sv := GetSysVar(MaxExecutionTime) + vars := NewSessionVars() + + val, err := sv.Validate(vars, "-10", ScopeSession) + c.Assert(err, IsNil) // it has autoconvert out of range. + c.Assert(val, Equals, "0") + + val, err = sv.Validate(vars, "99999", ScopeSession) + c.Assert(err, IsNil) // it has autoconvert out of range. + c.Assert(val, Equals, "99999") + + c.Assert(sv.SetSessionFromHook(vars, "99999"), IsNil) // sets + c.Assert(vars.MaxExecutionTime, Equals, uint64(99999)) +} + +func (*testSysVarSuite) TestCollationServer(c *C) { + sv := GetSysVar(CollationServer) + vars := NewSessionVars() + + val, err := sv.Validate(vars, "LATIN1_bin", ScopeSession) + c.Assert(err, IsNil) + c.Assert(val, Equals, "latin1_bin") // test normalization + + _, err = sv.Validate(vars, "BOGUSCOLLation", ScopeSession) + c.Assert(err.Error(), Equals, "[ddl:1273]Unknown collation: 'BOGUSCOLLation'") + + c.Assert(sv.SetSessionFromHook(vars, "latin1_bin"), IsNil) + c.Assert(vars.systems[CharacterSetServer], Equals, "latin1") // check it also changes charset. + + c.Assert(sv.SetSessionFromHook(vars, "utf8mb4_bin"), IsNil) + c.Assert(vars.systems[CharacterSetServer], Equals, "utf8mb4") // check it also changes charset. +} diff --git a/sessionctx/variable/varsutil.go b/sessionctx/variable/varsutil.go index bb9f0e1841e54..45cebbd62e0db 100644 --- a/sessionctx/variable/varsutil.go +++ b/sessionctx/variable/varsutil.go @@ -111,10 +111,11 @@ func int32ToBoolStr(i int32) string { } func checkCollation(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { - if _, err := collate.GetCollationByName(normalizedValue); err != nil { + coll, err := collate.GetCollationByName(normalizedValue) + if err != nil { return normalizedValue, errors.Trace(err) } - return normalizedValue, nil + return coll.Name, nil } func checkCharacterSet(normalizedValue string, argName string) (string, error) { From b0dd4dae2a42aaa8d191a01e8942e7f91243c7d1 Mon Sep 17 00:00:00 2001 From: amyangfei Date: Sat, 8 May 2021 15:24:41 +0800 Subject: [PATCH 33/67] tikv: add id to version map to accelerate get region by id (#24403) --- store/tikv/region_cache.go | 71 +++++++++++++++++++-------------- store/tikv/region_cache_test.go | 16 ++++++++ 2 files changed, 58 insertions(+), 29 deletions(-) diff --git a/store/tikv/region_cache.go b/store/tikv/region_cache.go index c06394c6b166c..a73684fdf49c5 100644 --- a/store/tikv/region_cache.go +++ b/store/tikv/region_cache.go @@ -275,9 +275,10 @@ type RegionCache struct { enableForwarding bool mu struct { - sync.RWMutex // mutex protect cached region - regions map[RegionVerID]*Region // cached regions are organized as regionVerID to region ref mapping - sorted *btree.BTree // cache regions are organized as sorted key to region ref mapping + sync.RWMutex // mutex protect cached region + regions map[RegionVerID]*Region // cached regions are organized as regionVerID to region ref mapping + latestVersions map[uint64]RegionVerID // cache the map from regionID to its latest RegionVerID + sorted *btree.BTree // cache regions are organized as sorted key to region ref mapping } storeMu struct { sync.RWMutex @@ -299,6 +300,7 @@ func NewRegionCache(pdClient pd.Client) *RegionCache { pdClient: pdClient, } c.mu.regions = make(map[RegionVerID]*Region) + c.mu.latestVersions = make(map[uint64]RegionVerID) c.mu.sorted = btree.New(btreeDegree) c.storeMu.stores = make(map[uint64]*Store) c.notifyCheckCh = make(chan struct{}, 1) @@ -976,6 +978,15 @@ func (c *RegionCache) UpdateLeader(regionID RegionVerID, leaderStoreID uint64, c } } +// removeVersionFromCache removes a RegionVerID from cache, tries to cleanup +// both c.mu.regions and c.mu.versions. Note this function is not thread-safe. +func (c *RegionCache) removeVersionFromCache(oldVer RegionVerID, regionID uint64) { + delete(c.mu.regions, oldVer) + if ver, ok := c.mu.latestVersions[regionID]; ok && ver.Equals(oldVer) { + delete(c.mu.latestVersions, regionID) + } +} + // insertRegionToCache tries to insert the Region to cache. // It should be protected by c.mu.Lock(). func (c *RegionCache) insertRegionToCache(cachedRegion *Region) { @@ -995,9 +1006,14 @@ func (c *RegionCache) insertRegionToCache(cachedRegion *Region) { // Don't refresh TiFlash work idx for region. Otherwise, it will always goto a invalid store which // is under transferring regions. store.workTiFlashIdx = atomic.LoadInt32(&oldRegionStore.workTiFlashIdx) - delete(c.mu.regions, oldRegion.VerID()) + c.removeVersionFromCache(oldRegion.VerID(), cachedRegion.VerID().id) } c.mu.regions[cachedRegion.VerID()] = cachedRegion + newVer := cachedRegion.VerID() + latest, ok := c.mu.latestVersions[cachedRegion.VerID().id] + if !ok || latest.GetVer() < newVer.GetVer() || latest.GetConfVer() < newVer.GetConfVer() { + c.mu.latestVersions[cachedRegion.VerID().id] = newVer + } } // searchCachedRegion finds a region from cache by key. Like `getCachedRegion`, @@ -1032,34 +1048,26 @@ func (c *RegionCache) searchCachedRegion(key []byte, isEndKey bool) *Region { // `getCachedRegion`, it should be called with c.mu.RLock(), and the returned // Region should not be used after c.mu is RUnlock(). func (c *RegionCache) getRegionByIDFromCache(regionID uint64) *Region { - var newestRegion *Region ts := time.Now().Unix() - for v, r := range c.mu.regions { - if v.id == regionID { - lastAccess := atomic.LoadInt64(&r.lastAccess) - if ts-lastAccess > RegionCacheTTLSec { - continue - } - if newestRegion == nil { - newestRegion = r - continue - } - nv := newestRegion.VerID() - cv := r.VerID() - if nv.GetConfVer() < cv.GetConfVer() { - newestRegion = r - continue - } - if nv.GetVer() < cv.GetVer() { - newestRegion = r - continue - } - } + ver, ok := c.mu.latestVersions[regionID] + if !ok { + return nil } - if newestRegion != nil { - atomic.CompareAndSwapInt64(&newestRegion.lastAccess, atomic.LoadInt64(&newestRegion.lastAccess), ts) + latestRegion, ok := c.mu.regions[ver] + if !ok { + // should not happen + logutil.BgLogger().Warn("region version not found", + zap.Uint64("regionID", regionID), zap.Stringer("version", &ver)) + return nil } - return newestRegion + lastAccess := atomic.LoadInt64(&latestRegion.lastAccess) + if ts-lastAccess > RegionCacheTTLSec { + return nil + } + if latestRegion != nil { + atomic.CompareAndSwapInt64(&latestRegion.lastAccess, atomic.LoadInt64(&latestRegion.lastAccess), ts) + } + return latestRegion } // TODO: revise it by get store by closure. @@ -1570,6 +1578,11 @@ func (r *RegionVerID) String() string { return fmt.Sprintf("{ region id: %v, ver: %v, confVer: %v }", r.id, r.ver, r.confVer) } +// Equals checks whether the RegionVerID equals to another one +func (r *RegionVerID) Equals(another RegionVerID) bool { + return r.id == another.id && r.confVer == another.confVer && r.ver == another.ver +} + // VerID returns the Region's RegionVerID. func (r *Region) VerID() RegionVerID { return RegionVerID{ diff --git a/store/tikv/region_cache_test.go b/store/tikv/region_cache_test.go index 592541eb5b9d2..efb2ae9df73ab 100644 --- a/store/tikv/region_cache_test.go +++ b/store/tikv/region_cache_test.go @@ -69,9 +69,25 @@ func (s *testRegionCacheSuite) storeAddr(id uint64) string { func (s *testRegionCacheSuite) checkCache(c *C, len int) { ts := time.Now().Unix() c.Assert(validRegions(s.cache.mu.regions, ts), Equals, len) + c.Assert(validRegionsSearchedByVersions(s.cache.mu.latestVersions, s.cache.mu.regions, ts), Equals, len) c.Assert(validRegionsInBtree(s.cache.mu.sorted, ts), Equals, len) } +func validRegionsSearchedByVersions( + versions map[uint64]RegionVerID, + regions map[RegionVerID]*Region, + ts int64, +) (count int) { + for _, ver := range versions { + region, ok := regions[ver] + if !ok || !region.checkRegionCacheTTL(ts) { + continue + } + count++ + } + return +} + func validRegions(regions map[RegionVerID]*Region, ts int64) (len int) { for _, region := range regions { if !region.checkRegionCacheTTL(ts) { From 8ed1d9d4a7987aa87b586020795d2587286402bf Mon Sep 17 00:00:00 2001 From: djshow832 Date: Sat, 8 May 2021 16:36:41 +0800 Subject: [PATCH 34/67] meta: add an in-memory autoID allocator (#24438) --- meta/autoid/errors.go | 1 + meta/autoid/memid.go | 145 ++++++++++++++++++++++++++++++++++++++ meta/autoid/memid_test.go | 106 ++++++++++++++++++++++++++++ 3 files changed, 252 insertions(+) create mode 100644 meta/autoid/memid.go create mode 100644 meta/autoid/memid_test.go diff --git a/meta/autoid/errors.go b/meta/autoid/errors.go index 093e0abbdea53..777de466b7704 100644 --- a/meta/autoid/errors.go +++ b/meta/autoid/errors.go @@ -22,6 +22,7 @@ import ( var ( errInvalidTableID = dbterror.ClassAutoid.NewStd(mysql.ErrInvalidTableID) errInvalidIncrementAndOffset = dbterror.ClassAutoid.NewStd(mysql.ErrInvalidIncrementAndOffset) + errNotImplemented = dbterror.ClassAutoid.NewStd(mysql.ErrNotImplemented) ErrAutoincReadFailed = dbterror.ClassAutoid.NewStd(mysql.ErrAutoincReadFailed) ErrWrongAutoKey = dbterror.ClassAutoid.NewStd(mysql.ErrWrongAutoKey) ErrInvalidAllocatorType = dbterror.ClassAutoid.NewStd(mysql.ErrUnknownAllocatorType) diff --git a/meta/autoid/memid.go b/meta/autoid/memid.go new file mode 100644 index 0000000000000..703ffe1db4bf8 --- /dev/null +++ b/meta/autoid/memid.go @@ -0,0 +1,145 @@ +// 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, +// See the License for the specific language governing permissions and +// limitations under the License. + +package autoid + +import ( + "context" + "math" + + "github.com/pingcap/parser/model" +) + +// NewAllocatorFromTempTblInfo creates an in-memory allocator from a temporary table info. +func NewAllocatorFromTempTblInfo(tblInfo *model.TableInfo) Allocator { + hasRowID := !tblInfo.PKIsHandle && !tblInfo.IsCommonHandle + hasAutoIncID := tblInfo.GetAutoIncrementColInfo() != nil + // Temporary tables don't support auto_random and sequence. + if hasRowID || hasAutoIncID { + return &inMemoryAllocator{ + isUnsigned: tblInfo.IsAutoIncColUnsigned(), + allocType: RowIDAllocType, + } + } + return nil +} + +// inMemoryAllocator is typically used for temporary tables. +// Some characteristics: +// - It allocates IDs from memory. +// - It's session-wide and thus won't be visited concurrently. +// - It doesn't support sequence. +// - The metrics are not reported. +type inMemoryAllocator struct { + base int64 + isUnsigned bool + allocType AllocatorType +} + +// Base implements autoid.Allocator Base interface. +func (alloc *inMemoryAllocator) Base() int64 { + return alloc.base +} + +// End implements autoid.Allocator End interface. +func (alloc *inMemoryAllocator) End() int64 { + // It doesn't matter. + return 0 +} + +// GetType implements autoid.Allocator GetType interface. +func (alloc *inMemoryAllocator) GetType() AllocatorType { + return alloc.allocType +} + +// NextGlobalAutoID implements autoid.Allocator NextGlobalAutoID interface. +func (alloc *inMemoryAllocator) NextGlobalAutoID(tableID int64) (int64, error) { + return 0, errNotImplemented.GenWithStackByArgs() +} + +func (alloc *inMemoryAllocator) Alloc(ctx context.Context, tableID int64, n uint64, increment, offset int64) (int64, int64, error) { + if n == 0 { + return 0, 0, nil + } + if alloc.allocType == AutoIncrementType || alloc.allocType == RowIDAllocType { + if !validIncrementAndOffset(increment, offset) { + return 0, 0, errInvalidIncrementAndOffset.GenWithStackByArgs(increment, offset) + } + } + if alloc.isUnsigned { + return alloc.alloc4Unsigned(n, increment, offset) + } + return alloc.alloc4Signed(n, increment, offset) +} + +// Rebase implements autoid.Allocator Rebase interface. +// The requiredBase is the minimum base value after Rebase. +// The real base may be greater than the required base. +func (alloc *inMemoryAllocator) Rebase(tableID, requiredBase int64, allocIDs bool) error { + if alloc.isUnsigned { + if uint64(requiredBase) > uint64(alloc.base) { + alloc.base = requiredBase + } + } else { + if requiredBase > alloc.base { + alloc.base = requiredBase + } + } + return nil +} + +func (alloc *inMemoryAllocator) alloc4Signed(n uint64, increment, offset int64) (int64, int64, error) { + // Check offset rebase if necessary. + if offset-1 > alloc.base { + alloc.base = offset - 1 + } + // CalcNeededBatchSize calculates the total batch size needed. + n1 := CalcNeededBatchSize(alloc.base, int64(n), increment, offset, alloc.isUnsigned) + + // Condition alloc.base+N1 > alloc.end will overflow when alloc.base + N1 > MaxInt64. So need this. + if math.MaxInt64-alloc.base <= n1 { + return 0, 0, ErrAutoincReadFailed + } + + min := alloc.base + alloc.base += n1 + return min, alloc.base, nil +} + +func (alloc *inMemoryAllocator) alloc4Unsigned(n uint64, increment, offset int64) (int64, int64, error) { + // Check offset rebase if necessary. + if uint64(offset)-1 > uint64(alloc.base) { + alloc.base = int64(uint64(offset) - 1) + } + + // CalcNeededBatchSize calculates the total batch size needed. + n1 := CalcNeededBatchSize(alloc.base, int64(n), increment, offset, alloc.isUnsigned) + + // Condition alloc.base+n1 > alloc.end will overflow when alloc.base + n1 > MaxInt64. So need this. + if math.MaxUint64-uint64(alloc.base) <= uint64(n1) { + return 0, 0, ErrAutoincReadFailed + } + + min := alloc.base + // Use uint64 n directly. + alloc.base = int64(uint64(alloc.base) + uint64(n1)) + return min, alloc.base, nil +} + +func (alloc *inMemoryAllocator) AllocSeqCache(tableID int64) (int64, int64, int64, error) { + return 0, 0, 0, errNotImplemented.GenWithStackByArgs() +} + +func (alloc *inMemoryAllocator) RebaseSeq(tableID, requiredBase int64) (int64, bool, error) { + return 0, false, errNotImplemented.GenWithStackByArgs() +} diff --git a/meta/autoid/memid_test.go b/meta/autoid/memid_test.go new file mode 100644 index 0000000000000..f4b1267ad8f99 --- /dev/null +++ b/meta/autoid/memid_test.go @@ -0,0 +1,106 @@ +// 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, +// See the License for the specific language governing permissions and +// limitations under the License. + +package autoid_test + +import ( + "context" + "math" + + . "github.com/pingcap/check" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" + "github.com/pingcap/parser/types" + "github.com/pingcap/tidb/meta/autoid" + "github.com/pingcap/tidb/store/mockstore" +) + +func (*testSuite) TestInMemoryAlloc(c *C) { + store, err := mockstore.NewMockStore() + c.Assert(err, IsNil) + defer func() { + err := store.Close() + c.Assert(err, IsNil) + }() + + columnInfo := &model.ColumnInfo{ + FieldType: types.FieldType{ + Flag: mysql.AutoIncrementFlag, + }, + } + tblInfo := &model.TableInfo{ + Columns: []*model.ColumnInfo{columnInfo}, + } + alloc := autoid.NewAllocatorFromTempTblInfo(tblInfo) + c.Assert(alloc, NotNil) + + // alloc 1 + ctx := context.Background() + _, id, err := alloc.Alloc(ctx, 1, 1, 1, 1) + c.Assert(err, IsNil) + c.Assert(id, Equals, int64(1)) + _, id, err = alloc.Alloc(ctx, 1, 1, 1, 1) + c.Assert(err, IsNil) + c.Assert(id, Equals, int64(2)) + + // alloc N + _, id, err = alloc.Alloc(ctx, 1, 10, 1, 1) + c.Assert(err, IsNil) + c.Assert(id, Equals, int64(12)) + + // increment > N + _, id, err = alloc.Alloc(ctx, 1, 1, 10, 1) + c.Assert(err, IsNil) + c.Assert(id, Equals, int64(21)) + + // offset + _, id, err = alloc.Alloc(ctx, 1, 1, 1, 30) + c.Assert(err, IsNil) + c.Assert(id, Equals, int64(30)) + + // rebase + err = alloc.Rebase(1, int64(40), true) + c.Assert(err, IsNil) + _, id, err = alloc.Alloc(ctx, 1, 1, 1, 1) + c.Assert(err, IsNil) + c.Assert(id, Equals, int64(41)) + err = alloc.Rebase(1, int64(10), true) + c.Assert(err, IsNil) + _, id, err = alloc.Alloc(ctx, 1, 1, 1, 1) + c.Assert(err, IsNil) + c.Assert(id, Equals, int64(42)) + + // maxInt64 + err = alloc.Rebase(1, int64(math.MaxInt64-2), true) + c.Assert(err, IsNil) + _, id, err = alloc.Alloc(ctx, 1, 1, 1, 1) + c.Assert(err, IsNil) + c.Assert(id, Equals, int64(math.MaxInt64-1)) + _, _, err = alloc.Alloc(ctx, 1, 1, 1, 1) + c.Assert(terror.ErrorEqual(err, autoid.ErrAutoincReadFailed), IsTrue) + + // test unsigned + columnInfo.FieldType.Flag |= mysql.UnsignedFlag + alloc = autoid.NewAllocatorFromTempTblInfo(tblInfo) + c.Assert(alloc, NotNil) + + var n uint64 = math.MaxUint64 - 2 + err = alloc.Rebase(1, int64(n), true) + c.Assert(err, IsNil) + _, id, err = alloc.Alloc(ctx, 1, 1, 1, 1) + c.Assert(err, IsNil) + c.Assert(id, Equals, int64(n+1)) + _, _, err = alloc.Alloc(ctx, 1, 1, 1, 1) + c.Assert(terror.ErrorEqual(err, autoid.ErrAutoincReadFailed), IsTrue) +} From 0859a3c029ada613f078ec8aa41fa8c1c4cd4d09 Mon Sep 17 00:00:00 2001 From: Zhuhe Fang Date: Sat, 8 May 2021 17:40:41 +0800 Subject: [PATCH 35/67] planner: refactor Converting Partition Keys for shuffle hash join (#24456) --- planner/core/integration_test.go | 5 +- planner/core/task.go | 52 ++++++------ .../testdata/integration_serial_suite_in.json | 3 +- .../integration_serial_suite_out.json | 84 ++++++++++++------- .../core/testdata/integration_suite_out.json | 44 +++++----- .../core/testdata/partition_pruner_out.json | 9 +- store/mockstore/unistore/cophandler/mpp.go | 5 ++ .../mockstore/unistore/cophandler/mpp_exec.go | 18 +++- 8 files changed, 133 insertions(+), 87 deletions(-) diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index 0a6a560802240..816c94fa32a66 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -3216,8 +3216,11 @@ func (s *testIntegrationSerialSuite) TestMppJoinDecimal(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t") + tk.MustExec("drop table if exists tt") tk.MustExec("create table t (c1 decimal(8, 5), c2 decimal(9, 5), c3 decimal(9, 4) NOT NULL, c4 decimal(8, 4) NOT NULL, c5 decimal(40, 20))") + tk.MustExec("create table tt (pk int(11) NOT NULL AUTO_INCREMENT primary key,col_varchar_64 varchar(64),col_char_64_not_null char(64) NOT null, col_decimal_30_10_key decimal(30,10), col_tinyint tinyint, col_varchar_key varchar(1), key col_decimal_30_10_key (col_decimal_30_10_key), key col_varchar_key(col_varchar_key));") tk.MustExec("analyze table t") + tk.MustExec("analyze table tt") // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Se) @@ -3225,7 +3228,7 @@ func (s *testIntegrationSerialSuite) TestMppJoinDecimal(c *C) { db, exists := is.SchemaByName(model.NewCIStr("test")) c.Assert(exists, IsTrue) for _, tblInfo := range db.Tables { - if tblInfo.Name.L == "t" { + if tblInfo.Name.L == "t" || tblInfo.Name.L == "tt" { tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ Count: 1, Available: true, diff --git a/planner/core/task.go b/planner/core/task.go index 82045579be155..205f5eb77b08a 100644 --- a/planner/core/task.go +++ b/planner/core/task.go @@ -652,9 +652,9 @@ func appendExpr(p *PhysicalProjection, expr expression.Expression) *expression.C return col } -// TiFlash join require that join key has exactly the same type, while TiDB only guarantee the join key is the same catalog, -// so if the join key type is not exactly the same, we need add a projection below the join or exchanger if exists. -func (p *PhysicalHashJoin) convertJoinKeyForTiFlashIfNeed(lTask, rTask *mppTask) (*mppTask, *mppTask) { +// TiFlash join require that partition key has exactly the same type, while TiDB only guarantee the partition key is the same catalog, +// so if the partition key type is not exactly the same, we need add a projection below the join or exchanger if exists. +func (p *PhysicalHashJoin) convertPartitionKeysIfNeed(lTask, rTask *mppTask) (*mppTask, *mppTask) { lp := lTask.p if _, ok := lp.(*PhysicalExchangeReceiver); ok { lp = lp.Children()[0].Children()[0] @@ -663,15 +663,15 @@ func (p *PhysicalHashJoin) convertJoinKeyForTiFlashIfNeed(lTask, rTask *mppTask) if _, ok := rp.(*PhysicalExchangeReceiver); ok { rp = rp.Children()[0].Children()[0] } - // to mark if any equal cond needs to convert - lMask := make([]bool, len(p.EqualConditions)) - rMask := make([]bool, len(p.EqualConditions)) - cTypes := make([]*types.FieldType, len(p.EqualConditions)) + // to mark if any partition key needs to convert + lMask := make([]bool, len(lTask.hashCols)) + rMask := make([]bool, len(rTask.hashCols)) + cTypes := make([]*types.FieldType, len(lTask.hashCols)) lChanged := false rChanged := false - for i, eqFunc := range p.EqualConditions { - lKey := eqFunc.GetArgs()[0].(*expression.Column) - rKey := eqFunc.GetArgs()[1].(*expression.Column) + for i := range lTask.hashCols { + lKey := lTask.hashCols[i] + rKey := rTask.hashCols[i] cType, lConvert, rConvert := negotiateCommonType(lKey.RetType, rKey.RetType) if lConvert { lMask[i] = true @@ -696,14 +696,12 @@ func (p *PhysicalHashJoin) convertJoinKeyForTiFlashIfNeed(lTask, rTask *mppTask) rProj = getProj(p.ctx, rp) rp = rProj } - newEqCondition := make([]*expression.ScalarFunction, 0, len(p.EqualConditions)) - newEqCondition = append(newEqCondition, p.EqualConditions...) - p.EqualConditions = newEqCondition - lKeys := make([]*expression.Column, 0, len(p.EqualConditions)) - rKeys := make([]*expression.Column, 0, len(p.EqualConditions)) - for i, eqFunc := range p.EqualConditions { - lKey := eqFunc.GetArgs()[0].(*expression.Column) - rKey := eqFunc.GetArgs()[1].(*expression.Column) + + lPartKeys := make([]*expression.Column, 0, len(rTask.hashCols)) + rPartKeys := make([]*expression.Column, 0, len(lTask.hashCols)) + for i := range lTask.hashCols { + lKey := lTask.hashCols[i] + rKey := rTask.hashCols[i] if lMask[i] { cType := cTypes[i].Clone() cType.Flag = lKey.RetType.Flag @@ -716,12 +714,8 @@ func (p *PhysicalHashJoin) convertJoinKeyForTiFlashIfNeed(lTask, rTask *mppTask) rCast := expression.BuildCastFunction(p.ctx, rKey, cType) rKey = appendExpr(rProj, rCast) } - if lMask[i] || rMask[i] { - eqCond := expression.NewFunctionInternal(p.ctx, ast.EQ, types.NewFieldType(mysql.TypeTiny), lKey, rKey) - p.EqualConditions[i] = eqCond.(*expression.ScalarFunction) - } - lKeys = append(lKeys, lKey) - rKeys = append(rKeys, rKey) + lPartKeys = append(lPartKeys, lKey) + rPartKeys = append(rPartKeys, rKey) } // if left or right child changes, we need to add enforcer. if lChanged { @@ -730,7 +724,7 @@ func (p *PhysicalHashJoin) convertJoinKeyForTiFlashIfNeed(lTask, rTask *mppTask) nlTask = nlTask.enforceExchangerImpl(&property.PhysicalProperty{ TaskTp: property.MppTaskType, PartitionTp: property.HashType, - PartitionCols: lKeys, + PartitionCols: lPartKeys, }) nlTask.cst = lTask.cst lProj.cost = nlTask.cst @@ -742,7 +736,7 @@ func (p *PhysicalHashJoin) convertJoinKeyForTiFlashIfNeed(lTask, rTask *mppTask) nrTask = nrTask.enforceExchangerImpl(&property.PhysicalProperty{ TaskTp: property.MppTaskType, PartitionTp: property.HashType, - PartitionCols: rKeys, + PartitionCols: rPartKeys, }) nrTask.cst = rTask.cst rProj.cost = nrTask.cst @@ -758,7 +752,11 @@ func (p *PhysicalHashJoin) attach2TaskForMpp(tasks ...task) task { return invalidTask } if p.mppShuffleJoin { - lTask, rTask = p.convertJoinKeyForTiFlashIfNeed(lTask, rTask) + // protection check is case of some bugs + if len(lTask.hashCols) != len(rTask.hashCols) || len(lTask.hashCols) == 0 { + return invalidTask + } + lTask, rTask = p.convertPartitionKeysIfNeed(lTask, rTask) } p.SetChildren(lTask.plan(), rTask.plan()) p.schema = BuildPhysicalJoinSchema(p.JoinType, p) diff --git a/planner/core/testdata/integration_serial_suite_in.json b/planner/core/testdata/integration_serial_suite_in.json index 9f9e6e31c3253..3234652e5d000 100644 --- a/planner/core/testdata/integration_serial_suite_in.json +++ b/planner/core/testdata/integration_serial_suite_in.json @@ -234,7 +234,8 @@ "desc format = 'brief' select * from t t1 join t t2 on t1.c1 = t2.c2 and t1.c2 = t2.c3 and t1.c3 = t2.c1 and t1.c4 = t2.c3 and t1.c1 = t2.c5", "desc format = 'brief' select * from t t1 join t t2 on t1.c1 + t1.c2 = t2.c2 / t2.c3", "desc format = 'brief' select * from t t1 where exists (select * from t t2 where t1.c1 = t2.c2 and t1.c2 = t2.c3 and t1.c3 = t2.c1 and t1.c4 = t2.c3 and t1.c1 = t2.c5)", - "desc format = 'brief' select * from t t1 left join t t2 on t1.c1 = t2.c2 join t t3 on t2.c5 = t3.c3 right join t t4 on t3.c3 = t4.c4 " + "desc format = 'brief' select * from t t1 left join t t2 on t1.c1 = t2.c2 join t t3 on t2.c5 = t3.c3 right join t t4 on t3.c3 = t4.c4 ", + "desc format = 'brief' SELECT STRAIGHT_JOIN t1 . col_varchar_64 , t1 . col_char_64_not_null FROM tt AS t1 INNER JOIN( tt AS t2 JOIN tt AS t3 ON(t3 . col_decimal_30_10_key = t2 . col_tinyint)) ON(t3 . col_varchar_64 = t2 . col_varchar_key) WHERE t3 . col_varchar_64 = t1 . col_char_64_not_null GROUP BY 1 , 2" ] }, { diff --git a/planner/core/testdata/integration_serial_suite_out.json b/planner/core/testdata/integration_serial_suite_out.json index e05c7ef22e4cb..8b67310894d4b 100644 --- a/planner/core/testdata/integration_serial_suite_out.json +++ b/planner/core/testdata/integration_serial_suite_out.json @@ -1805,7 +1805,7 @@ "Projection 12500.00 root test.t.c1, test.t.c2, test.t.c1, test.t.c2, test.t.c3", "└─TableReader 12500.00 root data:ExchangeSender", " └─ExchangeSender 12500.00 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 12500.00 cop[tiflash] inner join, equal:[eq(Column#21, Column#22) eq(Column#15, Column#16)]", + " └─HashJoin 12500.00 cop[tiflash] inner join, equal:[eq(Column#13, Column#14) eq(Column#15, Column#16)]", " ├─ExchangeReceiver(Build) 10000.00 cop[tiflash] ", " │ └─ExchangeSender 10000.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: Column#21, Column#15", " │ └─Projection 10000.00 cop[tiflash] test.t.c1, test.t.c2, Column#13, Column#15, cast(Column#13, decimal(34,8) BINARY)->Column#21", @@ -1824,7 +1824,7 @@ "Projection 7976.02 root test.t.c1, test.t.c2, test.t.c5, Column#7, test.t.c1, test.t.c2, test.t.c3, Column#14", "└─TableReader 7976.02 root data:ExchangeSender", " └─ExchangeSender 7976.02 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 7976.02 batchCop[tiflash] inner join, equal:[eq(test.t.c1, test.t.c2) eq(Column#31, Column#32) eq(test.t.c5, Column#33)]", + " └─HashJoin 7976.02 batchCop[tiflash] inner join, equal:[eq(test.t.c1, test.t.c2) eq(test.t.c2, test.t.c3) eq(test.t.c5, test.t.c1)]", " ├─ExchangeReceiver(Build) 7976.02 batchCop[tiflash] ", " │ └─ExchangeSender 7976.02 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.t.c1, Column#31, test.t.c5", " │ └─Projection 7976.02 batchCop[tiflash] Column#7, test.t.c1, test.t.c2, test.t.c5, cast(test.t.c2, decimal(10,5))->Column#31", @@ -1869,7 +1869,7 @@ "Projection 12462.54 root test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5", "└─TableReader 12462.54 root data:ExchangeSender", " └─ExchangeSender 12462.54 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 12462.54 cop[tiflash] inner join, equal:[eq(test.t.c2, test.t.c1) eq(Column#13, Column#14) eq(Column#15, Column#16) eq(test.t.c3, test.t.c4) eq(test.t.c5, Column#17)]", + " └─HashJoin 12462.54 cop[tiflash] inner join, equal:[eq(test.t.c2, test.t.c1) eq(test.t.c3, test.t.c2) eq(test.t.c1, test.t.c3) eq(test.t.c3, test.t.c4) eq(test.t.c5, test.t.c1)]", " ├─ExchangeReceiver(Build) 9970.03 cop[tiflash] ", " │ └─ExchangeSender 9970.03 cop[tiflash] ExchangeType: HashPartition, Hash Cols: test.t.c2, Column#13, Column#15, test.t.c3, test.t.c5", " │ └─Projection 9970.03 cop[tiflash] test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, cast(test.t.c3, decimal(10,5))->Column#13, cast(test.t.c1, decimal(10,5))->Column#15", @@ -1888,7 +1888,7 @@ "Projection 12500.00 root test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5", "└─TableReader 12500.00 root data:ExchangeSender", " └─ExchangeSender 12500.00 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 12500.00 cop[tiflash] inner join, equal:[eq(Column#17, Column#14)]", + " └─HashJoin 12500.00 cop[tiflash] inner join, equal:[eq(Column#13, Column#14)]", " ├─ExchangeReceiver(Build) 10000.00 cop[tiflash] ", " │ └─ExchangeSender 10000.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: Column#17", " │ └─Projection 10000.00 cop[tiflash] test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, Column#13, cast(Column#13, decimal(17,9) BINARY)->Column#17", @@ -1906,7 +1906,7 @@ "Projection 7984.01 root test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5", "└─TableReader 7984.01 root data:ExchangeSender", " └─ExchangeSender 7984.01 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 7984.01 cop[tiflash] semi join, equal:[eq(test.t.c1, test.t.c2) eq(Column#13, Column#14) eq(Column#15, Column#16) eq(test.t.c4, test.t.c3) eq(Column#17, test.t.c5)]", + " └─HashJoin 7984.01 cop[tiflash] semi join, equal:[eq(test.t.c1, test.t.c2) eq(test.t.c2, test.t.c3) eq(test.t.c3, test.t.c1) eq(test.t.c4, test.t.c3) eq(test.t.c1, test.t.c5)]", " ├─ExchangeReceiver(Build) 9970.03 cop[tiflash] ", " │ └─ExchangeSender 9970.03 cop[tiflash] ExchangeType: HashPartition, Hash Cols: test.t.c2, Column#14, Column#16, test.t.c3, test.t.c5", " │ └─Projection 9970.03 cop[tiflash] test.t.c1, test.t.c2, test.t.c3, test.t.c5, cast(test.t.c3, decimal(10,5))->Column#14, cast(test.t.c1, decimal(10,5))->Column#16", @@ -1922,29 +1922,57 @@ { "SQL": "desc format = 'brief' select * from t t1 left join t t2 on t1.c1 = t2.c2 join t t3 on t2.c5 = t3.c3 right join t t4 on t3.c3 = t4.c4 ", "Plan": [ - "TableReader 19492.21 root data:ExchangeSender", - "└─ExchangeSender 19492.21 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 19492.21 cop[tiflash] right outer join, equal:[eq(test.t.c3, test.t.c4)]", - " ├─Projection(Build) 15593.77 cop[tiflash] test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5", - " │ └─HashJoin 15593.77 cop[tiflash] inner join, equal:[eq(test.t.c5, Column#25)]", - " │ ├─ExchangeReceiver(Build) 10000.00 cop[tiflash] ", - " │ │ └─ExchangeSender 10000.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: Column#25", - " │ │ └─Projection 10000.00 cop[tiflash] test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, cast(test.t.c3, decimal(40,20))->Column#25", - " │ │ └─TableFullScan 10000.00 cop[tiflash] table:t3 keep order:false, stats:pseudo", - " │ └─ExchangeReceiver(Probe) 12475.01 cop[tiflash] ", - " │ └─ExchangeSender 12475.01 cop[tiflash] ExchangeType: HashPartition, Hash Cols: test.t.c5", - " │ └─HashJoin 12475.01 cop[tiflash] inner join, equal:[eq(test.t.c2, test.t.c1)]", - " │ ├─ExchangeReceiver(Build) 9980.01 cop[tiflash] ", - " │ │ └─ExchangeSender 9980.01 cop[tiflash] ExchangeType: HashPartition, Hash Cols: test.t.c2", - " │ │ └─Selection 9980.01 cop[tiflash] not(isnull(test.t.c2)), not(isnull(test.t.c5))", - " │ │ └─TableFullScan 10000.00 cop[tiflash] table:t2 keep order:false, stats:pseudo", - " │ └─ExchangeReceiver(Probe) 9990.00 cop[tiflash] ", - " │ └─ExchangeSender 9990.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: test.t.c1", - " │ └─Selection 9990.00 cop[tiflash] not(isnull(test.t.c1))", - " │ └─TableFullScan 10000.00 cop[tiflash] table:t1 keep order:false, stats:pseudo", - " └─ExchangeReceiver(Probe) 10000.00 cop[tiflash] ", - " └─ExchangeSender 10000.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: test.t.c4", - " └─TableFullScan 10000.00 cop[tiflash] table:t4 keep order:false, stats:pseudo" + "Projection 19492.21 root test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5", + "└─TableReader 19492.21 root data:ExchangeSender", + " └─ExchangeSender 19492.21 cop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 19492.21 cop[tiflash] right outer join, equal:[eq(test.t.c3, test.t.c4)]", + " ├─Projection(Build) 15593.77 cop[tiflash] test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5", + " │ └─HashJoin 15593.77 cop[tiflash] inner join, equal:[eq(test.t.c5, test.t.c3)]", + " │ ├─ExchangeReceiver(Build) 10000.00 cop[tiflash] ", + " │ │ └─ExchangeSender 10000.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: Column#25", + " │ │ └─Projection 10000.00 cop[tiflash] test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, cast(test.t.c3, decimal(40,20))->Column#25", + " │ │ └─TableFullScan 10000.00 cop[tiflash] table:t3 keep order:false, stats:pseudo", + " │ └─ExchangeReceiver(Probe) 12475.01 cop[tiflash] ", + " │ └─ExchangeSender 12475.01 cop[tiflash] ExchangeType: HashPartition, Hash Cols: test.t.c5", + " │ └─HashJoin 12475.01 cop[tiflash] inner join, equal:[eq(test.t.c2, test.t.c1)]", + " │ ├─ExchangeReceiver(Build) 9980.01 cop[tiflash] ", + " │ │ └─ExchangeSender 9980.01 cop[tiflash] ExchangeType: HashPartition, Hash Cols: test.t.c2", + " │ │ └─Selection 9980.01 cop[tiflash] not(isnull(test.t.c2)), not(isnull(test.t.c5))", + " │ │ └─TableFullScan 10000.00 cop[tiflash] table:t2 keep order:false, stats:pseudo", + " │ └─ExchangeReceiver(Probe) 9990.00 cop[tiflash] ", + " │ └─ExchangeSender 9990.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: test.t.c1", + " │ └─Selection 9990.00 cop[tiflash] not(isnull(test.t.c1))", + " │ └─TableFullScan 10000.00 cop[tiflash] table:t1 keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 10000.00 cop[tiflash] ", + " └─ExchangeSender 10000.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: Column#27", + " └─Projection 10000.00 cop[tiflash] test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, cast(test.t.c4, decimal(40,20))->Column#27", + " └─TableFullScan 10000.00 cop[tiflash] table:t4 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' SELECT STRAIGHT_JOIN t1 . col_varchar_64 , t1 . col_char_64_not_null FROM tt AS t1 INNER JOIN( tt AS t2 JOIN tt AS t3 ON(t3 . col_decimal_30_10_key = t2 . col_tinyint)) ON(t3 . col_varchar_64 = t2 . col_varchar_key) WHERE t3 . col_varchar_64 = t1 . col_char_64_not_null GROUP BY 1 , 2", + "Plan": [ + "TableReader 8000.00 root data:ExchangeSender", + "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 batchCop[tiflash] test.tt.col_varchar_64, test.tt.col_char_64_not_null", + " └─HashAgg 8000.00 batchCop[tiflash] group by:test.tt.col_char_64_not_null, test.tt.col_varchar_64, funcs:firstrow(test.tt.col_varchar_64)->test.tt.col_varchar_64, funcs:firstrow(test.tt.col_char_64_not_null)->test.tt.col_char_64_not_null", + " └─ExchangeReceiver 15609.38 batchCop[tiflash] ", + " └─ExchangeSender 15609.38 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.tt.col_varchar_64, test.tt.col_char_64_not_null", + " └─HashJoin 15609.38 batchCop[tiflash] inner join, equal:[eq(test.tt.col_char_64_not_null, test.tt.col_varchar_64)]", + " ├─ExchangeReceiver(Build) 10000.00 batchCop[tiflash] ", + " │ └─ExchangeSender 10000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: Column#29", + " │ └─Projection 10000.00 batchCop[tiflash] test.tt.col_varchar_64, test.tt.col_char_64_not_null, cast(test.tt.col_char_64_not_null, varchar(64) CHARACTER SET utf8mb4 COLLATE utf8mb4_bin)->Column#29", + " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t1 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 batchCop[tiflash] inner join, equal:[eq(test.tt.col_varchar_key, test.tt.col_varchar_64) eq(Column#19, test.tt.col_decimal_30_10_key)]", + " ├─ExchangeReceiver(Build) 9990.00 batchCop[tiflash] ", + " │ └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.tt.col_varchar_key", + " │ └─Projection 9990.00 batchCop[tiflash] test.tt.col_varchar_key, cast(test.tt.col_tinyint, decimal(20,0) BINARY)->Column#19", + " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.tt.col_varchar_key))", + " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t2 keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 9990.00 batchCop[tiflash] ", + " └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.tt.col_varchar_64", + " └─Selection 9990.00 batchCop[tiflash] not(isnull(test.tt.col_varchar_64))", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t3 keep order:false, stats:pseudo" ] } ] diff --git a/planner/core/testdata/integration_suite_out.json b/planner/core/testdata/integration_suite_out.json index 28fc950e37991..7c735fcb5657c 100644 --- a/planner/core/testdata/integration_suite_out.json +++ b/planner/core/testdata/integration_suite_out.json @@ -1527,40 +1527,40 @@ { "SQL": "select 1 from s1", "Plan": [ - "Projection 1.00 root 1->Column#1", - "└─TableDual 1.00 root rows:1" + "Projection 1.00 root 1->Column#1", + "└─TableDual 1.00 root rows:1" ] - }, - { + }, + { "SQL": "select count(1) from s1", "Plan": [ - "StreamAgg 1.00 root funcs:count(1)->Column#1", - "└─TableDual 1.00 root rows:1" + "StreamAgg 1.00 root funcs:count(1)->Column#1", + "└─TableDual 1.00 root rows:1" ] - }, - { + }, + { "SQL": "select count(*) from s1", "Plan": [ - "StreamAgg 1.00 root funcs:count(1)->Column#1", - "└─TableDual 1.00 root rows:1" + "StreamAgg 1.00 root funcs:count(1)->Column#1", + "└─TableDual 1.00 root rows:1" ] - }, - { + }, + { "SQL": "select sum(1) from s1", "Plan": [ - "StreamAgg 1.00 root funcs:sum(1)->Column#1", - "└─TableDual 1.00 root rows:1" + "StreamAgg 1.00 root funcs:sum(1)->Column#1", + "└─TableDual 1.00 root rows:1" ] - }, - { + }, + { "SQL": "select count(1) as cnt from s1 union select count(1) as cnt from s2", "Plan": [ - "HashAgg 2.00 root group by:Column#3, funcs:firstrow(Column#3)->Column#3", - "└─Union 2.00 root ", - " ├─StreamAgg 1.00 root funcs:count(1)->Column#1", - " │ └─TableDual 1.00 root rows:1", - " └─StreamAgg 1.00 root funcs:count(1)->Column#2", - " └─TableDual 1.00 root rows:1" + "HashAgg 2.00 root group by:Column#3, funcs:firstrow(Column#3)->Column#3", + "└─Union 2.00 root ", + " ├─StreamAgg 1.00 root funcs:count(1)->Column#1", + " │ └─TableDual 1.00 root rows:1", + " └─StreamAgg 1.00 root funcs:count(1)->Column#2", + " └─TableDual 1.00 root rows:1" ] } ] diff --git a/planner/core/testdata/partition_pruner_out.json b/planner/core/testdata/partition_pruner_out.json index bfea064683c39..5593ab52bee74 100644 --- a/planner/core/testdata/partition_pruner_out.json +++ b/planner/core/testdata/partition_pruner_out.json @@ -182,7 +182,6 @@ "TableReader 250.00 root partition:p4,p5 data:Selection", "└─Selection 250.00 cop[tikv] ge(test_partition.t8.a, 10), le(test_partition.t8.a, 11)", " └─TableFullScan 10000.00 cop[tikv] table:t8 keep order:false, stats:pseudo" - ] }, { @@ -191,7 +190,6 @@ "TableReader 260.00 root partition:p0,p1,p2,p5 data:Selection", "└─Selection 260.00 cop[tikv] or(isnull(test_partition.t8.a), and(ge(test_partition.t8.a, 5), le(test_partition.t8.a, 8)))", " └─TableFullScan 10000.00 cop[tikv] table:t8 keep order:false, stats:pseudo" - ] }, { @@ -200,7 +198,6 @@ "TableReader 250.00 root partition:all data:Selection", "└─Selection 250.00 cop[tikv] ge(test_partition.t8.a, 5), le(test_partition.t8.a, 12)", " └─TableFullScan 10000.00 cop[tikv] table:t8 keep order:false, stats:pseudo" - ] }, { @@ -216,7 +213,7 @@ "Result": [ "TableReader 250.00 root partition:p4,p5 data:Selection", "└─Selection 250.00 cop[tikv] gt(test_partition.t8.a, 9), lt(test_partition.t8.a, 12)", - " └─TableFullScan 10000.00 cop[tikv] table:t8 keep order:false, stats:pseudo" + " └─TableFullScan 10000.00 cop[tikv] table:t8 keep order:false, stats:pseudo" ] }, { @@ -2857,7 +2854,7 @@ "Result": [ "TableReader 10.00 root data:Selection", "└─Selection 10.00 cop[tikv] eq(test_partition.t.a, 10)", - " └─TableFullScan 10000.00 cop[tikv] table:t, partition:p_max keep order:false, stats:pseudo" + " └─TableFullScan 10000.00 cop[tikv] table:t, partition:p_max keep order:false, stats:pseudo" ] }, { @@ -2865,7 +2862,7 @@ "Result": [ "TableReader 10.00 root data:Selection", "└─Selection 10.00 cop[tikv] eq(test_partition.t.a, -1)", - " └─TableFullScan 10000.00 cop[tikv] table:t, partition:p0 keep order:false, stats:pseudo" + " └─TableFullScan 10000.00 cop[tikv] table:t, partition:p0 keep order:false, stats:pseudo" ] }, { diff --git a/store/mockstore/unistore/cophandler/mpp.go b/store/mockstore/unistore/cophandler/mpp.go index 7f5dfdc9e8707..bed35eebee01e 100644 --- a/store/mockstore/unistore/cophandler/mpp.go +++ b/store/mockstore/unistore/cophandler/mpp.go @@ -206,6 +206,11 @@ func (b *mppExecBuilder) buildMPPJoin(pb *tipb.Join, children []*tipb.Executor) } e.probeKey = probeExpr.(*expression.Column) } + e.comKeyTp = types.AggFieldType([]*types.FieldType{e.probeKey.RetType, e.buildKey.RetType}) + if e.comKeyTp.Tp == mysql.TypeNewDecimal { + e.comKeyTp.Flen = mysql.MaxDecimalWidth + e.comKeyTp.Decimal = mysql.MaxDecimalScale + } return e, nil } diff --git a/store/mockstore/unistore/cophandler/mpp_exec.go b/store/mockstore/unistore/cophandler/mpp_exec.go index 960f3bd976e60..0072456ecf8c3 100644 --- a/store/mockstore/unistore/cophandler/mpp_exec.go +++ b/store/mockstore/unistore/cophandler/mpp_exec.go @@ -343,6 +343,20 @@ type joinExec struct { defaultInner chunk.Row inited bool + // align the types of join keys and build keys + comKeyTp *types.FieldType +} + +func (e *joinExec) getHashKey(keyCol types.Datum) (str string, err error) { + keyCol, err = keyCol.ConvertTo(e.sc, e.comKeyTp) + if err != nil { + return str, errors.Trace(err) + } + str, err = keyCol.ToString() + if err != nil { + return "", errors.Trace(err) + } + return str, nil } func (e *joinExec) buildHashTable() error { @@ -358,7 +372,7 @@ func (e *joinExec) buildHashTable() error { for i := 0; i < rows; i++ { row := chk.GetRow(i) keyCol := row.GetDatum(e.buildKey.Index, e.buildChild.getFieldTypes()[e.buildKey.Index]) - key, err := keyCol.ToString() + key, err := e.getHashKey(keyCol) if err != nil { return errors.Trace(err) } @@ -386,7 +400,7 @@ func (e *joinExec) fetchRows() (bool, error) { for i := 0; i < chkSize; i++ { row := chk.GetRow(i) keyCol := row.GetDatum(e.probeKey.Index, e.probeChild.getFieldTypes()[e.probeKey.Index]) - key, err := keyCol.ToString() + key, err := e.getHashKey(keyCol) if err != nil { return false, errors.Trace(err) } From 04da3ceadef96a769bb76f48ed98b354409fccb8 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Sat, 8 May 2021 18:06:42 +0800 Subject: [PATCH 36/67] store/cop: reload region every time when meeting io error (#24447) --- store/copr/batch_request_sender.go | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/store/copr/batch_request_sender.go b/store/copr/batch_request_sender.go index bc2206b9780cb..139ee087ec290 100644 --- a/store/copr/batch_request_sender.go +++ b/store/copr/batch_request_sender.go @@ -78,7 +78,12 @@ func (ss *RegionBatchRequestSender) onSendFail(bo *backoffer, ctxs []copTaskAndR for _, failedCtx := range ctxs { ctx := failedCtx.ctx if ctx.Meta != nil { - ss.GetRegionCache().OnSendFail(bo.TiKVBackoffer(), ctx, ss.NeedReloadRegion(ctx), err) + // The reload region param is always true. Because that every time we try, we must + // re-build the range then re-create the batch sender. As a result, the len of "failStores" + // will change. If tiflash's replica is more than two, the "reload region" will always be false. + // Now that the batch cop and mpp has a relative low qps, it's reasonable to reload every time + // when meeting io error. + ss.GetRegionCache().OnSendFail(bo.TiKVBackoffer(), ctx, true, err) } } From 3b935289ae7e80968cb65c3d6a13c2b12f403231 Mon Sep 17 00:00:00 2001 From: Morgan Tocker Date: Sun, 9 May 2021 22:04:45 -0600 Subject: [PATCH 37/67] variable: improve test coverage (#24503) --- sessionctx/variable/sysvar_test.go | 229 +++++++++++++++++++++++++++++ sessionctx/variable/varsutil.go | 2 +- 2 files changed, 230 insertions(+), 1 deletion(-) diff --git a/sessionctx/variable/sysvar_test.go b/sessionctx/variable/sysvar_test.go index 4029072412977..cc765854a5c0b 100644 --- a/sessionctx/variable/sysvar_test.go +++ b/sessionctx/variable/sysvar_test.go @@ -14,13 +14,17 @@ package variable import ( + "encoding/json" "fmt" + "strconv" "strings" + "sync/atomic" "testing" . "github.com/pingcap/check" "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/terror" + "github.com/pingcap/tidb/config" ) func TestT(t *testing.T) { @@ -318,3 +322,228 @@ func (*testSysVarSuite) TestCollationServer(c *C) { c.Assert(sv.SetSessionFromHook(vars, "utf8mb4_bin"), IsNil) c.Assert(vars.systems[CharacterSetServer], Equals, "utf8mb4") // check it also changes charset. } + +func (*testSysVarSuite) TestTimeZone(c *C) { + sv := GetSysVar(TimeZone) + vars := NewSessionVars() + + // TiDB uses the Golang TZ library, so TZs are case-sensitive. + // Unfortunately this is not strictly MySQL compatible. i.e. + // This should not fail: + // val, err := sv.Validate(vars, "America/EDMONTON", ScopeSession) + // See: https://github.com/pingcap/tidb/issues/8087 + + val, err := sv.Validate(vars, "America/Edmonton", ScopeSession) + c.Assert(err, IsNil) + c.Assert(val, Equals, "America/Edmonton") + + val, err = sv.Validate(vars, "+10:00", ScopeSession) + c.Assert(err, IsNil) + c.Assert(val, Equals, "+10:00") + + val, err = sv.Validate(vars, "UTC", ScopeSession) + c.Assert(err, IsNil) + c.Assert(val, Equals, "UTC") + + val, err = sv.Validate(vars, "+00:00", ScopeSession) + c.Assert(err, IsNil) + c.Assert(val, Equals, "+00:00") + + c.Assert(sv.SetSessionFromHook(vars, "UTC"), IsNil) // sets + tz, err := parseTimeZone("UTC") + c.Assert(err, IsNil) + c.Assert(vars.TimeZone, Equals, tz) + +} + +func (*testSysVarSuite) TestForeignKeyChecks(c *C) { + sv := GetSysVar(ForeignKeyChecks) + vars := NewSessionVars() + + val, err := sv.Validate(vars, "on", ScopeSession) + c.Assert(err, IsNil) + c.Assert(val, Equals, "OFF") // warns and refuses to set ON. + + warn := vars.StmtCtx.GetWarnings()[0].Err + c.Assert(warn.Error(), Equals, "[variable:8047]variable 'foreign_key_checks' does not yet support value: on") + +} + +func (*testSysVarSuite) TestTxnIsolation(c *C) { + sv := GetSysVar(TxnIsolation) + vars := NewSessionVars() + + _, err := sv.Validate(vars, "on", ScopeSession) + c.Assert(err.Error(), Equals, "[variable:1231]Variable 'tx_isolation' can't be set to the value of 'on'") + + val, err := sv.Validate(vars, "read-COMMitted", ScopeSession) + c.Assert(err, IsNil) + c.Assert(val, Equals, "READ-COMMITTED") + + _, err = sv.Validate(vars, "Serializable", ScopeSession) + c.Assert(err.Error(), Equals, "[variable:8048]The isolation level 'SERIALIZABLE' is not supported. Set tidb_skip_isolation_level_check=1 to skip this error") + + _, err = sv.Validate(vars, "read-uncommitted", ScopeSession) + c.Assert(err.Error(), Equals, "[variable:8048]The isolation level 'READ-UNCOMMITTED' is not supported. Set tidb_skip_isolation_level_check=1 to skip this error") + + vars.systems[TiDBSkipIsolationLevelCheck] = "ON" + + val, err = sv.Validate(vars, "Serializable", ScopeSession) + c.Assert(err, IsNil) + c.Assert(val, Equals, "SERIALIZABLE") +} + +func (*testSysVarSuite) TestTiDBMultiStatementMode(c *C) { + sv := GetSysVar(TiDBMultiStatementMode) + vars := NewSessionVars() + + val, err := sv.Validate(vars, "on", ScopeSession) + c.Assert(err, IsNil) + c.Assert(val, Equals, "ON") + c.Assert(sv.SetSessionFromHook(vars, val), IsNil) + c.Assert(vars.MultiStatementMode, Equals, 1) + + val, err = sv.Validate(vars, "0", ScopeSession) + c.Assert(err, IsNil) + c.Assert(val, Equals, "OFF") + c.Assert(sv.SetSessionFromHook(vars, val), IsNil) + c.Assert(vars.MultiStatementMode, Equals, 0) + + val, err = sv.Validate(vars, "Warn", ScopeSession) + c.Assert(err, IsNil) + c.Assert(val, Equals, "WARN") + c.Assert(sv.SetSessionFromHook(vars, val), IsNil) + c.Assert(vars.MultiStatementMode, Equals, 2) +} + +func (*testSysVarSuite) TestReadOnlyNoop(c *C) { + vars := NewSessionVars() + for _, name := range []string{TxReadOnly, TransactionReadOnly} { + sv := GetSysVar(name) + val, err := sv.Validate(vars, "on", ScopeSession) + c.Assert(err.Error(), Equals, "[variable:1235]function READ ONLY has only noop implementation in tidb now, use tidb_enable_noop_functions to enable these functions") + c.Assert(val, Equals, "OFF") + } +} + +func (*testSysVarSuite) TestGetScopeNoneSystemVar(c *C) { + val, ok, err := GetScopeNoneSystemVar(Port) + c.Assert(err, IsNil) + c.Assert(ok, IsTrue) + c.Assert(val, Equals, "4000") + + val, ok, err = GetScopeNoneSystemVar("nonsensevar") + c.Assert(err.Error(), Equals, "[variable:1193]Unknown system variable 'nonsensevar'") + c.Assert(ok, IsFalse) + c.Assert(val, Equals, "") + + val, ok, err = GetScopeNoneSystemVar(CharacterSetClient) + c.Assert(err, IsNil) + c.Assert(ok, IsFalse) + c.Assert(val, Equals, "") +} + +func (*testSysVarSuite) TestInstanceScopedVars(c *C) { + // This tests instance scoped variables through GetSessionSystemVar(). + // Eventually these should be changed to use getters so that the switch + // statement in GetSessionOnlySysVars can be removed. + + vars := NewSessionVars() + + val, err := GetSessionSystemVar(vars, TiDBCurrentTS) + c.Assert(err, IsNil) + c.Assert(val, Equals, fmt.Sprintf("%d", vars.TxnCtx.StartTS)) + + val, err = GetSessionSystemVar(vars, TiDBLastTxnInfo) + c.Assert(err, IsNil) + c.Assert(val, Equals, vars.LastTxnInfo) + + val, err = GetSessionSystemVar(vars, TiDBLastQueryInfo) + c.Assert(err, IsNil) + info, err := json.Marshal(vars.LastQueryInfo) + c.Assert(err, IsNil) + c.Assert(val, Equals, string(info)) + + val, err = GetSessionSystemVar(vars, TiDBGeneralLog) + c.Assert(err, IsNil) + c.Assert(val, Equals, BoolToOnOff(ProcessGeneralLog.Load())) + + val, err = GetSessionSystemVar(vars, TiDBPProfSQLCPU) + c.Assert(err, IsNil) + expected := "0" + if EnablePProfSQLCPU.Load() { + expected = "1" + } + c.Assert(val, Equals, expected) + + val, err = GetSessionSystemVar(vars, TiDBExpensiveQueryTimeThreshold) + c.Assert(err, IsNil) + c.Assert(val, Equals, fmt.Sprintf("%d", atomic.LoadUint64(&ExpensiveQueryTimeThreshold))) + + val, err = GetSessionSystemVar(vars, TiDBMemoryUsageAlarmRatio) + c.Assert(err, IsNil) + c.Assert(val, Equals, fmt.Sprintf("%g", MemoryUsageAlarmRatio.Load())) + + val, err = GetSessionSystemVar(vars, TiDBConfig) + c.Assert(err, IsNil) + conf := config.GetGlobalConfig() + j, err := json.MarshalIndent(conf, "", "\t") + c.Assert(err, IsNil) + c.Assert(val, Equals, config.HideConfig(string(j))) + + val, err = GetSessionSystemVar(vars, TiDBForcePriority) + c.Assert(err, IsNil) + c.Assert(val, Equals, mysql.Priority2Str[mysql.PriorityEnum(atomic.LoadInt32(&ForcePriority))]) + + val, err = GetSessionSystemVar(vars, TiDBDDLSlowOprThreshold) + c.Assert(err, IsNil) + c.Assert(val, Equals, strconv.FormatUint(uint64(atomic.LoadUint32(&DDLSlowOprThreshold)), 10)) + + val, err = GetSessionSystemVar(vars, PluginDir) + c.Assert(err, IsNil) + c.Assert(val, Equals, config.GetGlobalConfig().Plugin.Dir) + + val, err = GetSessionSystemVar(vars, PluginLoad) + c.Assert(err, IsNil) + c.Assert(val, Equals, config.GetGlobalConfig().Plugin.Load) + + val, err = GetSessionSystemVar(vars, TiDBSlowLogThreshold) + c.Assert(err, IsNil) + c.Assert(val, Equals, strconv.FormatUint(atomic.LoadUint64(&config.GetGlobalConfig().Log.SlowThreshold), 10)) + + val, err = GetSessionSystemVar(vars, TiDBRecordPlanInSlowLog) + c.Assert(err, IsNil) + c.Assert(val, Equals, strconv.FormatUint(uint64(atomic.LoadUint32(&config.GetGlobalConfig().Log.RecordPlanInSlowLog)), 10)) + + val, err = GetSessionSystemVar(vars, TiDBEnableSlowLog) + c.Assert(err, IsNil) + c.Assert(val, Equals, BoolToOnOff(config.GetGlobalConfig().Log.EnableSlowLog)) + + val, err = GetSessionSystemVar(vars, TiDBQueryLogMaxLen) + c.Assert(err, IsNil) + c.Assert(val, Equals, strconv.FormatUint(atomic.LoadUint64(&config.GetGlobalConfig().Log.QueryLogMaxLen), 10)) + + val, err = GetSessionSystemVar(vars, TiDBCheckMb4ValueInUTF8) + c.Assert(err, IsNil) + c.Assert(val, Equals, BoolToOnOff(config.GetGlobalConfig().CheckMb4ValueInUTF8)) + + val, err = GetSessionSystemVar(vars, TiDBCapturePlanBaseline) + c.Assert(err, IsNil) + c.Assert(val, Equals, CapturePlanBaseline.GetVal()) + + val, err = GetSessionSystemVar(vars, TiDBFoundInPlanCache) + c.Assert(err, IsNil) + c.Assert(val, Equals, BoolToOnOff(vars.PrevFoundInPlanCache)) + + val, err = GetSessionSystemVar(vars, TiDBFoundInBinding) + c.Assert(err, IsNil) + c.Assert(val, Equals, BoolToOnOff(vars.PrevFoundInBinding)) + + val, err = GetSessionSystemVar(vars, TiDBEnableCollectExecutionInfo) + c.Assert(err, IsNil) + c.Assert(val, Equals, BoolToOnOff(config.GetGlobalConfig().EnableCollectExecutionInfo)) + + val, err = GetSessionSystemVar(vars, TiDBTxnScope) + c.Assert(err, IsNil) + c.Assert(val, Equals, vars.TxnScope.GetVarValue()) +} diff --git a/sessionctx/variable/varsutil.go b/sessionctx/variable/varsutil.go index 45cebbd62e0db..d43a45b1cdbc7 100644 --- a/sessionctx/variable/varsutil.go +++ b/sessionctx/variable/varsutil.go @@ -249,7 +249,7 @@ func GetSessionOnlySysVars(s *SessionVars, key string) (string, bool, error) { if ok { return sVal, true, nil } - if sysVar.Scope&ScopeGlobal == 0 { + if !sysVar.HasGlobalScope() { // None-Global variable can use pre-defined default value. return sysVar.Value, true, nil } From b0dfa746bf8d4064fced1713ce6a5ec33d7b9f37 Mon Sep 17 00:00:00 2001 From: jianzhiyao <739319867@qq.com> Date: Mon, 10 May 2021 12:56:43 +0800 Subject: [PATCH 38/67] *: fix gosimple check (#24460) --- cmd/importcheck/importcheck.go | 4 ++-- config/config_util.go | 4 ++-- store/tikv/mockstore/deadlock/deadlock_test.go | 3 +-- types/json/binary_functions.go | 2 -- util/timeutil/time.go | 2 +- 5 files changed, 6 insertions(+), 9 deletions(-) diff --git a/cmd/importcheck/importcheck.go b/cmd/importcheck/importcheck.go index 45e9cc2ab6365..e668705230e35 100644 --- a/cmd/importcheck/importcheck.go +++ b/cmd/importcheck/importcheck.go @@ -93,7 +93,7 @@ func checkFile(path string) error { continue } if !preIsStd { - return errors.New(fmt.Sprintf("stdlib %s need be group together and before non-stdlib group in %s", im.Path.Value, path)) + return fmt.Errorf("stdlib %s need be group together and before non-stdlib group in %s", im.Path.Value, path) } continue } @@ -103,7 +103,7 @@ func checkFile(path string) error { continue } if !checkSepWithNewline(src, importSpecs[i-1].Path.Pos(), im.Path.Pos()) { - return errors.New(fmt.Sprintf("non-stdlib %s need be group together and after stdlib group in %s", im.Path.Value, path)) + return fmt.Errorf("non-stdlib %s need be group together and after stdlib group in %s", im.Path.Value, path) } preIsStd = false } diff --git a/config/config_util.go b/config/config_util.go index e115808426fa9..0bf3374e001d7 100644 --- a/config/config_util.go +++ b/config/config_util.go @@ -142,9 +142,9 @@ func FlattenConfigItems(nestedConfig map[string]interface{}) map[string]interfac } func flatten(flatMap map[string]interface{}, nested interface{}, prefix string) { - switch nested.(type) { + switch nested := nested.(type) { case map[string]interface{}: - for k, v := range nested.(map[string]interface{}) { + for k, v := range nested { path := k if prefix != "" { path = prefix + "." + k diff --git a/store/tikv/mockstore/deadlock/deadlock_test.go b/store/tikv/mockstore/deadlock/deadlock_test.go index 0481c6014053c..6135650d617f3 100644 --- a/store/tikv/mockstore/deadlock/deadlock_test.go +++ b/store/tikv/mockstore/deadlock/deadlock_test.go @@ -14,7 +14,6 @@ package deadlock import ( - "fmt" "testing" . "github.com/pingcap/check" @@ -36,7 +35,7 @@ func (s *testDeadlockSuite) TestDeadlock(c *C) { c.Assert(err, IsNil) err = detector.Detect(3, 1, 300) c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, fmt.Sprintf("deadlock(200)")) + c.Assert(err.Error(), Equals, "deadlock(200)") detector.CleanUp(2) list2 := detector.waitForMap[2] c.Assert(list2, IsNil) diff --git a/types/json/binary_functions.go b/types/json/binary_functions.go index 3a9e21d1a783a..8f6b0a626be10 100644 --- a/types/json/binary_functions.go +++ b/types/json/binary_functions.go @@ -527,7 +527,6 @@ func (bm *binaryModifier) doInsert(path PathExpression, newBj BinaryJSON) { elems = append(elems, newBj) } bm.modifyValue, bm.err = buildBinaryObject(keys, elems) - return } func (bm *binaryModifier) remove(path PathExpression) BinaryJSON { @@ -582,7 +581,6 @@ func (bm *binaryModifier) doRemove(path PathExpression) { } } bm.modifyValue, bm.err = buildBinaryObject(keys, elems) - return } // rebuild merges the old and the modified JSON into a new BinaryJSON diff --git a/util/timeutil/time.go b/util/timeutil/time.go index 47e9d8fa657c1..e9914ba4ef350 100644 --- a/util/timeutil/time.go +++ b/util/timeutil/time.go @@ -81,7 +81,7 @@ func InferSystemTZ() string { case !ok: path, err1 := filepath.EvalSymlinks("/etc/localtime") if err1 == nil { - if strings.Index(path, "posixrules") != -1 { + if strings.Contains(path, "posixrules") { path, err1 = inferOneStepLinkForPath("/etc/localtime") if err1 != nil { logutil.BgLogger().Error("locate timezone files failed", zap.Error(err1)) From 96577f9d1d05789f266f74a1c6daae5ffeb8fbd9 Mon Sep 17 00:00:00 2001 From: xhe Date: Mon, 10 May 2021 13:38:43 +0800 Subject: [PATCH 39/67] expression: try to fix TestExprPushDownToFlash tests (#24235) --- expression/builtin_time_test.go | 8 -------- expression/evaluator_test.go | 20 ++++++++++++++++++++ expression/expr_to_pb_test.go | 9 --------- 3 files changed, 20 insertions(+), 17 deletions(-) diff --git a/expression/builtin_time_test.go b/expression/builtin_time_test.go index 97152bf81a6cb..f82f6fb8f76ea 100644 --- a/expression/builtin_time_test.go +++ b/expression/builtin_time_test.go @@ -34,14 +34,6 @@ import ( "github.com/pingcap/tidb/util/timeutil" ) -func init() { - // Some test depends on the values of timeutil.SystemLocation() - // If we don't SetSystemTZ() here, the value would change unpredictable. - // Affectd by the order whether a testsuite runs before or after integration test. - // Note, SetSystemTZ() is a sync.Once operation. - timeutil.SetSystemTZ("system") -} - func (s *testEvaluatorSuite) TestDate(c *C) { tblDate := []struct { Input interface{} diff --git a/expression/evaluator_test.go b/expression/evaluator_test.go index db9905e259716..4cd5c09ffef6d 100644 --- a/expression/evaluator_test.go +++ b/expression/evaluator_test.go @@ -19,6 +19,7 @@ import ( "time" . "github.com/pingcap/check" + "github.com/pingcap/failpoint" "github.com/pingcap/parser" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/charset" @@ -30,6 +31,7 @@ import ( "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/testleak" "github.com/pingcap/tidb/util/testutil" + "github.com/pingcap/tidb/util/timeutil" ) var _ = SerialSuites(&testEvaluatorSerialSuites{}) @@ -48,7 +50,25 @@ func TestT(t *testing.T) { CustomVerboseFlag = true *CustomParallelSuiteFlag = true + + // Some test depends on the values of timeutil.SystemLocation() + // If we don't SetSystemTZ() here, the value would change unpredictable. + // Affectd by the order whether a testsuite runs before or after integration test. + // Note, SetSystemTZ() is a sync.Once operation. + timeutil.SetSystemTZ("system") + + fpname := "github.com/pingcap/tidb/expression/PanicIfPbCodeUnspecified" + err := failpoint.Enable(fpname, "return(true)") + if err != nil { + t.Fatalf("enable global failpoint `%s` failed: %v", fpname, err) + } + TestingT(t) + + err = failpoint.Disable(fpname) + if err != nil { + t.Fatalf("disable global failpoint `%s` failed: %v", fpname, err) + } } type testEvaluatorSuiteBase struct { diff --git a/expression/expr_to_pb_test.go b/expression/expr_to_pb_test.go index 8dab2151c7dac..d997e9cf1691f 100644 --- a/expression/expr_to_pb_test.go +++ b/expression/expr_to_pb_test.go @@ -20,7 +20,6 @@ import ( "github.com/gogo/protobuf/proto" . "github.com/pingcap/check" - "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/charset" @@ -33,14 +32,6 @@ import ( "github.com/pingcap/tipb/go-tipb" ) -func init() { - fpname := "github.com/pingcap/tidb/expression/PanicIfPbCodeUnspecified" - err := failpoint.Enable(fpname, "return(true)") - if err != nil { - panic(errors.Errorf("enable global failpoint `%s` failed: %v", fpname, err)) - } -} - type dataGen4Expr2PbTest struct { } From f29cbd3b6b2cef46f95592c913ae211440e4c9c5 Mon Sep 17 00:00:00 2001 From: Shirly Date: Mon, 10 May 2021 16:28:45 +0800 Subject: [PATCH 40/67] store/tikv/error: remove unused error to driver package (#24426) --- store/driver/txn/error.go | 7 +++++++ store/tikv/error/errcode.go | 3 --- store/tikv/error/error.go | 7 ------- 3 files changed, 7 insertions(+), 10 deletions(-) diff --git a/store/driver/txn/error.go b/store/driver/txn/error.go index 6794a016a1060..b5e69b4e522a9 100644 --- a/store/driver/txn/error.go +++ b/store/driver/txn/error.go @@ -58,6 +58,13 @@ var ( ErrRegionUnavailable = dbterror.ClassTiKV.NewStd(errno.ErrRegionUnavailable) ) +// Registers error returned from TiKV. +var ( + _ = dbterror.ClassTiKV.NewStd(errno.ErrDataOutOfRange) + _ = dbterror.ClassTiKV.NewStd(errno.ErrTruncatedWrongValue) + _ = dbterror.ClassTiKV.NewStd(errno.ErrDivisionByZero) +) + func genKeyExistsError(name string, value string, err error) error { if err != nil { logutil.BgLogger().Info("extractKeyExistsErr meets error", zap.Error(err)) diff --git a/store/tikv/error/errcode.go b/store/tikv/error/errcode.go index 10cc2292be7e7..bf0e3c7c4b91d 100644 --- a/store/tikv/error/errcode.go +++ b/store/tikv/error/errcode.go @@ -18,10 +18,7 @@ package error const ( CodeUnknown = 1105 CodeLockWaitTimeout = 1205 - CodeTruncatedWrongValue = 1292 CodeQueryInterrupted = 1317 - CodeDivisionByZero = 1365 - CodeDataOutOfRange = 1690 CodeLockAcquireFailAndNoWaitSet = 3572 // TiKV/PD/TiFlash errors. diff --git a/store/tikv/error/error.go b/store/tikv/error/error.go index a4f213ab09a02..8ed21d8db192f 100644 --- a/store/tikv/error/error.go +++ b/store/tikv/error/error.go @@ -64,13 +64,6 @@ var ( ErrUnknown = dbterror.ClassTiKV.NewStd(CodeUnknown) ) -// Registers error returned from TiKV. -var ( - _ = dbterror.ClassTiKV.NewStd(CodeDataOutOfRange) - _ = dbterror.ClassTiKV.NewStd(CodeTruncatedWrongValue) - _ = dbterror.ClassTiKV.NewStd(CodeDivisionByZero) -) - // IsErrNotFound checks if err is a kind of NotFound error. func IsErrNotFound(err error) bool { return errors.ErrorEqual(err, ErrNotExist) From 0581cb4ed9267a202c771fc969d432c4f1535c81 Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Mon, 10 May 2021 18:47:27 +0800 Subject: [PATCH 41/67] executor: add some test cases about partition-table dynamic mode with view (#24448) --- executor/partition_table_test.go | 72 ++++++++++++++++++++++++++++++++ 1 file changed, 72 insertions(+) diff --git a/executor/partition_table_test.go b/executor/partition_table_test.go index ee7d807ef4e8f..58d6293b4ce54 100644 --- a/executor/partition_table_test.go +++ b/executor/partition_table_test.go @@ -227,6 +227,78 @@ func (s *partitionTableSuite) TestPartitionInfoDisable(c *C) { tk.MustQuery("select * from t_info_null where (date = '2020-10-02' or date = '2020-10-06') and app = 'xxx' and media = '19003006'").Check(testkit.Rows()) } +func (s *partitionTableSuite) TestView(c *C) { + if israce.RaceEnabled { + c.Skip("exhaustive types test, skip race test") + } + + tk := testkit.NewTestKitWithInit(c, s.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, key(a)) partition by hash(a) partitions 4`) + tk.MustExec(`create table trange (a varchar(10), b varchar(10), key(a)) partition by range columns(a) ( + partition p0 values less than ('300'), + partition p1 values less than ('600'), + partition p2 values less than ('900'), + partition p3 values less than ('9999'))`) + tk.MustExec(`create table t1 (a int, b int, key(a))`) + tk.MustExec(`create table t2 (a varchar(10), b varchar(10), key(a))`) + + // insert the same data into thash and t1 + vals := make([]string, 0, 3000) + for i := 0; i < 3000; i++ { + vals = append(vals, fmt.Sprintf(`(%v, %v)`, rand.Intn(10000), rand.Intn(10000))) + } + tk.MustExec(fmt.Sprintf(`insert into thash values %v`, strings.Join(vals, ", "))) + tk.MustExec(fmt.Sprintf(`insert into t1 values %v`, strings.Join(vals, ", "))) + + // insert the same data into trange and t2 + vals = vals[:0] + for i := 0; i < 2000; i++ { + vals = append(vals, fmt.Sprintf(`("%v", "%v")`, rand.Intn(1000), rand.Intn(1000))) + } + tk.MustExec(fmt.Sprintf(`insert into trange values %v`, strings.Join(vals, ", "))) + tk.MustExec(fmt.Sprintf(`insert into t2 values %v`, strings.Join(vals, ", "))) + + // test views on a single table + tk.MustExec(`create definer='root'@'localhost' view vhash as select a*2 as a, a+b as b from thash`) + tk.MustExec(`create definer='root'@'localhost' view v1 as select a*2 as a, a+b as b from t1`) + tk.MustExec(`create definer='root'@'localhost' view vrange as select concat(a, b) as a, a+b as b from trange`) + tk.MustExec(`create definer='root'@'localhost' view v2 as select concat(a, b) as a, a+b as b from t2`) + for i := 0; i < 100; i++ { + xhash := rand.Intn(10000) + tk.MustQuery(fmt.Sprintf(`select * from vhash where a>=%v`, xhash)).Sort().Check( + tk.MustQuery(fmt.Sprintf(`select * from v1 where a>=%v`, xhash)).Sort().Rows()) + tk.MustQuery(fmt.Sprintf(`select * from vhash where b>=%v`, xhash)).Sort().Check( + tk.MustQuery(fmt.Sprintf(`select * from v1 where b>=%v`, xhash)).Sort().Rows()) + tk.MustQuery(fmt.Sprintf(`select * from vhash where a>=%v and b>=%v`, xhash, xhash)).Sort().Check( + tk.MustQuery(fmt.Sprintf(`select * from v1 where a>=%v and b>=%v`, xhash, xhash)).Sort().Rows()) + + xrange := fmt.Sprintf(`"%v"`, rand.Intn(1000)) + tk.MustQuery(fmt.Sprintf(`select * from vrange where a>=%v`, xrange)).Sort().Check( + tk.MustQuery(fmt.Sprintf(`select * from v2 where a>=%v`, xrange)).Sort().Rows()) + tk.MustQuery(fmt.Sprintf(`select * from vrange where b>=%v`, xrange)).Sort().Check( + tk.MustQuery(fmt.Sprintf(`select * from v2 where b>=%v`, xrange)).Sort().Rows()) + tk.MustQuery(fmt.Sprintf(`select * from vrange where a>=%v and b<=%v`, xrange, xrange)).Sort().Check( + tk.MustQuery(fmt.Sprintf(`select * from v2 where a>=%v and b<=%v`, xrange, xrange)).Sort().Rows()) + } + + // test views on both tables + tk.MustExec(`create definer='root'@'localhost' view vboth as select thash.a+trange.a as a, thash.b+trange.b as b from thash, trange where thash.a=trange.a`) + tk.MustExec(`create definer='root'@'localhost' view vt as select t1.a+t2.a as a, t1.b+t2.b as b from t1, t2 where t1.a=t2.a`) + for i := 0; i < 100; i++ { + x := rand.Intn(10000) + tk.MustQuery(fmt.Sprintf(`select * from vboth where a>=%v`, x)).Sort().Check( + tk.MustQuery(fmt.Sprintf(`select * from vt where a>=%v`, x)).Sort().Rows()) + tk.MustQuery(fmt.Sprintf(`select * from vboth where b>=%v`, x)).Sort().Check( + tk.MustQuery(fmt.Sprintf(`select * from vt where b>=%v`, x)).Sort().Rows()) + tk.MustQuery(fmt.Sprintf(`select * from vboth where a>=%v and b>=%v`, x, x)).Sort().Check( + tk.MustQuery(fmt.Sprintf(`select * from vt where a>=%v and b>=%v`, x, x)).Sort().Rows()) + } +} + func (s *partitionTableSuite) TestGlobalStatsAndSQLBinding(c *C) { if israce.RaceEnabled { c.Skip("exhaustive types test, skip race test") From a0fad5a9ef886296c8b6ebbf1551ec763c64a098 Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Mon, 10 May 2021 19:57:30 +0800 Subject: [PATCH 42/67] planner: add more test cases about dynamic-mode with new-collation (#24454) --- planner/core/integration_test.go | 40 ++++++++++++++++++++++++++++++++ 1 file changed, 40 insertions(+) diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index 816c94fa32a66..f717888caf2c9 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -806,6 +806,46 @@ func (s *testIntegrationSerialSuite) TestMPPWithBroadcastExchangeUnderNewCollati } } +func (s *testIntegrationSerialSuite) TestPartitionTableDynamicModeUnderNewCollation(c *C) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create database test_new_collation") + tk.MustExec("use test_new_collation") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + + // hash + range partition + tk.MustExec(`CREATE TABLE thash (a int, c varchar(20) charset utf8mb4 collate utf8mb4_general_ci, key(a)) partition by hash(a) partitions 4`) + tk.MustExec(`CREATE TABLE trange (a int, c varchar(20) charset utf8mb4 collate utf8mb4_general_ci, key(a)) partition by range(a) ( + partition p0 values less than (10), + partition p1 values less than (20), + partition p2 values less than (30), + partition p3 values less than (40))`) + tk.MustExec(`insert into thash values (1, 'a'), (1, 'A'), (11, 'a'), (11, 'A'), (21, 'a'), (21, 'A'), (31, 'a'), (31, 'A')`) + tk.MustExec(`insert into trange values (1, 'a'), (1, 'A'), (11, 'a'), (11, 'A'), (21, 'a'), (21, 'A'), (31, 'a'), (31, 'A')`) + tk.MustQuery(`select * from thash use index(a) where a in (1, 11, 31) and c='a'`).Sort().Check(testkit.Rows("1 A", "1 a", "11 A", "11 a", "31 A", "31 a")) + tk.MustQuery(`select * from thash ignore index(a) where a in (1, 11, 31) and c='a'`).Sort().Check(testkit.Rows("1 A", "1 a", "11 A", "11 a", "31 A", "31 a")) + tk.MustQuery(`select * from trange use index(a) where a in (1, 11, 31) and c='a'`).Sort().Check(testkit.Rows("1 A", "1 a", "11 A", "11 a", "31 A", "31 a")) + tk.MustQuery(`select * from trange ignore index(a) where a in (1, 11, 31) and c='a'`).Sort().Check(testkit.Rows("1 A", "1 a", "11 A", "11 a", "31 A", "31 a")) + + // range partition and partitioned by utf8mb4_general_ci + tk.MustExec(`create table strrange(a varchar(10) charset utf8mb4 collate utf8mb4_general_ci, b int) partition by range columns(a) ( + partition p0 values less than ('a'), + partition p1 values less than ('k'), + partition p2 values less than ('z'))`) + tk.MustExec("insert into strrange values ('a', 1), ('A', 1), ('y', 1), ('Y', 1), ('q', 1)") + tk.MustQuery("select * from strrange where a in ('a', 'y')").Sort().Check(testkit.Rows("A 1", "Y 1", "a 1", "y 1")) + + // list partition and partitioned by utf8mb4_general_ci + tk.MustExec(`create table strlist(a varchar(10) charset utf8mb4 collate utf8mb4_general_ci, b int) partition by list(a) ( + partition p0 values in ('a', 'b'), + partition p1 values in ('c', 'd'), + partition p2 values in ('e', 'f'))`) + tk.MustExec("insert into strlist values ('a', 1), ('A', 1), ('d', 1), ('D', 1), ('e', 1)") + tk.MustQuery(`select * from strlist where a='a'`).Sort().Check(testkit.Rows("A 1", "a 1")) + tk.MustQuery(`select * from strlist where a in ('D', 'e')`).Sort().Check(testkit.Rows("D 1", "d 1", "e 1")) +} + func (s *testIntegrationSerialSuite) TestMPPAvgRewrite(c *C) { defer collate.SetNewCollationEnabledForTest(false) tk := testkit.NewTestKit(c, s.store) From 5e9e0e6e37be14a84f22ac86a2e98dd36b092f66 Mon Sep 17 00:00:00 2001 From: xhe Date: Mon, 10 May 2021 21:00:36 +0800 Subject: [PATCH 43/67] *: consitent get infoschema (#24230) --- distsql/request_builder.go | 7 +++---- executor/adapter.go | 2 +- executor/analyze.go | 4 ++-- executor/analyze_test.go | 12 ++++++------ executor/builder.go | 2 +- executor/compiler.go | 2 +- executor/coprocessor.go | 2 +- executor/distsql.go | 18 ++++++++++++------ executor/executor_test.go | 4 ++-- executor/grant.go | 2 +- executor/index_merge_reader.go | 2 +- executor/infoschema_reader.go | 16 ++++++++-------- executor/load_stats.go | 2 +- executor/metrics_reader_test.go | 2 +- executor/partition_table_test.go | 2 +- executor/point_get.go | 2 +- executor/prepared.go | 2 +- executor/simple.go | 2 +- executor/table_reader.go | 11 +++++++---- expression/builtin_info.go | 6 +++--- infoschema/infoschema.go | 26 -------------------------- infoschema/tables.go | 2 +- planner/core/cacheable_checker_test.go | 2 +- planner/core/explain.go | 8 ++++---- planner/core/expression_rewriter.go | 5 +++-- planner/core/integration_test.go | 2 +- planner/core/point_get_plan.go | 6 +++--- planner/core/prepare_test.go | 2 +- session/session.go | 10 +++++----- sessionctx/variable/session.go | 19 +++++++++++++++++++ statistics/handle/ddl.go | 2 +- statistics/handle/update.go | 2 +- 32 files changed, 95 insertions(+), 93 deletions(-) diff --git a/distsql/request_builder.go b/distsql/request_builder.go index ce577b993d009..69a6da548ec60 100644 --- a/distsql/request_builder.go +++ b/distsql/request_builder.go @@ -230,10 +230,9 @@ func (builder *RequestBuilder) SetFromSessionVars(sv *variable.SessionVars) *Req builder.Request.TaskID = sv.StmtCtx.TaskID builder.Request.Priority = builder.getKVPriority(sv) builder.Request.ReplicaRead = sv.GetReplicaRead() - if sv.SnapshotInfoschema != nil { - builder.Request.SchemaVar = infoschema.GetInfoSchemaBySessionVars(sv).SchemaMetaVersion() - } else { - builder.Request.SchemaVar = sv.TxnCtx.SchemaVersion + // in tests, it may be null + if is, ok := sv.GetInfoSchema().(infoschema.InfoSchema); ok { + builder.Request.SchemaVar = is.SchemaMetaVersion() } builder.txnScope = sv.TxnCtx.TxnScope builder.IsStaleness = sv.TxnCtx.IsStaleness diff --git a/executor/adapter.go b/executor/adapter.go index 5e5b7990f61d9..5f5229195c3f9 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -268,7 +268,7 @@ func (a *ExecStmt) IsReadOnly(vars *variable.SessionVars) bool { // RebuildPlan rebuilds current execute statement plan. // It returns the current information schema version that 'a' is using. func (a *ExecStmt) RebuildPlan(ctx context.Context) (int64, error) { - is := infoschema.GetInfoSchema(a.Ctx) + is := a.Ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) a.InfoSchema = is if err := plannercore.Preprocess(a.Ctx, a.StmtNode, is, plannercore.InTxnRetry); err != nil { return 0, err diff --git a/executor/analyze.go b/executor/analyze.go index 2d3187842845c..b7e9e51fec7da 100644 --- a/executor/analyze.go +++ b/executor/analyze.go @@ -187,7 +187,7 @@ func (e *AnalyzeExec) Next(ctx context.Context, req *chunk.Chunk) error { } if needGlobalStats { for globalStatsID, info := range globalStatsMap { - globalStats, err := statsHandle.MergePartitionStats2GlobalStatsByTableID(e.ctx, e.opts, infoschema.GetInfoSchema(e.ctx), globalStatsID.tableID, info.isIndex, info.idxID) + globalStats, err := statsHandle.MergePartitionStats2GlobalStatsByTableID(e.ctx, e.opts, e.ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema), globalStatsID.tableID, info.isIndex, info.idxID) if err != nil { if types.ErrPartitionStatsMissing.Equal(err) { // When we find some partition-level stats are missing, we need to report warning. @@ -205,7 +205,7 @@ func (e *AnalyzeExec) Next(ctx context.Context, req *chunk.Chunk) error { } } } - return statsHandle.Update(infoschema.GetInfoSchema(e.ctx)) + return statsHandle.Update(e.ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema)) } func getBuildStatsConcurrency(ctx sessionctx.Context) (int, error) { diff --git a/executor/analyze_test.go b/executor/analyze_test.go index f8eff902fcb3d..39ec524dbacc4 100644 --- a/executor/analyze_test.go +++ b/executor/analyze_test.go @@ -69,7 +69,7 @@ PARTITION BY RANGE ( a ) ( } tk.MustExec("analyze table t") - is := infoschema.GetInfoSchema(tk.Se.(sessionctx.Context)) + is := tk.Se.(sessionctx.Context).GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) pi := table.Meta().GetPartitionInfo() @@ -96,7 +96,7 @@ PARTITION BY RANGE ( a ) ( tk.MustExec(fmt.Sprintf(`insert into t values (%d, %d, "hello")`, i, i)) } tk.MustExec("alter table t analyze partition p0") - is = infoschema.GetInfoSchema(tk.Se.(sessionctx.Context)) + is = tk.Se.(sessionctx.Context).GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) table, err = is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) pi = table.Meta().GetPartitionInfo() @@ -176,7 +176,7 @@ func (s *testSuite1) TestAnalyzeParameters(c *C) { tk.MustExec("set @@tidb_enable_fast_analyze = 1") tk.MustExec("analyze table t with 30 samples") - is := infoschema.GetInfoSchema(tk.Se.(sessionctx.Context)) + is := tk.Se.(sessionctx.Context).GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) tableInfo := table.Meta() @@ -227,7 +227,7 @@ func (s *testSuite1) TestAnalyzeTooLongColumns(c *C) { tk.MustExec(fmt.Sprintf("insert into t values ('%s')", value)) tk.MustExec("analyze table t") - is := infoschema.GetInfoSchema(tk.Se.(sessionctx.Context)) + is := tk.Se.(sessionctx.Context).GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) tableInfo := table.Meta() @@ -259,7 +259,7 @@ func (s *testSuite1) TestAnalyzeIndexExtractTopN(c *C) { tk.MustExec("set @@session.tidb_analyze_version=2") tk.MustExec("analyze table t with 10 cmsketch width") - is := infoschema.GetInfoSchema(tk.Se.(sessionctx.Context)) + is := tk.Se.(sessionctx.Context).GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) tableInfo := table.Meta() @@ -435,7 +435,7 @@ func (s *testFastAnalyze) TestFastAnalyze(c *C) { } tk.MustExec("analyze table t with 5 buckets, 6 samples") - is := infoschema.GetInfoSchema(tk.Se.(sessionctx.Context)) + is := tk.Se.(sessionctx.Context).GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) tableInfo := table.Meta() diff --git a/executor/builder.go b/executor/builder.go index e82db0d6aaccc..40282d1030b2c 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -3432,7 +3432,7 @@ func (builder *dataReaderBuilder) buildTableReaderBase(ctx context.Context, e *T SetKeepOrder(e.keepOrder). SetStreaming(e.streaming). SetFromSessionVars(e.ctx.GetSessionVars()). - SetFromInfoSchema(infoschema.GetInfoSchema(e.ctx)). + SetFromInfoSchema(e.ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema)). Build() if err != nil { return nil, err diff --git a/executor/compiler.go b/executor/compiler.go index bb0f5274a159e..bb00bfe14602d 100644 --- a/executor/compiler.go +++ b/executor/compiler.go @@ -53,7 +53,7 @@ func (c *Compiler) Compile(ctx context.Context, stmtNode ast.StmtNode) (*ExecStm ctx = opentracing.ContextWithSpan(ctx, span1) } - infoSchema := infoschema.GetInfoSchema(c.Ctx) + infoSchema := c.Ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) if err := plannercore.Preprocess(c.Ctx, stmtNode, infoSchema); err != nil { return nil, err } diff --git a/executor/coprocessor.go b/executor/coprocessor.go index 25959e5454655..490b981add461 100644 --- a/executor/coprocessor.go +++ b/executor/coprocessor.go @@ -159,7 +159,7 @@ func (h *CoprocessorDAGHandler) buildDAGExecutor(req *coprocessor.Request) (Exec return nil, errors.Trace(err) } h.dagReq = dagReq - is := h.sctx.GetSessionVars().TxnCtx.InfoSchema.(infoschema.InfoSchema) + is := h.sctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) // Build physical plan. bp := core.NewPBPlanBuilder(h.sctx, is, req.Ranges) plan, err := bp.Build(dagReq.Executors) diff --git a/executor/distsql.go b/executor/distsql.go index 6e3105a30142d..bd422a0458ef1 100644 --- a/executor/distsql.go +++ b/executor/distsql.go @@ -271,16 +271,19 @@ func (e *IndexReaderExecutor) open(ctx context.Context, kvRanges []kv.KeyRange) e.memTracker = memory.NewTracker(e.id, -1) e.memTracker.AttachTo(e.ctx.GetSessionVars().StmtCtx.MemTracker) var builder distsql.RequestBuilder - kvReq, err := builder.SetKeyRanges(kvRanges). + builder.SetKeyRanges(kvRanges). SetDAGRequest(e.dagPB). SetStartTS(e.startTS). SetDesc(e.desc). SetKeepOrder(e.keepOrder). SetStreaming(e.streaming). SetFromSessionVars(e.ctx.GetSessionVars()). - SetMemTracker(e.memTracker). - SetFromInfoSchema(infoschema.GetInfoSchema(e.ctx)). - Build() + SetMemTracker(e.memTracker) + // for tests, infoschema may be null + if is, ok := e.ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema); ok { + builder.SetFromInfoSchema(is) + } + kvReq, err := builder.Build() if err != nil { e.feedback.Invalidate() return err @@ -527,8 +530,11 @@ func (e *IndexLookUpExecutor) startIndexWorker(ctx context.Context, workCh chan< SetKeepOrder(e.keepOrder). SetStreaming(e.indexStreaming). SetFromSessionVars(e.ctx.GetSessionVars()). - SetMemTracker(tracker). - SetFromInfoSchema(infoschema.GetInfoSchema(e.ctx)) + SetMemTracker(tracker) + // for tests, infoschema may be null + if is, ok := e.ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema); ok { + builder.SetFromInfoSchema(is) + } for partTblIdx, kvRange := range kvRanges { // check if executor is closed diff --git a/executor/executor_test.go b/executor/executor_test.go index 80056439ec7c6..823967f24f217 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -2338,7 +2338,7 @@ func (s *testSuiteP2) TestIsPointGet(c *C) { "select * from help_topic where help_topic_id=1": true, "select * from help_topic where help_category_id=1": false, } - infoSchema := infoschema.GetInfoSchema(ctx) + infoSchema := ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) for sqlStr, result := range tests { stmtNode, err := s.ParseOneStmt(sqlStr, "", "") @@ -2370,7 +2370,7 @@ func (s *testSuiteP2) TestClusteredIndexIsPointGet(c *C) { "select * from t where a='x' and c='x'": true, "select * from t where a='x' and c='x' and b=1": false, } - infoSchema := infoschema.GetInfoSchema(ctx) + infoSchema := ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) for sqlStr, result := range tests { stmtNode, err := s.ParseOneStmt(sqlStr, "", "") c.Check(err, IsNil) diff --git a/executor/grant.go b/executor/grant.go index f6a8453ba72dd..6c715758b6c7f 100644 --- a/executor/grant.go +++ b/executor/grant.go @@ -74,7 +74,7 @@ func (e *GrantExec) Next(ctx context.Context, req *chunk.Chunk) error { // Make sure the table exist. if e.Level.Level == ast.GrantLevelTable { dbNameStr := model.NewCIStr(dbName) - schema := infoschema.GetInfoSchema(e.ctx) + schema := e.ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) tbl, err := schema.TableByName(dbNameStr, model.NewCIStr(e.Level.TableName)) if err != nil { return err diff --git a/executor/index_merge_reader.go b/executor/index_merge_reader.go index c769d2705c44b..84b4b810b059d 100644 --- a/executor/index_merge_reader.go +++ b/executor/index_merge_reader.go @@ -245,7 +245,7 @@ func (e *IndexMergeReaderExecutor) startPartialIndexWorker(ctx context.Context, SetStreaming(e.partialStreamings[workID]). SetFromSessionVars(e.ctx.GetSessionVars()). SetMemTracker(e.memTracker). - SetFromInfoSchema(infoschema.GetInfoSchema(e.ctx)) + SetFromInfoSchema(e.ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema)) worker := &partialIndexWorker{ stats: e.stats, diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index 368d8838a777e..a39992799b27a 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -79,7 +79,7 @@ func (e *memtableRetriever) retrieve(ctx context.Context, sctx sessionctx.Contex // Cache the ret full rows in schemataRetriever if !e.initialized { - is := infoschema.GetInfoSchema(sctx) + is := sctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) dbs := is.AllSchemas() sort.Sort(infoschema.SchemasSorter(dbs)) var err error @@ -295,7 +295,7 @@ func (c *statsCache) get(ctx sessionctx.Context) (map[int64]uint64, map[tableHis } func getAutoIncrementID(ctx sessionctx.Context, schema *model.DBInfo, tblInfo *model.TableInfo) (int64, error) { - is := infoschema.GetInfoSchema(ctx) + is := ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) tbl, err := is.TableByName(schema.Name, tblInfo.Name) if err != nil { return 0, err @@ -583,7 +583,7 @@ func (e *hugeMemTableRetriever) setDataForColumns(ctx context.Context, sctx sess } func (e *hugeMemTableRetriever) dataForColumnsInTable(ctx context.Context, sctx sessionctx.Context, schema *model.DBInfo, tbl *model.TableInfo) { - if err := tryFillViewColumnType(ctx, sctx, infoschema.GetInfoSchema(sctx), schema.Name, tbl); err != nil { + if err := tryFillViewColumnType(ctx, sctx, sctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema), schema.Name, tbl); err != nil { sctx.GetSessionVars().StmtCtx.AppendWarning(err) return } @@ -1330,7 +1330,7 @@ func (e *memtableRetriever) setDataForTiKVRegionStatus(ctx sessionctx.Context) e if err != nil { return err } - allSchemas := ctx.GetSessionVars().TxnCtx.InfoSchema.(infoschema.InfoSchema).AllSchemas() + allSchemas := ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema).AllSchemas() tableInfos := tikvHelper.GetRegionsTableInfo(regionsInfo, allSchemas) for _, region := range regionsInfo.Regions { tableList := tableInfos[region.ID] @@ -1442,7 +1442,7 @@ func (e *memtableRetriever) setDataForTiDBHotRegions(ctx sessionctx.Context) err if !ok { return errors.New("Information about hot region can be gotten only when the storage is TiKV") } - allSchemas := ctx.GetSessionVars().TxnCtx.InfoSchema.(infoschema.InfoSchema).AllSchemas() + allSchemas := ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema).AllSchemas() tikvHelper := &helper.Helper{ Store: tikvStore, RegionCache: tikvStore.GetRegionCache(), @@ -1591,7 +1591,7 @@ type initialTable struct { } func (e *tableStorageStatsRetriever) initialize(sctx sessionctx.Context) error { - is := infoschema.GetInfoSchema(sctx) + is := sctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) var databases []string schemas := e.extractor.TableSchema tables := e.extractor.TableName @@ -1883,7 +1883,7 @@ func (e *memtableRetriever) setDataForStatementsSummary(ctx sessionctx.Context, func (e *memtableRetriever) setDataForPlacementPolicy(ctx sessionctx.Context) error { checker := privilege.GetPrivilegeManager(ctx) - is := infoschema.GetInfoSchema(ctx) + is := ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) var rows [][]types.Datum for _, bundle := range is.RuleBundles() { id, err := placement.ObjectIDFromGroupID(bundle.ID) @@ -2030,7 +2030,7 @@ func (e *hugeMemTableRetriever) retrieve(ctx context.Context, sctx sessionctx.Co } if !e.initialized { - is := infoschema.GetInfoSchema(sctx) + is := sctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) dbs := is.AllSchemas() sort.Sort(infoschema.SchemasSorter(dbs)) e.dbs = dbs diff --git a/executor/load_stats.go b/executor/load_stats.go index 83fbb3ad188f7..984f649e1291f 100644 --- a/executor/load_stats.go +++ b/executor/load_stats.go @@ -86,5 +86,5 @@ func (e *LoadStatsInfo) Update(data []byte) error { if h == nil { return errors.New("Load Stats: handle is nil") } - return h.LoadStatsFromJSON(infoschema.GetInfoSchema(e.Ctx), jsonTbl) + return h.LoadStatsFromJSON(e.Ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema), jsonTbl) } diff --git a/executor/metrics_reader_test.go b/executor/metrics_reader_test.go index 19000b4faee5d..662c3b917ce9c 100644 --- a/executor/metrics_reader_test.go +++ b/executor/metrics_reader_test.go @@ -62,7 +62,7 @@ func (s *testSuite7) TestStmtLabel(c *C) { for _, tt := range tests { stmtNode, err := parser.New().ParseOneStmt(tt.sql, "", "") c.Check(err, IsNil) - is := infoschema.GetInfoSchema(tk.Se) + is := tk.Se.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) err = plannercore.Preprocess(tk.Se.(sessionctx.Context), stmtNode, is) c.Assert(err, IsNil) _, _, err = planner.Optimize(context.TODO(), tk.Se, stmtNode, is) diff --git a/executor/partition_table_test.go b/executor/partition_table_test.go index 58d6293b4ce54..55b8fa51786d7 100644 --- a/executor/partition_table_test.go +++ b/executor/partition_table_test.go @@ -209,7 +209,7 @@ func (s *partitionTableSuite) TestPartitionInfoDisable(c *C) { PARTITION p202010 VALUES LESS THAN ("2020-11-01"), PARTITION p202011 VALUES LESS THAN ("2020-12-01") )`) - is := infoschema.GetInfoSchema(tk.Se) + is := tk.Se.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t_info_null")) c.Assert(err, IsNil) diff --git a/executor/point_get.go b/executor/point_get.go index 241f52d421344..b0cd700c5c920 100644 --- a/executor/point_get.go +++ b/executor/point_get.go @@ -398,7 +398,7 @@ func (e *PointGetExecutor) verifyTxnScope() error { var tblID int64 var tblName string var partName string - is := infoschema.GetInfoSchema(e.ctx) + is := e.ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) if e.partInfo != nil { tblID = e.partInfo.ID tblInfo, _, partInfo := is.FindTableByPartitionID(tblID) diff --git a/executor/prepared.go b/executor/prepared.go index c5fdd5c1bf404..448ee3b7fdc66 100644 --- a/executor/prepared.go +++ b/executor/prepared.go @@ -320,7 +320,7 @@ func CompileExecutePreparedStmt(ctx context.Context, sctx sessionctx.Context, return nil, false, false, err } execStmt.BinaryArgs = args - is := infoschema.GetInfoSchema(sctx) + is := sctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) execPlan, names, err := planner.Optimize(ctx, sctx, execStmt, is) if err != nil { return nil, false, false, err diff --git a/executor/simple.go b/executor/simple.go index 5ed8ced28af48..65df5ca43117f 100644 --- a/executor/simple.go +++ b/executor/simple.go @@ -1406,7 +1406,7 @@ func (e *SimpleExec) executeDropStats(s *ast.DropStatsStmt) (err error) { if err := h.DeleteTableStatsFromKV(statsIDs); err != nil { return err } - return h.Update(infoschema.GetInfoSchema(e.ctx)) + return h.Update(e.ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema)) } func (e *SimpleExec) autoNewTxn() bool { diff --git a/executor/table_reader.go b/executor/table_reader.go index 767826f0c3b6c..1a76598fb2250 100644 --- a/executor/table_reader.go +++ b/executor/table_reader.go @@ -221,7 +221,7 @@ func (e *TableReaderExecutor) buildResp(ctx context.Context, ranges []*ranger.Ra } else { reqBuilder = builder.SetHandleRanges(e.ctx.GetSessionVars().StmtCtx, getPhysicalTableID(e.table), e.table.Meta() != nil && e.table.Meta().IsCommonHandle, ranges, e.feedback) } - kvReq, err := reqBuilder. + reqBuilder. SetDAGRequest(e.dagPB). SetStartTS(e.startTS). SetDesc(e.desc). @@ -230,9 +230,12 @@ func (e *TableReaderExecutor) buildResp(ctx context.Context, ranges []*ranger.Ra SetFromSessionVars(e.ctx.GetSessionVars()). SetMemTracker(e.memTracker). SetStoreType(e.storeType). - SetAllowBatchCop(e.batchCop). - SetFromInfoSchema(infoschema.GetInfoSchema(e.ctx)). - Build() + SetAllowBatchCop(e.batchCop) + // infoschema maybe null for tests + if is, ok := e.ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema); ok { + reqBuilder.SetFromInfoSchema(is) + } + kvReq, err := reqBuilder.Build() if err != nil { return nil, err } diff --git a/expression/builtin_info.go b/expression/builtin_info.go index 6a41b20ef75af..fda57a884f1d8 100644 --- a/expression/builtin_info.go +++ b/expression/builtin_info.go @@ -847,7 +847,7 @@ func (b *builtinNextValSig) evalInt(row chunk.Row) (int64, bool, error) { db = b.ctx.GetSessionVars().CurrentDB } // Check the tableName valid. - sequence, err := b.ctx.GetSessionVars().TxnCtx.InfoSchema.(util.SequenceSchema).SequenceByName(model.NewCIStr(db), model.NewCIStr(seq)) + sequence, err := b.ctx.GetSessionVars().GetInfoSchema().(util.SequenceSchema).SequenceByName(model.NewCIStr(db), model.NewCIStr(seq)) if err != nil { return 0, false, err } @@ -903,7 +903,7 @@ func (b *builtinLastValSig) evalInt(row chunk.Row) (int64, bool, error) { db = b.ctx.GetSessionVars().CurrentDB } // Check the tableName valid. - sequence, err := b.ctx.GetSessionVars().TxnCtx.InfoSchema.(util.SequenceSchema).SequenceByName(model.NewCIStr(db), model.NewCIStr(seq)) + sequence, err := b.ctx.GetSessionVars().GetInfoSchema().(util.SequenceSchema).SequenceByName(model.NewCIStr(db), model.NewCIStr(seq)) if err != nil { return 0, false, err } @@ -953,7 +953,7 @@ func (b *builtinSetValSig) evalInt(row chunk.Row) (int64, bool, error) { db = b.ctx.GetSessionVars().CurrentDB } // Check the tableName valid. - sequence, err := b.ctx.GetSessionVars().TxnCtx.InfoSchema.(util.SequenceSchema).SequenceByName(model.NewCIStr(db), model.NewCIStr(seq)) + sequence, err := b.ctx.GetSessionVars().GetInfoSchema().(util.SequenceSchema).SequenceByName(model.NewCIStr(db), model.NewCIStr(seq)) if err != nil { return 0, false, err } diff --git a/infoschema/infoschema.go b/infoschema/infoschema.go index 4fcbdc042de85..ac8afd14605f1 100644 --- a/infoschema/infoschema.go +++ b/infoschema/infoschema.go @@ -24,12 +24,8 @@ import ( "github.com/pingcap/tidb/ddl/placement" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta/autoid" - "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/util" - "github.com/pingcap/tidb/util/logutil" - "go.uber.org/zap" ) // InfoSchema is the interface used to retrieve the schema information. @@ -386,28 +382,6 @@ func HasAutoIncrementColumn(tbInfo *model.TableInfo) (bool, string) { return false, "" } -// GetInfoSchema gets TxnCtx InfoSchema if snapshot schema is not set, -// Otherwise, snapshot schema is returned. -func GetInfoSchema(ctx sessionctx.Context) InfoSchema { - return GetInfoSchemaBySessionVars(ctx.GetSessionVars()) -} - -// GetInfoSchemaBySessionVars gets TxnCtx InfoSchema if snapshot schema is not set, -// Otherwise, snapshot schema is returned. -func GetInfoSchemaBySessionVars(sessVar *variable.SessionVars) InfoSchema { - var is InfoSchema - if snap := sessVar.SnapshotInfoschema; snap != nil { - is = snap.(InfoSchema) - logutil.BgLogger().Info("use snapshot schema", zap.Uint64("conn", sessVar.ConnectionID), zap.Int64("schemaVersion", is.SchemaMetaVersion())) - } else { - if sessVar.TxnCtx == nil || sessVar.TxnCtx.InfoSchema == nil { - return nil - } - is = sessVar.TxnCtx.InfoSchema.(InfoSchema) - } - return is -} - func (is *infoSchema) BundleByName(name string) (*placement.Bundle, bool) { is.ruleBundleMutex.RLock() defer is.ruleBundleMutex.RUnlock() diff --git a/infoschema/tables.go b/infoschema/tables.go index 085bc6a96a77d..bfca649e89fdd 100644 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -1737,7 +1737,7 @@ func (s SchemasSorter) Less(i, j int) bool { } func (it *infoschemaTable) getRows(ctx sessionctx.Context, cols []*table.Column) (fullRows [][]types.Datum, err error) { - is := GetInfoSchema(ctx) + is := ctx.GetSessionVars().GetInfoSchema().(InfoSchema) dbs := is.AllSchemas() sort.Sort(SchemasSorter(dbs)) switch it.meta.Name.O { diff --git a/planner/core/cacheable_checker_test.go b/planner/core/cacheable_checker_test.go index eb33790dfd74f..fb9d05d528ec0 100644 --- a/planner/core/cacheable_checker_test.go +++ b/planner/core/cacheable_checker_test.go @@ -42,7 +42,7 @@ func (s *testCacheableSuite) TestCacheable(c *C) { tk.MustExec("create table t2(a int, b int) partition by hash(a) partitions 11") tk.MustExec("create table t3(a int, b int)") tbl := &ast.TableName{Schema: model.NewCIStr("test"), Name: model.NewCIStr("t3")} - is := infoschema.GetInfoSchema(tk.Se) + is := tk.Se.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) // test non-SelectStmt/-InsertStmt/-DeleteStmt/-UpdateStmt/-SetOprStmt var stmt ast.Node = &ast.ShowStmt{} c.Assert(core.Cacheable(stmt, is), IsFalse) diff --git a/planner/core/explain.go b/planner/core/explain.go index e3e7e4e06d0b3..913b4a88b5dac 100644 --- a/planner/core/explain.go +++ b/planner/core/explain.go @@ -306,7 +306,7 @@ func (p *PhysicalTableReader) accessObject(sctx sessionctx.Context) string { return "" } - is := infoschema.GetInfoSchema(sctx) + is := sctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) tmp, ok := is.TableByID(ts.Table.ID) if !ok { return "partition table not found" + strconv.FormatInt(ts.Table.ID, 10) @@ -366,7 +366,7 @@ func (p *PhysicalIndexReader) accessObject(sctx sessionctx.Context) string { } var buffer bytes.Buffer - is := infoschema.GetInfoSchema(sctx) + is := sctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) tmp, ok := is.TableByID(ts.Table.ID) if !ok { fmt.Fprintf(&buffer, "partition table not found: %d", ts.Table.ID) @@ -394,7 +394,7 @@ func (p *PhysicalIndexLookUpReader) accessObject(sctx sessionctx.Context) string } var buffer bytes.Buffer - is := infoschema.GetInfoSchema(sctx) + is := sctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) tmp, ok := is.TableByID(ts.Table.ID) if !ok { fmt.Fprintf(&buffer, "partition table not found: %d", ts.Table.ID) @@ -417,7 +417,7 @@ func (p *PhysicalIndexMergeReader) accessObject(sctx sessionctx.Context) string return "" } - is := infoschema.GetInfoSchema(sctx) + is := sctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) tmp, ok := is.TableByID(ts.Table.ID) if !ok { return "partition table not found" + strconv.FormatInt(ts.Table.ID, 10) diff --git a/planner/core/expression_rewriter.go b/planner/core/expression_rewriter.go index 5d6a11ad982a9..25ba4a21460fe 100644 --- a/planner/core/expression_rewriter.go +++ b/planner/core/expression_rewriter.go @@ -63,8 +63,9 @@ func evalAstExpr(sctx sessionctx.Context, expr ast.ExprNode) (types.Datum, error // rewriteAstExpr rewrites ast expression directly. func rewriteAstExpr(sctx sessionctx.Context, expr ast.ExprNode, schema *expression.Schema, names types.NameSlice) (expression.Expression, error) { var is infoschema.InfoSchema - if sctx.GetSessionVars().TxnCtx.InfoSchema != nil { - is = sctx.GetSessionVars().TxnCtx.InfoSchema.(infoschema.InfoSchema) + // in tests, it may be null + if s, ok := sctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema); ok { + is = s } b, savedBlockNames := NewPlanBuilder(sctx, is, &hint.BlockHintProcessor{}) fakePlan := LogicalTableDual{}.Init(sctx, 0) diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index f717888caf2c9..2d167906c7177 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -1205,7 +1205,7 @@ func (s *testIntegrationSuite) TestPartitionPruningForEQ(c *C) { tk.MustExec("drop table if exists t") tk.MustExec("create table t(a datetime, b int) partition by range(weekday(a)) (partition p0 values less than(10), partition p1 values less than (100))") - is := infoschema.GetInfoSchema(tk.Se) + is := tk.Se.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) pt := tbl.(table.PartitionedTable) diff --git a/planner/core/point_get_plan.go b/planner/core/point_get_plan.go index e2d74d92376ad..77a853a4568b6 100644 --- a/planner/core/point_get_plan.go +++ b/planner/core/point_get_plan.go @@ -1007,7 +1007,7 @@ func checkFastPlanPrivilege(ctx sessionctx.Context, dbName, tableName string, ch }) } - infoSchema := infoschema.GetInfoSchema(ctx) + infoSchema := ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) return CheckTableLock(ctx, infoSchema, visitInfos) } @@ -1313,7 +1313,7 @@ func buildPointUpdatePlan(ctx sessionctx.Context, pointPlan PhysicalPlan, dbName VirtualAssignmentsOffset: len(orderedList), }.Init(ctx) updatePlan.names = pointPlan.OutputNames() - is := infoschema.GetInfoSchema(ctx) + is := ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) t, _ := is.TableByID(tbl.ID) updatePlan.tblID2Table = map[int64]table.Table{ tbl.ID: t, @@ -1509,7 +1509,7 @@ func getHashPartitionColumnName(ctx sessionctx.Context, tbl *model.TableInfo) *a if pi.Type != model.PartitionTypeHash { return nil } - is := infoschema.GetInfoSchema(ctx) + is := ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) table, ok := is.TableByID(tbl.ID) if !ok { return nil diff --git a/planner/core/prepare_test.go b/planner/core/prepare_test.go index 537f1e06468bd..cd43b3964d59b 100644 --- a/planner/core/prepare_test.go +++ b/planner/core/prepare_test.go @@ -201,7 +201,7 @@ func (s *testPlanSerialSuite) TestPrepareCacheDeferredFunction(c *C) { for i := 0; i < 2; i++ { stmt, err := s.ParseOneStmt(sql1, "", "") c.Check(err, IsNil) - is := tk.Se.GetSessionVars().TxnCtx.InfoSchema.(infoschema.InfoSchema) + is := tk.Se.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) builder, _ := core.NewPlanBuilder(tk.Se, is, &hint.BlockHintProcessor{}) p, err := builder.Build(ctx, stmt) c.Check(err, IsNil) diff --git a/session/session.go b/session/session.go index 13df91510f61a..94582fbcb6886 100644 --- a/session/session.go +++ b/session/session.go @@ -406,7 +406,7 @@ func (s *session) StoreIndexUsage(tblID int64, idxID int64, rowsSelected int64) // FieldList returns fields list of a table. func (s *session) FieldList(tableName string) ([]*ast.ResultField, error) { - is := infoschema.GetInfoSchema(s) + is := s.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) dbName := model.NewCIStr(s.GetSessionVars().CurrentDB) tName := model.NewCIStr(tableName) pm := privilege.GetPrivilegeManager(s) @@ -1660,7 +1660,7 @@ func (s *session) PrepareStmt(sql string) (stmtID uint32, paramCount int, fields // So we have to call PrepareTxnCtx here. s.PrepareTxnCtx(ctx) s.PrepareTSFuture(ctx) - prepareExec := executor.NewPrepareExec(s, infoschema.GetInfoSchema(s), sql) + prepareExec := executor.NewPrepareExec(s, s.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema), sql) err = prepareExec.Next(ctx, nil) if err != nil { return @@ -1701,7 +1701,7 @@ func (s *session) cachedPlanExec(ctx context.Context, if prepareStmt.ForUpdateRead { is = domain.GetDomain(s).InfoSchema() } else { - is = infoschema.GetInfoSchema(s) + is = s.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) } execAst := &ast.ExecuteStmt{ExecID: stmtID} if err := executor.ResetContextOfStmt(s, execAst); err != nil { @@ -1781,7 +1781,7 @@ func (s *session) IsCachedExecOk(ctx context.Context, preparedStmt *plannercore. return false, nil } // check schema version - is := infoschema.GetInfoSchema(s) + is := s.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) if prepared.SchemaVersion != is.SchemaMetaVersion() { prepared.CachedPlan = nil return false, nil @@ -2892,7 +2892,7 @@ func (s *session) checkPlacementPolicyBeforeCommit() error { txnScope = oracle.GlobalTxnScope } if txnScope != oracle.GlobalTxnScope { - is := infoschema.GetInfoSchema(s) + is := s.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) deltaMap := s.GetSessionVars().TxnCtx.TableDeltaMap for physicalTableID := range deltaMap { var tableName string diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 1554f3c429d65..89b31da2e8bbd 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -46,11 +46,13 @@ import ( "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/execdetails" + "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/rowcodec" "github.com/pingcap/tidb/util/stringutil" "github.com/pingcap/tidb/util/timeutil" "github.com/twmb/murmur3" atomic2 "go.uber.org/atomic" + "go.uber.org/zap" ) // PreparedStmtCount is exported for test. @@ -868,6 +870,23 @@ func (s *SessionVars) BuildParserConfig() parser.ParserConfig { } } +// GetInfoSchema returns snapshotInfoSchema if snapshot schema is set. +// Otherwise, transaction infoschema is returned. +// Nil if there is no available infoschema. +func (s *SessionVars) GetInfoSchema() interface{} { + type IS interface { + SchemaMetaVersion() int64 + } + if snap, ok := s.SnapshotInfoschema.(IS); ok { + logutil.BgLogger().Info("use snapshot schema", zap.Uint64("conn", s.ConnectionID), zap.Int64("schemaVersion", snap.SchemaMetaVersion())) + return snap + } + if s.TxnCtx != nil && s.TxnCtx.InfoSchema != nil { + return s.TxnCtx.InfoSchema + } + return nil +} + // PartitionPruneMode presents the prune mode used. type PartitionPruneMode string diff --git a/statistics/handle/ddl.go b/statistics/handle/ddl.go index 168c8a07daad1..34f30c1241e1c 100644 --- a/statistics/handle/ddl.go +++ b/statistics/handle/ddl.go @@ -74,7 +74,7 @@ var analyzeOptionDefault = map[ast.AnalyzeOptionType]uint64{ func (h *Handle) updateGlobalStats(tblInfo *model.TableInfo) error { // We need to merge the partition-level stats to global-stats when we drop table partition in dynamic mode. tableID := tblInfo.ID - is := infoschema.GetInfoSchema(h.mu.ctx) + is := h.mu.ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) globalStats, err := h.TableStatsFromStorage(tblInfo, tableID, true, 0) if err != nil { return err diff --git a/statistics/handle/update.go b/statistics/handle/update.go index c65f0885877f6..6f472fc61fdc7 100644 --- a/statistics/handle/update.go +++ b/statistics/handle/update.go @@ -480,7 +480,7 @@ func (h *Handle) dumpTableStatCountToKV(id int64, delta variable.TableDelta) (up affectedRows := h.mu.ctx.GetSessionVars().StmtCtx.AffectedRows() // if it's a partitioned table and its global-stats exists, update its count and modify_count as well. - is := infoschema.GetInfoSchema(h.mu.ctx) + is := h.mu.ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) if is == nil { return false, errors.New("cannot get the information schema") } From b8cad01bef3019dd6ba44ded294b990be73b022d Mon Sep 17 00:00:00 2001 From: Morgan Tocker Date: Mon, 10 May 2021 08:08:42 -0600 Subject: [PATCH 44/67] privilege: fix RequestVerificationWithUser use of default roles (#24442) --- privilege/privileges/privileges.go | 3 ++- privilege/privileges/privileges_test.go | 28 +++++++++++++++++++++++++ 2 files changed, 30 insertions(+), 1 deletion(-) diff --git a/privilege/privileges/privileges.go b/privilege/privileges/privileges.go index 5b7917e802aac..c5ec2f8394385 100644 --- a/privilege/privileges/privileges.go +++ b/privilege/privileges/privileges.go @@ -140,7 +140,8 @@ func (p *UserPrivileges) RequestVerificationWithUser(db, table, column string, p } mysqlPriv := p.Handle.Get() - return mysqlPriv.RequestVerification(nil, user.Username, user.Hostname, db, table, column, priv) + roles := mysqlPriv.getDefaultRoles(user.Username, user.Hostname) + return mysqlPriv.RequestVerification(roles, user.Username, user.Hostname, db, table, column, priv) } // GetEncodedPassword implements the Manager interface. diff --git a/privilege/privileges/privileges_test.go b/privilege/privileges/privileges_test.go index 816fe5a59d0bd..2efb565b3ed2b 100644 --- a/privilege/privileges/privileges_test.go +++ b/privilege/privileges/privileges_test.go @@ -1393,3 +1393,31 @@ func (s *testPrivilegeSuite) TestSecurityEnhancedModeStatusVars(c *C) { AuthHostname: "%", }, nil, nil) } + +// TestViewDefiner tests that default roles are correctly applied in the algorithm definer +// See: https://github.com/pingcap/tidb/issues/24414 +func (s *testPrivilegeSuite) TestViewDefiner(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("CREATE DATABASE issue24414") + tk.MustExec("USE issue24414") + tk.MustExec(`create table table1( + col1 int, + col2 int, + col3 int + )`) + tk.MustExec(`insert into table1 values (1,1,1),(2,2,2)`) + tk.MustExec(`CREATE ROLE 'ACL-mobius-admin'`) + tk.MustExec(`CREATE USER 'mobius-admin'`) + tk.MustExec(`CREATE USER 'mobius-admin-no-role'`) + tk.MustExec(`GRANT Select,Insert,Update,Delete,Create,Drop,Alter,Index,Create View,Show View ON issue24414.* TO 'ACL-mobius-admin'@'%'`) + tk.MustExec(`GRANT Select,Insert,Update,Delete,Create,Drop,Alter,Index,Create View,Show View ON issue24414.* TO 'mobius-admin-no-role'@'%'`) + tk.MustExec(`GRANT 'ACL-mobius-admin'@'%' to 'mobius-admin'@'%'`) + tk.MustExec(`SET DEFAULT ROLE ALL TO 'mobius-admin'`) + // create tables + tk.MustExec(`CREATE ALGORITHM = UNDEFINED DEFINER = 'mobius-admin'@'127.0.0.1' SQL SECURITY DEFINER VIEW test_view (col1 , col2 , col3) AS SELECT * from table1`) + tk.MustExec(`CREATE ALGORITHM = UNDEFINED DEFINER = 'mobius-admin-no-role'@'127.0.0.1' SQL SECURITY DEFINER VIEW test_view2 (col1 , col2 , col3) AS SELECT * from table1`) + + // all examples should work + tk.MustExec("select * from test_view") + tk.MustExec("select * from test_view2") +} From c6c8265e098b65339a85be2e35678b1c13f0f53a Mon Sep 17 00:00:00 2001 From: disksing Date: Mon, 10 May 2021 22:51:37 +0800 Subject: [PATCH 45/67] store/tikv: remove use of TaskID transaction option in store/tikv (#24407) --- store/driver/txn/snapshot.go | 2 ++ store/driver/txn/txn_driver.go | 2 ++ store/tikv/snapshot.go | 12 ++++++++---- 3 files changed, 12 insertions(+), 4 deletions(-) diff --git a/store/driver/txn/snapshot.go b/store/driver/txn/snapshot.go index 1a3f7bf9c3bbc..6064db02cdeba 100644 --- a/store/driver/txn/snapshot.go +++ b/store/driver/txn/snapshot.go @@ -73,6 +73,8 @@ func (s *tikvSnapshot) SetOption(opt int, val interface{}) { s.KVSnapshot.SetNotFillCache(val.(bool)) case tikvstore.SnapshotTS: s.KVSnapshot.SetSnapshotTS(val.(uint64)) + case tikvstore.TaskID: + s.KVSnapshot.SetTaskID(val.(uint64)) default: s.KVSnapshot.SetOption(opt, val) } diff --git a/store/driver/txn/txn_driver.go b/store/driver/txn/txn_driver.go index 8595b2fe874ae..50bba80d2b54e 100644 --- a/store/driver/txn/txn_driver.go +++ b/store/driver/txn/txn_driver.go @@ -144,6 +144,8 @@ func (txn *tikvTxn) SetOption(opt int, val interface{}) { txn.SetPessimistic(val.(bool)) case tikvstore.SnapshotTS: txn.KVTxn.GetSnapshot().SetSnapshotTS(val.(uint64)) + case tikvstore.TaskID: + txn.KVTxn.GetSnapshot().SetTaskID(val.(uint64)) case tikvstore.InfoSchema: txn.SetSchemaVer(val.(tikv.SchemaVer)) case tikvstore.CommitHook: diff --git a/store/tikv/snapshot.go b/store/tikv/snapshot.go index 2b9926c7a2b9a..a37e1d8343c5e 100644 --- a/store/tikv/snapshot.go +++ b/store/tikv/snapshot.go @@ -569,10 +569,6 @@ func (s *KVSnapshot) SetOption(opt int, val interface{}) { s.mu.Lock() s.mu.replicaRead = val.(kv.ReplicaReadType) s.mu.Unlock() - case kv.TaskID: - s.mu.Lock() - s.mu.taskID = val.(uint64) - s.mu.Unlock() case kv.CollectRuntimeStats: s.mu.Lock() s.mu.stats = val.(*SnapshotRuntimeStats) @@ -625,6 +621,14 @@ func (s *KVSnapshot) SetPriority(pri Priority) { s.priority = pri } +// SetTaskID marks current task's unique ID to allow TiKV to schedule +// tasks more fairly. +func (s *KVSnapshot) SetTaskID(id uint64) { + s.mu.Lock() + defer s.mu.Unlock() + s.mu.taskID = id +} + // SnapCacheHitCount gets the snapshot cache hit count. Only for test. func (s *KVSnapshot) SnapCacheHitCount() int { return int(atomic.LoadInt64(&s.mu.hitCnt)) From f135c534a50cc3b3242047aee8860e81cc1a5a05 Mon Sep 17 00:00:00 2001 From: Zhuhe Fang Date: Mon, 10 May 2021 23:25:37 +0800 Subject: [PATCH 46/67] plan: merge continuous selections and delete surely true expressions (#24214) --- cmd/explaintest/r/explain_easy.result | 6 +- executor/executor_test.go | 2 +- expression/integration_test.go | 10 +- expression/testdata/expression_suite_out.json | 4 +- go.sum | 1 + planner/core/integration_test.go | 38 +++++++ planner/core/optimizer.go | 26 +++++ planner/core/rule_predicate_push_down.go | 31 ++++++ .../testdata/integration_serial_suite_in.json | 6 + .../integration_serial_suite_out.json | 28 +++++ .../core/testdata/partition_pruner_out.json | 103 +++++++++--------- planner/core/testdata/plan_suite_out.json | 6 +- planner/core/testdata/point_get_plan_out.json | 3 +- 13 files changed, 195 insertions(+), 69 deletions(-) diff --git a/cmd/explaintest/r/explain_easy.result b/cmd/explaintest/r/explain_easy.result index a33fe5a791bbc..927e25c8024f2 100644 --- a/cmd/explaintest/r/explain_easy.result +++ b/cmd/explaintest/r/explain_easy.result @@ -108,9 +108,9 @@ HashJoin 9990.00 root inner join, equal:[eq(test.t1.c1, test.t2.c2)] └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo explain format = 'brief' select (select count(1) k from t1 s where s.c1 = t1.c1 having k != 0) from t1; id estRows task access object operator info -Projection 10000.00 root ifnull(Column#10, 0)->Column#10 -└─MergeJoin 10000.00 root left outer join, left key:test.t1.c1, right key:test.t1.c1 - ├─Projection(Build) 8000.00 root 1->Column#10, test.t1.c1 +Projection 12500.00 root ifnull(Column#10, 0)->Column#10 +└─MergeJoin 12500.00 root left outer join, left key:test.t1.c1, right key:test.t1.c1 + ├─Projection(Build) 10000.00 root 1->Column#10, test.t1.c1 │ └─TableReader 10000.00 root data:TableFullScan │ └─TableFullScan 10000.00 cop[tikv] table:s keep order:true, stats:pseudo └─TableReader(Probe) 10000.00 root data:TableFullScan diff --git a/executor/executor_test.go b/executor/executor_test.go index 823967f24f217..e69b956f8d82b 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -8128,7 +8128,7 @@ func (s *testSerialSuite) TestIssue24210(c *C) { // for SelectionExec c.Assert(failpoint.Enable("github.com/pingcap/tidb/executor/mockSelectionExecBaseExecutorOpenReturnedError", `return(true)`), IsNil) - _, err = tk.Exec("select * from (select 1 as a) t where a > 0") + _, err = tk.Exec("select * from (select rand() as a) t where a > 0") c.Assert(err, NotNil) c.Assert(err.Error(), Equals, "mock SelectionExec.baseExecutor.Open returned error") err = failpoint.Disable("github.com/pingcap/tidb/executor/mockSelectionExecBaseExecutorOpenReturnedError") diff --git a/expression/integration_test.go b/expression/integration_test.go index 76ba37b49d4bd..d4b6a031087e4 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -5474,16 +5474,14 @@ func (s *testIntegrationSuite) TestExprPushdownBlacklist(c *C) { // > pushed to both TiKV and TiFlash rows := tk.MustQuery("explain format = 'brief' select * from test.t where b > date'1988-01-01' and b < date'1994-01-01' " + "and cast(a as decimal(10,2)) > 10.10 and date_format(b,'%m') = '11'").Rows() - c.Assert(fmt.Sprintf("%v", rows[0][4]), Equals, "lt(test.t.b, 1994-01-01)") - c.Assert(fmt.Sprintf("%v", rows[1][4]), Equals, "gt(cast(test.t.a, decimal(10,2) BINARY), 10.10)") - c.Assert(fmt.Sprintf("%v", rows[3][4]), Equals, "eq(date_format(test.t.b, \"%m\"), \"11\"), gt(test.t.b, 1988-01-01)") + c.Assert(fmt.Sprintf("%v", rows[0][4]), Equals, "gt(cast(test.t.a, decimal(10,2) BINARY), 10.10), lt(test.t.b, 1994-01-01)") + c.Assert(fmt.Sprintf("%v", rows[2][4]), Equals, "eq(date_format(test.t.b, \"%m\"), \"11\"), gt(test.t.b, 1988-01-01)") tk.MustExec("set @@session.tidb_isolation_read_engines = 'tikv'") rows = tk.MustQuery("explain format = 'brief' select * from test.t where b > date'1988-01-01' and b < date'1994-01-01' " + "and cast(a as decimal(10,2)) > 10.10 and date_format(b,'%m') = '11'").Rows() - c.Assert(fmt.Sprintf("%v", rows[0][4]), Equals, "lt(test.t.b, 1994-01-01)") - c.Assert(fmt.Sprintf("%v", rows[1][4]), Equals, "eq(date_format(test.t.b, \"%m\"), \"11\")") - c.Assert(fmt.Sprintf("%v", rows[3][4]), Equals, "gt(cast(test.t.a, decimal(10,2) BINARY), 10.10), gt(test.t.b, 1988-01-01)") + c.Assert(fmt.Sprintf("%v", rows[0][4]), Equals, "eq(date_format(test.t.b, \"%m\"), \"11\"), lt(test.t.b, 1994-01-01)") + c.Assert(fmt.Sprintf("%v", rows[2][4]), Equals, "gt(cast(test.t.a, decimal(10,2) BINARY), 10.10), gt(test.t.b, 1988-01-01)") tk.MustExec("delete from mysql.expr_pushdown_blacklist where name = '<' and store_type = 'tikv,tiflash,tidb' and reason = 'for test'") tk.MustExec("delete from mysql.expr_pushdown_blacklist where name = 'date_format' and store_type = 'tikv' and reason = 'for test'") diff --git a/expression/testdata/expression_suite_out.json b/expression/testdata/expression_suite_out.json index ea36fc2923764..8b89ee1e4dc81 100644 --- a/expression/testdata/expression_suite_out.json +++ b/expression/testdata/expression_suite_out.json @@ -186,7 +186,7 @@ { "SQL": "explain format = 'brief' select * from t1 left join t2 on true where t1.a = 1 and t1.a = 1", "Result": [ - "HashJoin 80000.00 root CARTESIAN left outer join", + "HashJoin 100000.00 root CARTESIAN left outer join", "├─TableReader(Build) 10.00 root data:Selection", "│ └─Selection 10.00 cop[tikv] eq(test.t1.a, 1)", "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", @@ -253,7 +253,7 @@ { "SQL": "explain format = 'brief' select * from t1 left join t2 on true where t1.a = 1 or (t1.a = 2 and t1.a = 3)", "Result": [ - "HashJoin 80000.00 root CARTESIAN left outer join", + "HashJoin 100000.00 root CARTESIAN left outer join", "├─TableReader(Build) 10.00 root data:Selection", "│ └─Selection 10.00 cop[tikv] or(eq(test.t1.a, 1), 0)", "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", diff --git a/go.sum b/go.sum index 74b4f623789b8..a3ebad580db64 100644 --- a/go.sum +++ b/go.sum @@ -500,6 +500,7 @@ github.com/russross/blackfriday/v2 v2.0.1/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQD github.com/sasha-s/go-deadlock v0.2.0/go.mod h1:StQn567HiB1fF2yJ44N9au7wOhrPS3iZqiDbRupzT10= github.com/satori/go.uuid v1.2.0/go.mod h1:dA0hQrYB0VpLJoorglMZABFdXlWrHn1NEOzdhQKdks0= github.com/sergi/go-diff v1.0.1-0.20180205163309-da645544ed44/go.mod h1:0CfEIISq7TuYL3j771MWULgwwjU+GofnZX9QAmXWZgo= +github.com/sergi/go-diff v1.1.0 h1:we8PVUC3FE2uYfodKH/nBHMSetSfHDR6scGdBi+erh0= github.com/sergi/go-diff v1.1.0/go.mod h1:STckp+ISIX8hZLjrqAeVduY0gWCT9IjLuqbuNXdaHfM= github.com/shirou/gopsutil v2.19.10+incompatible/go.mod h1:5b4v6he4MtMOwMlS0TUMTu2PcXUg8+E1lC7eC3UO/RA= github.com/shirou/gopsutil v3.21.2+incompatible h1:U+YvJfjCh6MslYlIAXvPtzhW3YZEtc9uncueUNpD/0A= diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index 2d167906c7177..4ae7342a4f7d7 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -3651,3 +3651,41 @@ func (s *testIntegrationSuite) TestSequenceAsDataSource(c *C) { tk.MustQuery("explain format = 'brief' " + tt).Check(testkit.Rows(output[i].Plan...)) } } + +func (s *testIntegrationSerialSuite) TestMergeContinuousSelections(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists ts") + tk.MustExec("create table ts (col_char_64 char(64), col_varchar_64_not_null varchar(64) not null, col_varchar_key varchar(1), id int primary key, col_varchar_64 varchar(64),col_char_64_not_null char(64) not null);") + + // Create virtual tiflash replica info. + dom := domain.GetDomain(tk.Se) + is := dom.InfoSchema() + db, exists := is.SchemaByName(model.NewCIStr("test")) + c.Assert(exists, IsTrue) + for _, tblInfo := range db.Tables { + if tblInfo.Name.L == "ts" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + tk.MustExec(" set @@tidb_allow_mpp=1;") + + var input []string + var output []struct { + SQL string + Plan []string + } + s.testData.GetTestCases(c, &input, &output) + for i, tt := range input { + s.testData.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + }) + res := tk.MustQuery(tt) + res.Check(testkit.Rows(output[i].Plan...)) + } +} diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index 2ad7fc1136d5b..d79d83331723b 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/infoschema" + "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/lock" "github.com/pingcap/tidb/planner/property" "github.com/pingcap/tidb/privilege" @@ -156,9 +157,34 @@ func DoOptimize(ctx context.Context, sctx sessionctx.Context, flag uint64, logic return finalPlan, cost, nil } +// mergeContinuousSelections merge continuous selections which may occur after changing plans. +func mergeContinuousSelections(p PhysicalPlan) { + if sel, ok := p.(*PhysicalSelection); ok { + for { + childSel := sel.children[0] + if tmp, ok := childSel.(*PhysicalSelection); ok { + sel.Conditions = append(sel.Conditions, tmp.Conditions...) + sel.SetChild(0, tmp.children[0]) + } else { + break + } + } + } + for _, child := range p.Children() { + mergeContinuousSelections(child) + } + // merge continuous selections in a coprocessor task of tiflash + tableReader, isTableReader := p.(*PhysicalTableReader) + if isTableReader && tableReader.StoreType == kv.TiFlash { + mergeContinuousSelections(tableReader.tablePlan) + tableReader.TablePlans = flattenPushDownPlan(tableReader.tablePlan) + } +} + func postOptimize(sctx sessionctx.Context, plan PhysicalPlan) PhysicalPlan { plan = eliminatePhysicalProjection(plan) plan = InjectExtraProjection(plan) + mergeContinuousSelections(plan) plan = eliminateUnionScanAndLock(sctx, plan) plan = enableParallelApply(sctx, plan) return plan diff --git a/planner/core/rule_predicate_push_down.go b/planner/core/rule_predicate_push_down.go index 936c4720cbc87..a3c6737db74d7 100644 --- a/planner/core/rule_predicate_push_down.go +++ b/planner/core/rule_predicate_push_down.go @@ -42,6 +42,12 @@ func addSelection(p LogicalPlan, child LogicalPlan, conditions []expression.Expr p.Children()[chIdx] = dual return } + + conditions = DeleteTrueExprs(p, conditions) + if len(conditions) == 0 { + p.Children()[chIdx] = child + return + } selection := LogicalSelection{Conditions: conditions}.Init(p.SCtx(), p.SelectBlockOffset()) selection.SetChildren(child) p.Children()[chIdx] = selection @@ -73,6 +79,8 @@ func splitSetGetVarFunc(filters []expression.Expression) ([]expression.Expressio // PredicatePushDown implements LogicalPlan PredicatePushDown interface. func (p *LogicalSelection) PredicatePushDown(predicates []expression.Expression) ([]expression.Expression, LogicalPlan) { + predicates = DeleteTrueExprs(p, predicates) + p.Conditions = DeleteTrueExprs(p, p.Conditions) canBePushDown, canNotBePushDown := splitSetGetVarFunc(p.Conditions) retConditions, child := p.children[0].PredicatePushDown(append(canBePushDown, predicates...)) retConditions = append(retConditions, canNotBePushDown...) @@ -100,6 +108,7 @@ func (p *LogicalUnionScan) PredicatePushDown(predicates []expression.Expression) // PredicatePushDown implements LogicalPlan PredicatePushDown interface. func (ds *DataSource) PredicatePushDown(predicates []expression.Expression) ([]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) return predicates, ds @@ -532,6 +541,28 @@ func Conds2TableDual(p LogicalPlan, conds []expression.Expression) LogicalPlan { return nil } +// DeleteTrueExprs deletes the surely true expressions +func DeleteTrueExprs(p LogicalPlan, conds []expression.Expression) []expression.Expression { + newConds := make([]expression.Expression, 0, len(conds)) + for _, cond := range conds { + con, ok := cond.(*expression.Constant) + if !ok { + newConds = append(newConds, cond) + continue + } + if expression.ContainMutableConst(p.SCtx(), []expression.Expression{con}) { + newConds = append(newConds, cond) + continue + } + sc := p.SCtx().GetSessionVars().StmtCtx + if isTrue, err := con.Value.ToBool(sc); err == nil && isTrue == 1 { + continue + } + newConds = append(newConds, cond) + } + return newConds +} + // outerJoinPropConst propagates constant equal and column equal conditions over outer join. func (p *LogicalJoin) outerJoinPropConst(predicates []expression.Expression) []expression.Expression { outerTable := p.children[0] diff --git a/planner/core/testdata/integration_serial_suite_in.json b/planner/core/testdata/integration_serial_suite_in.json index 3234652e5d000..34e50df03661b 100644 --- a/planner/core/testdata/integration_serial_suite_in.json +++ b/planner/core/testdata/integration_serial_suite_in.json @@ -300,5 +300,11 @@ "cases": [ "select (2) in (select b from t) from (select t.a < (select t.a from t t1 limit 1) from t) t" ] + }, + { + "name": "TestMergeContinuousSelections", + "cases": [ + "desc format = 'brief' SELECT table2 . `col_char_64` AS field1 FROM `ts` AS table2 INNER JOIN (SELECT DISTINCT SUBQUERY3_t1 . * FROM `ts` AS SUBQUERY3_t1 LEFT OUTER JOIN `ts` AS SUBQUERY3_t2 ON SUBQUERY3_t2 . `col_varchar_64_not_null` = SUBQUERY3_t1 . `col_varchar_key`) AS table3 ON (table3 . `col_varchar_key` = table2 . `col_varchar_64`) WHERE table3 . `col_char_64_not_null` >= SOME (SELECT SUBQUERY4_t1 . `col_varchar_64` AS SUBQUERY4_field1 FROM `ts` AS SUBQUERY4_t1) GROUP BY field1 ;" + ] } ] diff --git a/planner/core/testdata/integration_serial_suite_out.json b/planner/core/testdata/integration_serial_suite_out.json index 8b67310894d4b..1f25f899a68d9 100644 --- a/planner/core/testdata/integration_serial_suite_out.json +++ b/planner/core/testdata/integration_serial_suite_out.json @@ -2608,5 +2608,33 @@ ] } ] + }, + { + "Name": "TestMergeContinuousSelections", + "Cases": [ + { + "SQL": "desc format = 'brief' SELECT table2 . `col_char_64` AS field1 FROM `ts` AS table2 INNER JOIN (SELECT DISTINCT SUBQUERY3_t1 . * FROM `ts` AS SUBQUERY3_t1 LEFT OUTER JOIN `ts` AS SUBQUERY3_t2 ON SUBQUERY3_t2 . `col_varchar_64_not_null` = SUBQUERY3_t1 . `col_varchar_key`) AS table3 ON (table3 . `col_varchar_key` = table2 . `col_varchar_64`) WHERE table3 . `col_char_64_not_null` >= SOME (SELECT SUBQUERY4_t1 . `col_varchar_64` AS SUBQUERY4_field1 FROM `ts` AS SUBQUERY4_t1) GROUP BY field1 ;", + "Plan": [ + "HashAgg 7992.00 root group by:test.ts.col_char_64, funcs:firstrow(test.ts.col_char_64)->test.ts.col_char_64", + "└─HashJoin 9990.00 root CARTESIAN inner join, other cond:or(ge(test.ts.col_char_64_not_null, Column#25), if(ne(Column#26, 0), NULL, 0))", + " ├─Selection(Build) 0.80 root ne(Column#27, 0)", + " │ └─HashAgg 1.00 root funcs:min(Column#33)->Column#25, funcs:sum(Column#34)->Column#26, funcs:count(Column#35)->Column#27", + " │ └─TableReader 1.00 root data:ExchangeSender", + " │ └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " │ └─HashAgg 1.00 batchCop[tiflash] funcs:min(Column#39)->Column#33, funcs:sum(Column#40)->Column#34, funcs:count(1)->Column#35", + " │ └─Projection 10000.00 batchCop[tiflash] test.ts.col_varchar_64, cast(isnull(test.ts.col_varchar_64), decimal(22,0) BINARY)->Column#40", + " │ └─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", + " └─HashJoin 12487.50 cop[tiflash] inner join, equal:[eq(test.ts.col_varchar_64, test.ts.col_varchar_key)]", + " ├─ExchangeReceiver(Build) 9990.00 cop[tiflash] ", + " │ └─ExchangeSender 9990.00 cop[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 cop[tiflash] not(isnull(test.ts.col_varchar_64))", + " │ └─TableFullScan 10000.00 cop[tiflash] table:table2 keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 cop[tiflash] not(isnull(test.ts.col_varchar_key))", + " └─TableFullScan 10000.00 cop[tiflash] table:SUBQUERY3_t1 keep order:false, stats:pseudo" + ] + } + ] } ] diff --git a/planner/core/testdata/partition_pruner_out.json b/planner/core/testdata/partition_pruner_out.json index 5593ab52bee74..962894b5c35c9 100644 --- a/planner/core/testdata/partition_pruner_out.json +++ b/planner/core/testdata/partition_pruner_out.json @@ -714,13 +714,13 @@ "SQL": "select * from t1 join t2 on true where t1.a=5 and t2.a in (6,7,8) and t1.a-t2.a=1 and t2.b = 6", "Result": null, "Plan": [ - "Projection 80.00 root test_partition.t1.id, test_partition.t1.a, test_partition.t1.b, test_partition.t2.a, test_partition.t2.id, test_partition.t2.b", - "└─HashJoin 80.00 root CARTESIAN inner join", - " ├─TableReader(Build) 8.00 root partition:p1 data:Selection", - " │ └─Selection 8.00 cop[tikv] 1, eq(minus(5, test_partition.t2.a), 1), eq(test_partition.t2.b, 6), in(test_partition.t2.a, 6, 7, 8)", + "Projection 0.24 root test_partition.t1.id, test_partition.t1.a, test_partition.t1.b, test_partition.t2.a, test_partition.t2.id, test_partition.t2.b", + "└─HashJoin 0.24 root CARTESIAN inner join", + " ├─TableReader(Build) 0.02 root partition:p1 data:Selection", + " │ └─Selection 0.02 cop[tikv] eq(minus(5, test_partition.t2.a), 1), eq(test_partition.t2.b, 6), in(test_partition.t2.a, 6, 7, 8)", " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", " └─TableReader(Probe) 10.00 root partition:p0 data:Selection", - " └─Selection 10.00 cop[tikv] 1, eq(test_partition.t1.a, 5)", + " └─Selection 10.00 cop[tikv] eq(test_partition.t1.a, 5)", " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ] }, @@ -1129,25 +1129,24 @@ "3 3 3 7 7 7" ], "Plan": [ - "Sort 80.16 root test_partition.t1.id, test_partition.t1.a", - "└─Projection 80.16 root test_partition.t1.id, test_partition.t1.a, test_partition.t1.b, test_partition.t2.id, test_partition.t2.a, test_partition.t2.b", - " └─HashJoin 80.16 root CARTESIAN inner join", - " ├─TableReader(Build) 8.00 root partition:p1 data:Selection", - " │ └─Selection 8.00 cop[tikv] 1, eq(test_partition.t2.b, 7), eq(test_partition.t2.id, 7), in(test_partition.t2.a, 6, 7, 8)", + "Sort 0.00 root test_partition.t1.id, test_partition.t1.a", + "└─Projection 0.00 root test_partition.t1.id, test_partition.t1.a, test_partition.t1.b, test_partition.t2.id, test_partition.t2.a, test_partition.t2.b", + " └─HashJoin 0.00 root CARTESIAN inner join", + " ├─TableReader(Build) 0.00 root partition:p1 data:Selection", + " │ └─Selection 0.00 cop[tikv] eq(test_partition.t2.b, 7), eq(test_partition.t2.id, 7), in(test_partition.t2.a, 6, 7, 8)", " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", " └─TableReader(Probe) 10.02 root partition:p0 data:Selection", - " └─Selection 10.02 cop[tikv] 1, or(eq(test_partition.t1.a, 1), and(eq(test_partition.t1.a, 3), in(test_partition.t1.b, 3, 5)))", + " └─Selection 10.02 cop[tikv] or(eq(test_partition.t1.a, 1), and(eq(test_partition.t1.a, 3), in(test_partition.t1.b, 3, 5)))", " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], "IndexPlan": [ - "Sort 0.05 root test_partition_1.t1.id, test_partition_1.t1.a", - "└─HashJoin 0.05 root CARTESIAN inner join", - " ├─IndexReader(Build) 0.02 root partition:p0 index:Selection", - " │ └─Selection 0.02 cop[tikv] or(eq(test_partition_1.t1.a, 1), and(eq(test_partition_1.t1.a, 3), in(test_partition_1.t1.b, 3, 5)))", - " │ └─IndexRangeScan 20.00 cop[tikv] table:t1, index:a(a, b, id) range:[1,1], [3,3], keep order:false, stats:pseudo", - " └─IndexReader(Probe) 2.40 root partition:p1 index:Selection", - " └─Selection 2.40 cop[tikv] 1", - " └─IndexRangeScan 3.00 cop[tikv] table:t2, index:a(a, b, id) range:[6 7 7,6 7 7], [7 7 7,7 7 7], [8 7 7,8 7 7], keep order:false, stats:pseudo" + "Sort 30.60 root test_partition_1.t1.id, test_partition_1.t1.a", + "└─Projection 30.60 root test_partition_1.t1.id, test_partition_1.t1.a, test_partition_1.t1.b, test_partition_1.t2.id, test_partition_1.t2.a, test_partition_1.t2.b", + " └─HashJoin 30.60 root CARTESIAN inner join", + " ├─IndexReader(Build) 3.00 root partition:p1 index:IndexRangeScan", + " │ └─IndexRangeScan 3.00 cop[tikv] table:t2, index:a(a, b, id) range:[6 7 7,6 7 7], [7 7 7,7 7 7], [8 7 7,8 7 7], keep order:false, stats:pseudo", + " └─IndexReader(Probe) 10.20 root partition:p0 index:IndexRangeScan", + " └─IndexRangeScan 10.20 cop[tikv] table:t1, index:a(a, b, id) range:[1,1], [3 3,3 3], [3 5,3 5], keep order:false, stats:pseudo" ] }, { @@ -1734,22 +1733,22 @@ "5 5 5 6 6 6" ], "Plan": [ - "Projection 80.00 root test_partition.t1.id, test_partition.t1.a, test_partition.t1.b, test_partition.t2.id, test_partition.t2.a, test_partition.t2.b", - "└─HashJoin 80.00 root CARTESIAN inner join", - " ├─TableReader(Build) 8.00 root partition:p1 data:Selection", - " │ └─Selection 8.00 cop[tikv] 1, eq(test_partition.t2.b, 6), in(test_partition.t2.a, 6, 7, 8)", + "Projection 0.30 root test_partition.t1.id, test_partition.t1.a, test_partition.t1.b, test_partition.t2.id, test_partition.t2.a, test_partition.t2.b", + "└─HashJoin 0.30 root CARTESIAN inner join", + " ├─TableReader(Build) 0.03 root partition:p1 data:Selection", + " │ └─Selection 0.03 cop[tikv] eq(test_partition.t2.b, 6), in(test_partition.t2.a, 6, 7, 8)", " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", " └─TableReader(Probe) 10.00 root partition:p0 data:Selection", - " └─Selection 10.00 cop[tikv] 1, eq(test_partition.t1.a, 5)", + " └─Selection 10.00 cop[tikv] eq(test_partition.t1.a, 5)", " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], "IndexPlan": [ - "Projection 300.00 root test_partition_1.t1.id, test_partition_1.t1.a, test_partition_1.t1.b, test_partition_1.t2.id, test_partition_1.t2.a, test_partition_1.t2.b", - "└─HashJoin 300.00 root CARTESIAN inner join", - " ├─IndexReader(Build) 3.00 root partition:p1 index:IndexRangeScan", - " │ └─IndexRangeScan 3.00 cop[tikv] table:t2, index:a(a, b, id) range:[6 6 NULL,6 6 +inf], [7 6 NULL,7 6 +inf], [8 6 NULL,8 6 +inf], keep order:false, stats:pseudo", - " └─IndexReader(Probe) 100.00 root partition:p0 index:IndexRangeScan", - " └─IndexRangeScan 100.00 cop[tikv] table:t1, index:a(a, b, id) range:[5 NULL,5 +inf], keep order:false, stats:pseudo" + "Projection 3.00 root test_partition_1.t1.id, test_partition_1.t1.a, test_partition_1.t1.b, test_partition_1.t2.id, test_partition_1.t2.a, test_partition_1.t2.b", + "└─HashJoin 3.00 root CARTESIAN inner join", + " ├─IndexReader(Build) 0.30 root partition:p1 index:IndexRangeScan", + " │ └─IndexRangeScan 0.30 cop[tikv] table:t2, index:a(a, b, id) range:[6 6,6 6], [7 6,7 6], [8 6,8 6], keep order:false, stats:pseudo", + " └─IndexReader(Probe) 10.00 root partition:p0 index:IndexRangeScan", + " └─IndexRangeScan 10.00 cop[tikv] table:t1, index:a(a, b, id) range:[5,5], keep order:false, stats:pseudo" ] }, { @@ -2127,25 +2126,25 @@ "3 3 3 7 7 7" ], "Plan": [ - "Sort 675761.06 root test_partition.t1.id, test_partition.t1.a", - "└─Projection 675761.06 root test_partition.t1.id, test_partition.t1.a, test_partition.t1.b, test_partition.t2.id, test_partition.t2.a, test_partition.t2.b", - " └─HashJoin 675761.06 root CARTESIAN inner join", - " ├─TableReader(Build) 200.00 root partition:p1 data:Selection", - " │ └─Selection 200.00 cop[tikv] 1, ge(test_partition.t2.a, 6), ge(test_partition.t2.b, 7), ge(test_partition.t2.id, 7), le(test_partition.t2.a, 8)", + "Sort 93855.70 root test_partition.t1.id, test_partition.t1.a", + "└─Projection 93855.70 root test_partition.t1.id, test_partition.t1.a, test_partition.t1.b, test_partition.t2.id, test_partition.t2.a, test_partition.t2.b", + " └─HashJoin 93855.70 root CARTESIAN inner join", + " ├─TableReader(Build) 27.78 root partition:p1 data:Selection", + " │ └─Selection 27.78 cop[tikv] ge(test_partition.t2.a, 6), ge(test_partition.t2.b, 7), ge(test_partition.t2.id, 7), le(test_partition.t2.a, 8)", " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", " └─TableReader(Probe) 3378.81 root partition:p0 data:Selection", - " └─Selection 3378.81 cop[tikv] 1, or(le(test_partition.t1.a, 1), and(le(test_partition.t1.a, 3), and(ge(test_partition.t1.b, 3), le(test_partition.t1.b, 5))))", + " └─Selection 3378.81 cop[tikv] or(le(test_partition.t1.a, 1), and(le(test_partition.t1.a, 3), and(ge(test_partition.t1.b, 3), le(test_partition.t1.b, 5))))", " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], "IndexPlan": [ - "Sort 224577.93 root test_partition_1.t1.id, test_partition_1.t1.a", - "└─Projection 224577.93 root test_partition_1.t1.id, test_partition_1.t1.a, test_partition_1.t1.b, test_partition_1.t2.id, test_partition_1.t2.a, test_partition_1.t2.b", - " └─HashJoin 224577.93 root CARTESIAN inner join", - " ├─IndexReader(Build) 200.00 root partition:p1 index:Selection", - " │ └─Selection 200.00 cop[tikv] ge(test_partition_1.t2.b, 7), ge(test_partition_1.t2.id, 7)", + "Sort 73851.85 root test_partition_1.t1.id, test_partition_1.t1.a", + "└─Projection 73851.85 root test_partition_1.t1.id, test_partition_1.t1.a, test_partition_1.t1.b, test_partition_1.t2.id, test_partition_1.t2.a, test_partition_1.t2.b", + " └─HashJoin 73851.85 root CARTESIAN inner join", + " ├─IndexReader(Build) 27.78 root partition:p1 index:Selection", + " │ └─Selection 27.78 cop[tikv] ge(test_partition_1.t2.b, 7), ge(test_partition_1.t2.id, 7)", " │ └─IndexRangeScan 250.00 cop[tikv] table:t2, index:a(a, b, id) range:[6,8], keep order:false, stats:pseudo", - " └─IndexReader(Probe) 1122.89 root partition:p0 index:Selection", - " └─Selection 1122.89 cop[tikv] or(le(test_partition_1.t1.a, 1), and(le(test_partition_1.t1.a, 3), and(ge(test_partition_1.t1.b, 3), le(test_partition_1.t1.b, 5))))", + " └─IndexReader(Probe) 2658.67 root partition:p0 index:Selection", + " └─Selection 2658.67 cop[tikv] or(le(test_partition_1.t1.a, 1), and(le(test_partition_1.t1.a, 3), and(ge(test_partition_1.t1.b, 3), le(test_partition_1.t1.b, 5))))", " └─IndexRangeScan 3323.33 cop[tikv] table:t1, index:a(a, b, id) range:[-inf,3], keep order:false, stats:pseudo" ] }, @@ -2761,20 +2760,20 @@ "5 5 5 8 8 8" ], "Plan": [ - "HashJoin 2000.00 root CARTESIAN inner join", + "HashJoin 833.33 root CARTESIAN inner join", "├─TableReader(Build) 10.00 root partition:p0 data:Selection", - "│ └─Selection 10.00 cop[tikv] 1, eq(test_partition.t1.a, 5)", + "│ └─Selection 10.00 cop[tikv] eq(test_partition.t1.a, 5)", "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", - "└─TableReader(Probe) 200.00 root partition:p1 data:Selection", - " └─Selection 200.00 cop[tikv] 1, ge(test_partition.t2.a, 6), ge(test_partition.t2.b, 6), le(test_partition.t2.a, 8)", + "└─TableReader(Probe) 83.33 root partition:p1 data:Selection", + " └─Selection 83.33 cop[tikv] ge(test_partition.t2.a, 6), ge(test_partition.t2.b, 6), le(test_partition.t2.a, 8)", " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" ], "IndexPlan": [ - "HashJoin 20000.00 root CARTESIAN inner join", - "├─IndexReader(Build) 100.00 root partition:p0 index:IndexRangeScan", - "│ └─IndexRangeScan 100.00 cop[tikv] table:t1, index:a(a, b, id) range:[5 NULL,5 +inf], keep order:false, stats:pseudo", - "└─IndexReader(Probe) 200.00 root partition:p1 index:Selection", - " └─Selection 200.00 cop[tikv] ge(test_partition_1.t2.b, 6)", + "HashJoin 833.33 root CARTESIAN inner join", + "├─IndexReader(Build) 10.00 root partition:p0 index:IndexRangeScan", + "│ └─IndexRangeScan 10.00 cop[tikv] table:t1, index:a(a, b, id) range:[5,5], keep order:false, stats:pseudo", + "└─IndexReader(Probe) 83.33 root partition:p1 index:Selection", + " └─Selection 83.33 cop[tikv] ge(test_partition_1.t2.b, 6)", " └─IndexRangeScan 250.00 cop[tikv] table:t2, index:a(a, b, id) range:[6,8], keep order:false, stats:pseudo" ] }, diff --git a/planner/core/testdata/plan_suite_out.json b/planner/core/testdata/plan_suite_out.json index a0125e601c94a..ce33bec905757 100644 --- a/planner/core/testdata/plan_suite_out.json +++ b/planner/core/testdata/plan_suite_out.json @@ -818,7 +818,7 @@ }, { "SQL": "select (select count(1) k from t s where s.a = t.a having k != 0) from t", - "Best": "LeftHashJoin{IndexReader(Index(t.f)[[NULL,+inf]])->IndexReader(Index(t.f)[[NULL,+inf]]->Sel([1]))->Projection}(test.t.a,test.t.a)->Projection" + "Best": "LeftHashJoin{IndexReader(Index(t.f)[[NULL,+inf]])->IndexReader(Index(t.f)[[NULL,+inf]])->Projection}(test.t.a,test.t.a)->Projection" }, { "SQL": "select sum(to_base64(e)) from t group by e,d,c order by c", @@ -1233,12 +1233,12 @@ }, { "SQL": "select /*+ HASH_AGG() */ t1.a from t t1 where t1.a < any(select t2.b from t t2)", - "Best": "LeftHashJoin{IndexReader(Index(t.f)[[NULL,+inf]]->Sel([1]))->TableReader(Table(t)->Sel([1])->HashAgg)->HashAgg->Sel([ne(Column#27, 0) 1])}", + "Best": "LeftHashJoin{IndexReader(Index(t.f)[[NULL,+inf]])->TableReader(Table(t)->HashAgg)->HashAgg->Sel([ne(Column#27, 0)])}", "Warning": "" }, { "SQL": "select /*+ hash_agg() */ t1.a from t t1 where t1.a != any(select t2.b from t t2)", - "Best": "LeftHashJoin{IndexReader(Index(t.f)[[NULL,+inf]]->Sel([1]))->TableReader(Table(t)->Sel([1]))->HashAgg->Sel([ne(Column#28, 0) 1])}", + "Best": "LeftHashJoin{IndexReader(Index(t.f)[[NULL,+inf]])->TableReader(Table(t))->HashAgg->Sel([ne(Column#28, 0)])}", "Warning": "" }, { diff --git a/planner/core/testdata/point_get_plan_out.json b/planner/core/testdata/point_get_plan_out.json index b0053fde6044d..9f9ac9e27deb9 100644 --- a/planner/core/testdata/point_get_plan_out.json +++ b/planner/core/testdata/point_get_plan_out.json @@ -15,8 +15,7 @@ { "SQL": "select b, c from t where t.b = 2 and t.c = 2 and t.b+1=3", "Plan": [ - "Selection 0.80 root 1", - "└─Point_Get 1.00 root table:t, index:b(b, c) " + "Point_Get 1.00 root table:t, index:b(b, c) " ], "Res": [ "2 2" From 3dd2e546f841fa3e1f32812fce8e560cef66e897 Mon Sep 17 00:00:00 2001 From: Shirly Date: Tue, 11 May 2021 10:15:39 +0800 Subject: [PATCH 47/67] store/tikv: make tikv.ErrTokenLimit as a normal error instead of terror (#24484) --- store/driver/txn/error.go | 6 ++++++ store/tikv/error/errcode.go | 3 --- store/tikv/error/error.go | 10 +++++++++- store/tikv/region_request.go | 3 +-- store/tikv/region_request_test.go | 5 ++++- 5 files changed, 20 insertions(+), 7 deletions(-) diff --git a/store/driver/txn/error.go b/store/driver/txn/error.go index b5e69b4e522a9..c026b8899ae72 100644 --- a/store/driver/txn/error.go +++ b/store/driver/txn/error.go @@ -38,6 +38,8 @@ import ( // tikv error instance var ( + // ErrTokenLimit is the error that token is up to the limit. + ErrTokenLimit = dbterror.ClassTiKV.NewStd(errno.ErrTiKVStoreLimit) // ErrTiKVServerTimeout is the error when tikv server is timeout. ErrTiKVServerTimeout = dbterror.ClassTiKV.NewStd(errno.ErrTiKVServerTimeout) ErrTiFlashServerTimeout = dbterror.ClassTiKV.NewStd(errno.ErrTiFlashServerTimeout) @@ -253,6 +255,10 @@ func ToTiDBErr(err error) error { return ErrRegionUnavailable } + if e, ok := err.(*tikverr.ErrTokenLimit); ok { + return ErrTokenLimit.GenWithStackByArgs(e.StoreID) + } + return errors.Trace(originErr) } diff --git a/store/tikv/error/errcode.go b/store/tikv/error/errcode.go index bf0e3c7c4b91d..01e8db4d12473 100644 --- a/store/tikv/error/errcode.go +++ b/store/tikv/error/errcode.go @@ -20,7 +20,4 @@ const ( CodeLockWaitTimeout = 1205 CodeQueryInterrupted = 1317 CodeLockAcquireFailAndNoWaitSet = 3572 - - // TiKV/PD/TiFlash errors. - CodeTiKVStoreLimit = 9008 ) diff --git a/store/tikv/error/error.go b/store/tikv/error/error.go index 8ed21d8db192f..70ec995e88489 100644 --- a/store/tikv/error/error.go +++ b/store/tikv/error/error.go @@ -60,7 +60,6 @@ var ( ErrQueryInterrupted = dbterror.ClassTiKV.NewStd(CodeQueryInterrupted) ErrLockAcquireFailAndNoWaitSet = dbterror.ClassTiKV.NewStd(CodeLockAcquireFailAndNoWaitSet) ErrLockWaitTimeout = dbterror.ClassTiKV.NewStd(CodeLockWaitTimeout) - ErrTokenLimit = dbterror.ClassTiKV.NewStd(CodeTiKVStoreLimit) ErrUnknown = dbterror.ClassTiKV.NewStd(CodeUnknown) ) @@ -190,3 +189,12 @@ type ErrGCTooEarly struct { func (e *ErrGCTooEarly) Error() string { return fmt.Sprintf("GC life time is shorter than transaction duration, transaction starts at %v, GC safe point is %v", e.TxnStartTS, e.GCSafePoint) } + +// ErrTokenLimit is the error that token is up to the limit. +type ErrTokenLimit struct { + StoreID uint64 +} + +func (e *ErrTokenLimit) Error() string { + return fmt.Sprintf("Store token is up to the limit, store id = %d.", e.StoreID) +} diff --git a/store/tikv/region_request.go b/store/tikv/region_request.go index cad0ed0379e96..74ecdd7ce72b8 100644 --- a/store/tikv/region_request.go +++ b/store/tikv/region_request.go @@ -530,8 +530,7 @@ func (s *RegionRequestSender) getStoreToken(st *Store, limit int64) error { return nil } metrics.TiKVStoreLimitErrorCounter.WithLabelValues(st.addr, strconv.FormatUint(st.storeID, 10)).Inc() - return tikverr.ErrTokenLimit.GenWithStackByArgs(st.storeID) - + return &tikverr.ErrTokenLimit{StoreID: st.storeID} } func (s *RegionRequestSender) releaseStoreToken(st *Store) { diff --git a/store/tikv/region_request_test.go b/store/tikv/region_request_test.go index 81e9cc4498a07..9c5172e52f372 100644 --- a/store/tikv/region_request_test.go +++ b/store/tikv/region_request_test.go @@ -31,6 +31,7 @@ import ( "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/mpp" "github.com/pingcap/kvproto/pkg/tikvpb" + tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/config" @@ -177,7 +178,9 @@ func (s *testRegionRequestToThreeStoresSuite) TestStoreTokenLimit(c *C) { resp, err := s.regionRequestSender.SendReq(s.bo, req, region.Region, time.Second) c.Assert(err, NotNil) c.Assert(resp, IsNil) - c.Assert(err.Error(), Equals, "[tikv:9008]Store token is up to the limit, store id = 1") + e, ok := errors.Cause(err).(*tikverr.ErrTokenLimit) + c.Assert(ok, IsTrue) + c.Assert(e.StoreID, Equals, uint64(1)) kv.StoreLimit.Store(oldStoreLimit) } From 9e13287d9db85860c1adeed51b419e5cba2362fa Mon Sep 17 00:00:00 2001 From: xhe Date: Tue, 11 May 2021 10:31:39 +0800 Subject: [PATCH 48/67] *: remove SchemaVersion in TransactionContext (#24236) --- executor/ddl.go | 1 - executor/simple.go | 6 ++-- session/session.go | 47 ++++++++++++++--------------- sessionctx/binloginfo/binloginfo.go | 2 +- sessionctx/variable/session.go | 34 ++++++++++++--------- 5 files changed, 46 insertions(+), 44 deletions(-) diff --git a/executor/ddl.go b/executor/ddl.go index 64a597e2eb024..81f7221d1e60e 100644 --- a/executor/ddl.go +++ b/executor/ddl.go @@ -140,7 +140,6 @@ func (e *DDLExec) Next(ctx context.Context, req *chunk.Chunk) (err error) { is := dom.InfoSchema() txnCtx := e.ctx.GetSessionVars().TxnCtx txnCtx.InfoSchema = is - txnCtx.SchemaVersion = is.SchemaMetaVersion() // DDL will force commit old transaction, after DDL, in transaction status should be false. e.ctx.GetSessionVars().SetInTxn(false) return nil diff --git a/executor/simple.go b/executor/simple.go index 65df5ca43117f..24cb857aec3d5 100644 --- a/executor/simple.go +++ b/executor/simple.go @@ -686,13 +686,13 @@ func (e *SimpleExec) executeStartTransactionReadOnlyWithTimestampBound(ctx conte } failpoint.Inject("mockStalenessTxnSchemaVer", func(val failpoint.Value) { if val.(bool) { - staleVer = e.ctx.GetSessionVars().TxnCtx.SchemaVersion - 1 + staleVer = e.ctx.GetSessionVars().GetInfoSchema().SchemaMetaVersion() - 1 } else { - staleVer = e.ctx.GetSessionVars().TxnCtx.SchemaVersion + staleVer = e.ctx.GetSessionVars().GetInfoSchema().SchemaMetaVersion() } }) // TODO: currently we directly check the schema version. In future, we can cache the stale infoschema instead. - if e.ctx.GetSessionVars().TxnCtx.SchemaVersion > staleVer { + if e.ctx.GetSessionVars().GetInfoSchema().SchemaMetaVersion() > staleVer { return errors.New("schema version changed after the staleness startTS") } diff --git a/session/session.go b/session/session.go index 94582fbcb6886..7c499d83afda1 100644 --- a/session/session.go +++ b/session/session.go @@ -491,7 +491,7 @@ func (s *session) doCommit(ctx context.Context) error { physicalTableIDs = append(physicalTableIDs, id) } // Set this option for 2 phase commit to validate schema lease. - s.txn.SetOption(tikvstore.SchemaChecker, domain.NewSchemaChecker(domain.GetDomain(s), s.sessionVars.TxnCtx.SchemaVersion, physicalTableIDs)) + s.txn.SetOption(tikvstore.SchemaChecker, domain.NewSchemaChecker(domain.GetDomain(s), s.sessionVars.GetInfoSchema().SchemaMetaVersion(), physicalTableIDs)) s.txn.SetOption(tikvstore.InfoSchema, s.sessionVars.TxnCtx.InfoSchema) s.txn.SetOption(tikvstore.CommitHook, func(info string, _ error) { s.sessionVars.LastTxnInfo = info }) if s.GetSessionVars().EnableAmendPessimisticTxn { @@ -1485,7 +1485,7 @@ func (s *session) ExecuteStmt(ctx context.Context, stmtNode ast.StmtNode) (sqlex if err != nil { if !kv.ErrKeyExists.Equal(err) { logutil.Logger(ctx).Warn("run statement failed", - zap.Int64("schemaVersion", s.sessionVars.TxnCtx.SchemaVersion), + zap.Int64("schemaVersion", s.sessionVars.GetInfoSchema().SchemaMetaVersion()), zap.Error(err), zap.String("session", s.String())) } @@ -1926,7 +1926,7 @@ func (s *session) NewTxn(ctx context.Context) error { } vars := s.GetSessionVars() logutil.Logger(ctx).Info("NewTxn() inside a transaction auto commit", - zap.Int64("schemaVersion", vars.TxnCtx.SchemaVersion), + zap.Int64("schemaVersion", vars.GetInfoSchema().SchemaMetaVersion()), zap.Uint64("txnStartTS", txnID), zap.String("txnScope", txnScope)) } @@ -1942,13 +1942,12 @@ func (s *session) NewTxn(ctx context.Context) error { s.txn.changeInvalidToValid(txn) is := domain.GetDomain(s).InfoSchema() s.sessionVars.TxnCtx = &variable.TransactionContext{ - InfoSchema: is, - SchemaVersion: is.SchemaMetaVersion(), - CreateTime: time.Now(), - StartTS: txn.StartTS(), - ShardStep: int(s.sessionVars.ShardAllocateStep), - IsStaleness: false, - TxnScope: s.sessionVars.CheckAndGetTxnScope(), + InfoSchema: is, + CreateTime: time.Now(), + StartTS: txn.StartTS(), + ShardStep: int(s.sessionVars.ShardAllocateStep), + IsStaleness: false, + TxnScope: s.sessionVars.CheckAndGetTxnScope(), } return nil } @@ -2678,11 +2677,10 @@ func (s *session) PrepareTxnCtx(ctx context.Context) { is := domain.GetDomain(s).InfoSchema() s.sessionVars.TxnCtx = &variable.TransactionContext{ - InfoSchema: is, - SchemaVersion: is.SchemaMetaVersion(), - CreateTime: time.Now(), - ShardStep: int(s.sessionVars.ShardAllocateStep), - TxnScope: s.GetSessionVars().CheckAndGetTxnScope(), + InfoSchema: is, + CreateTime: time.Now(), + ShardStep: int(s.sessionVars.ShardAllocateStep), + TxnScope: s.GetSessionVars().CheckAndGetTxnScope(), } if !s.sessionVars.IsAutocommit() || s.sessionVars.RetryInfo.Retrying { if s.sessionVars.TxnMode == ast.Pessimistic { @@ -2754,7 +2752,7 @@ func (s *session) NewTxnWithStalenessOption(ctx context.Context, option sessionc } vars := s.GetSessionVars() logutil.Logger(ctx).Info("InitTxnWithExactStaleness() inside a transaction auto commit", - zap.Int64("schemaVersion", vars.TxnCtx.SchemaVersion), + zap.Int64("schemaVersion", vars.GetInfoSchema().SchemaMetaVersion()), zap.Uint64("txnStartTS", txnID), zap.String("txnScope", txnScope)) } @@ -2792,13 +2790,12 @@ func (s *session) NewTxnWithStalenessOption(ctx context.Context, option sessionc s.txn.changeInvalidToValid(txn) is := domain.GetDomain(s).InfoSchema() s.sessionVars.TxnCtx = &variable.TransactionContext{ - InfoSchema: is, - SchemaVersion: is.SchemaMetaVersion(), - CreateTime: time.Now(), - StartTS: txn.StartTS(), - ShardStep: int(s.sessionVars.ShardAllocateStep), - IsStaleness: true, - TxnScope: txnScope, + InfoSchema: is, + CreateTime: time.Now(), + StartTS: txn.StartTS(), + ShardStep: int(s.sessionVars.ShardAllocateStep), + IsStaleness: true, + TxnScope: txnScope, } return nil } @@ -2825,7 +2822,7 @@ func logStmt(execStmt *executor.ExecStmt, vars *variable.SessionVars) { *ast.RevokeStmt, *ast.AlterTableStmt, *ast.CreateDatabaseStmt, *ast.CreateIndexStmt, *ast.CreateTableStmt, *ast.DropDatabaseStmt, *ast.DropIndexStmt, *ast.DropTableStmt, *ast.RenameTableStmt, *ast.TruncateTableStmt: user := vars.User - schemaVersion := vars.TxnCtx.SchemaVersion + schemaVersion := vars.GetInfoSchema().SchemaMetaVersion() if ss, ok := execStmt.StmtNode.(ast.SensitiveStmtNode); ok { logutil.BgLogger().Info("CRUCIAL OPERATION", zap.Uint64("conn", vars.ConnectionID), @@ -2854,7 +2851,7 @@ func logQuery(query string, vars *variable.SessionVars) { logutil.BgLogger().Info("GENERAL_LOG", zap.Uint64("conn", vars.ConnectionID), zap.Stringer("user", vars.User), - zap.Int64("schemaVersion", vars.TxnCtx.SchemaVersion), + zap.Int64("schemaVersion", vars.GetInfoSchema().SchemaMetaVersion()), zap.Uint64("txnStartTS", vars.TxnCtx.StartTS), zap.Uint64("forUpdateTS", vars.TxnCtx.GetForUpdateTS()), zap.Bool("isReadConsistency", vars.IsIsolation(ast.ReadCommitted)), diff --git a/sessionctx/binloginfo/binloginfo.go b/sessionctx/binloginfo/binloginfo.go index 044e6cdc11df9..58313505e1c8e 100644 --- a/sessionctx/binloginfo/binloginfo.go +++ b/sessionctx/binloginfo/binloginfo.go @@ -101,7 +101,7 @@ func GetPrewriteValue(ctx sessionctx.Context, createIfNotExists bool) *binlog.Pr vars := ctx.GetSessionVars() v, ok := vars.TxnCtx.Binlog.(*binlog.PrewriteValue) if !ok && createIfNotExists { - schemaVer := ctx.GetSessionVars().TxnCtx.SchemaVersion + schemaVer := ctx.GetSessionVars().GetInfoSchema().SchemaMetaVersion() v = &binlog.PrewriteValue{SchemaVersion: schemaVer} vars.TxnCtx.Binlog = v } diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 89b31da2e8bbd..7db9de383ba55 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -129,13 +129,12 @@ func (r *retryInfoAutoIDs) getCurrent() (int64, bool) { // TransactionContext is used to store variables that has transaction scope. type TransactionContext struct { - forUpdateTS uint64 - stmtFuture oracle.Future - Binlog interface{} - InfoSchema interface{} - History interface{} - SchemaVersion int64 - StartTS uint64 + forUpdateTS uint64 + stmtFuture oracle.Future + Binlog interface{} + InfoSchema interface{} + History interface{} + StartTS uint64 // ShardStep indicates the max size of continuous rowid shard in one transaction. ShardStep int @@ -870,19 +869,26 @@ func (s *SessionVars) BuildParserConfig() parser.ParserConfig { } } +// FIXME: remove this interface +// infoschemaMetaVersion is a workaround. Due to circular dependency, +// can not return the complete interface. But SchemaMetaVersion is widely used for logging. +// So we give a convenience for that +type infoschemaMetaVersion interface { + SchemaMetaVersion() int64 +} + // GetInfoSchema returns snapshotInfoSchema if snapshot schema is set. // Otherwise, transaction infoschema is returned. // Nil if there is no available infoschema. -func (s *SessionVars) GetInfoSchema() interface{} { - type IS interface { - SchemaMetaVersion() int64 - } - if snap, ok := s.SnapshotInfoschema.(IS); ok { +func (s *SessionVars) GetInfoSchema() infoschemaMetaVersion { + if snap, ok := s.SnapshotInfoschema.(infoschemaMetaVersion); ok { logutil.BgLogger().Info("use snapshot schema", zap.Uint64("conn", s.ConnectionID), zap.Int64("schemaVersion", snap.SchemaMetaVersion())) return snap } - if s.TxnCtx != nil && s.TxnCtx.InfoSchema != nil { - return s.TxnCtx.InfoSchema + if s.TxnCtx != nil { + if is, ok := s.TxnCtx.InfoSchema.(infoschemaMetaVersion); ok { + return is + } } return nil } From f13f696400be61ae43dd28aeaf61920817732b17 Mon Sep 17 00:00:00 2001 From: ZhuoZhi <517770911@qq.com> Date: Tue, 11 May 2021 10:57:38 +0800 Subject: [PATCH 49/67] executor: add correctness tests about PointGet and BatchGet (#24467) --- executor/partition_table_test.go | 57 ++++++++++++++++++++++++++++++++ 1 file changed, 57 insertions(+) diff --git a/executor/partition_table_test.go b/executor/partition_table_test.go index 55b8fa51786d7..555e152265349 100644 --- a/executor/partition_table_test.go +++ b/executor/partition_table_test.go @@ -227,6 +227,63 @@ func (s *partitionTableSuite) TestPartitionInfoDisable(c *C) { tk.MustQuery("select * from t_info_null where (date = '2020-10-02' or date = '2020-10-06') and app = 'xxx' and media = '19003006'").Check(testkit.Rows()) } +func (s *partitionTableSuite) TestBatchGetandPointGetwithHashPartition(c *C) { + if israce.RaceEnabled { + c.Skip("exhaustive types test, skip race test") + } + + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create database test_batchget_pointget") + tk.MustExec("use test_batchget_pointget") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + + // hash partition table + tk.MustExec("create table thash(a int, unique key(a)) partition by hash(a) partitions 4;") + + // regular partition table + tk.MustExec("create table tregular(a int, unique key(a));") + + vals := make([]string, 0, 100) + // insert data into range partition table and hash partition table + for i := 0; i < 100; i++ { + vals = append(vals, fmt.Sprintf("(%v)", i+1)) + } + tk.MustExec("insert into thash values " + strings.Join(vals, ",")) + tk.MustExec("insert into tregular values " + strings.Join(vals, ",")) + + // test PointGet + for i := 0; i < 100; i++ { + // explain select a from t where a = {x}; // x >= 1 and x <= 100 Check if PointGet is used + // select a from t where a={x}; // the result is {x} + x := rand.Intn(100) + 1 + queryHash := fmt.Sprintf("select a from thash where a=%v", x) + queryRegular := fmt.Sprintf("select a from thash where a=%v", x) + c.Assert(tk.HasPlan(queryHash, "Point_Get"), IsTrue) // check if PointGet is used + tk.MustQuery(queryHash).Check(tk.MustQuery(queryRegular).Rows()) + } + + // test empty PointGet + queryHash := fmt.Sprintf("select a from thash where a=200") + c.Assert(tk.HasPlan(queryHash, "Point_Get"), IsTrue) // check if PointGet is used + tk.MustQuery(queryHash).Check(testkit.Rows()) + + // test BatchGet + for i := 0; i < 100; i++ { + // explain select a from t where a in ({x1}, {x2}, ... {x10}); // BatchGet is used + // select a from t where where a in ({x1}, {x2}, ... {x10}); + points := make([]string, 0, 10) + for i := 0; i < 10; i++ { + x := rand.Intn(100) + 1 + points = append(points, fmt.Sprintf("%v", x)) + } + + queryHash := fmt.Sprintf("select a from thash where a in (%v)", strings.Join(points, ",")) + queryRegular := fmt.Sprintf("select a from tregular where a in (%v)", strings.Join(points, ",")) + c.Assert(tk.HasPlan(queryHash, "Point_Get"), IsTrue) // check if PointGet is used + tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + } +} + func (s *partitionTableSuite) TestView(c *C) { if israce.RaceEnabled { c.Skip("exhaustive types test, skip race test") From 7c1997517c76fda1b27bf64e4e653a33ddfe88e0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=BE=99=E6=96=B9=E6=B7=9E?= Date: Tue, 11 May 2021 11:17:38 +0800 Subject: [PATCH 50/67] txn: Use TransactionOption in store/tikv module (#23255) --- kv/kv.go | 7 ++ session/session.go | 12 +-- session/txn.go | 4 +- store/driver/tikv_driver.go | 20 +--- store/mockstore/mockstorage/storage.go | 16 +--- store/tikv/extract_start_ts_test.go | 122 +++++++++++++++++++++++++ store/tikv/kv.go | 70 +------------- store/tikv/tests/2pc_test.go | 5 +- store/tikv/txn.go | 106 ++++++++++++++------- 9 files changed, 220 insertions(+), 142 deletions(-) create mode 100644 store/tikv/extract_start_ts_test.go diff --git a/kv/kv.go b/kv/kv.go index 1b1e1a5f46a4a..a6a23a88df01d 100644 --- a/kv/kv.go +++ b/kv/kv.go @@ -341,6 +341,8 @@ type Driver interface { } // TransactionOption indicates the option when beginning a transaction +// `TxnScope` must be set for each object +// Every other fields are optional, but currently at most one of them can be set type TransactionOption struct { TxnScope string StartTS *uint64 @@ -349,6 +351,11 @@ type TransactionOption struct { MaxPrevSec *uint64 } +// DefaultTransactionOption creates a default TransactionOption, ie. Work in GlobalTxnScope and get start ts when got used +func DefaultTransactionOption() TransactionOption { + return TransactionOption{TxnScope: oracle.GlobalTxnScope} +} + // SetMaxPrevSec set maxPrevSec func (to TransactionOption) SetMaxPrevSec(maxPrevSec uint64) TransactionOption { to.MaxPrevSec = &maxPrevSec diff --git a/session/session.go b/session/session.go index 7c499d83afda1..2f842f92e183a 100644 --- a/session/session.go +++ b/session/session.go @@ -1931,7 +1931,7 @@ func (s *session) NewTxn(ctx context.Context) error { zap.String("txnScope", txnScope)) } - txn, err := s.store.BeginWithOption(kv.TransactionOption{}.SetTxnScope(s.sessionVars.CheckAndGetTxnScope())) + txn, err := s.store.BeginWithOption(kv.DefaultTransactionOption().SetTxnScope(s.sessionVars.CheckAndGetTxnScope())) if err != nil { return err } @@ -2728,7 +2728,7 @@ func (s *session) InitTxnWithStartTS(startTS uint64) error { } // no need to get txn from txnFutureCh since txn should init with startTs - txn, err := s.store.BeginWithOption(kv.TransactionOption{}.SetTxnScope(s.GetSessionVars().CheckAndGetTxnScope()).SetStartTs(startTS)) + txn, err := s.store.BeginWithOption(kv.DefaultTransactionOption().SetTxnScope(s.GetSessionVars().CheckAndGetTxnScope()).SetStartTs(startTS)) if err != nil { return err } @@ -2761,22 +2761,22 @@ func (s *session) NewTxnWithStalenessOption(ctx context.Context, option sessionc txnScope := s.GetSessionVars().CheckAndGetTxnScope() switch option.Mode { case ast.TimestampBoundReadTimestamp: - txn, err = s.store.BeginWithOption(kv.TransactionOption{}.SetTxnScope(txnScope).SetStartTs(option.StartTS)) + txn, err = s.store.BeginWithOption(kv.DefaultTransactionOption().SetTxnScope(txnScope).SetStartTs(option.StartTS)) if err != nil { return err } case ast.TimestampBoundExactStaleness: - txn, err = s.store.BeginWithOption(kv.TransactionOption{}.SetTxnScope(txnScope).SetPrevSec(option.PrevSec)) + txn, err = s.store.BeginWithOption(kv.DefaultTransactionOption().SetTxnScope(txnScope).SetPrevSec(option.PrevSec)) if err != nil { return err } case ast.TimestampBoundMaxStaleness: - txn, err = s.store.BeginWithOption(kv.TransactionOption{}.SetTxnScope(txnScope).SetMaxPrevSec(option.PrevSec)) + txn, err = s.store.BeginWithOption(kv.DefaultTransactionOption().SetTxnScope(txnScope).SetMaxPrevSec(option.PrevSec)) if err != nil { return err } case ast.TimestampBoundMinReadTimestamp: - txn, err = s.store.BeginWithOption(kv.TransactionOption{}.SetTxnScope(txnScope).SetMinStartTS(option.StartTS)) + txn, err = s.store.BeginWithOption(kv.DefaultTransactionOption().SetTxnScope(txnScope).SetMinStartTS(option.StartTS)) if err != nil { return err } diff --git a/session/txn.go b/session/txn.go index eed4698f60a65..aebed7ed920b2 100644 --- a/session/txn.go +++ b/session/txn.go @@ -353,14 +353,14 @@ type txnFuture struct { func (tf *txnFuture) wait() (kv.Transaction, error) { startTS, err := tf.future.Wait() if err == nil { - return tf.store.BeginWithOption(kv.TransactionOption{}.SetTxnScope(tf.txnScope).SetStartTs(startTS)) + return tf.store.BeginWithOption(kv.DefaultTransactionOption().SetTxnScope(tf.txnScope).SetStartTs(startTS)) } else if config.GetGlobalConfig().Store == "unistore" { return nil, err } logutil.BgLogger().Warn("wait tso failed", zap.Error(err)) // It would retry get timestamp. - return tf.store.BeginWithOption(kv.TransactionOption{}.SetTxnScope(tf.txnScope)) + return tf.store.BeginWithOption(kv.DefaultTransactionOption().SetTxnScope(tf.txnScope)) } func (s *session) getTxnFuture(ctx context.Context) *txnFuture { diff --git a/store/driver/tikv_driver.go b/store/driver/tikv_driver.go index 17308d33c6be3..cb14736844e68 100644 --- a/store/driver/tikv_driver.go +++ b/store/driver/tikv_driver.go @@ -30,7 +30,6 @@ import ( "github.com/pingcap/tidb/store/gcworker" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/config" - "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/store/tikv/util" "github.com/pingcap/tidb/util/logutil" pd "github.com/tikv/pd/client" @@ -306,27 +305,10 @@ func (s *tikvStore) Begin() (kv.Transaction, error) { // BeginWithOption begins a transaction with given option func (s *tikvStore) BeginWithOption(option kv.TransactionOption) (kv.Transaction, error) { - txnScope := option.TxnScope - if txnScope == "" { - txnScope = oracle.GlobalTxnScope - } - var txn *tikv.KVTxn - var err error - if option.StartTS != nil { - txn, err = s.BeginWithStartTS(txnScope, *option.StartTS) - } else if option.PrevSec != nil { - txn, err = s.BeginWithExactStaleness(txnScope, *option.PrevSec) - } else if option.MaxPrevSec != nil { - txn, err = s.BeginWithMaxPrevSec(txnScope, *option.MaxPrevSec) - } else if option.MinStartTS != nil { - txn, err = s.BeginWithMinStartTS(txnScope, *option.MinStartTS) - } else { - txn, err = s.BeginWithTxnScope(txnScope) - } + txn, err := s.KVStore.BeginWithOption(option) if err != nil { return nil, txn_driver.ToTiDBErr(err) } - return txn_driver.NewTiKVTxn(txn), err } diff --git a/store/mockstore/mockstorage/storage.go b/store/mockstore/mockstorage/storage.go index 05ece29c57a6a..36ded5e434817 100644 --- a/store/mockstore/mockstorage/storage.go +++ b/store/mockstore/mockstorage/storage.go @@ -22,7 +22,6 @@ import ( driver "github.com/pingcap/tidb/store/driver/txn" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/config" - "github.com/pingcap/tidb/store/tikv/oracle" ) // Wraps tikv.KVStore and make it compatible with kv.Storage. @@ -85,20 +84,7 @@ func (s *mockStorage) ShowStatus(ctx context.Context, key string) (interface{}, // BeginWithOption begins a transaction with given option func (s *mockStorage) BeginWithOption(option kv.TransactionOption) (kv.Transaction, error) { - txnScope := option.TxnScope - if txnScope == "" { - txnScope = oracle.GlobalTxnScope - } - if option.StartTS != nil { - return newTiKVTxn(s.BeginWithStartTS(txnScope, *option.StartTS)) - } else if option.PrevSec != nil { - return newTiKVTxn(s.BeginWithExactStaleness(txnScope, *option.PrevSec)) - } else if option.MaxPrevSec != nil { - return newTiKVTxn(s.BeginWithMaxPrevSec(txnScope, *option.MaxPrevSec)) - } else if option.MinStartTS != nil { - return newTiKVTxn(s.BeginWithMinStartTS(txnScope, *option.MinStartTS)) - } - return newTiKVTxn(s.BeginWithTxnScope(txnScope)) + return newTiKVTxn(s.KVStore.BeginWithOption(option)) } // GetSnapshot gets a snapshot that is able to read any data which data is <= ver. diff --git a/store/tikv/extract_start_ts_test.go b/store/tikv/extract_start_ts_test.go new file mode 100644 index 0000000000000..1422e387bfda5 --- /dev/null +++ b/store/tikv/extract_start_ts_test.go @@ -0,0 +1,122 @@ +// 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, +// See the License for the specific language governing permissions and +// limitations under the License. + +package tikv + +import ( + "context" + + . "github.com/pingcap/check" + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/store/mockstore/unistore" + "github.com/pingcap/tidb/store/tikv/oracle" + "github.com/pingcap/tidb/store/tikv/tikvrpc" +) + +type extractStartTsSuite struct { + store *KVStore +} + +var _ = Suite(&extractStartTsSuite{}) + +func (s *extractStartTsSuite) SetUpTest(c *C) { + client, pdClient, cluster, err := unistore.New("") + c.Assert(err, IsNil) + unistore.BootstrapWithSingleStore(cluster) + store, err := NewTestTiKVStore(client, pdClient, nil, nil, 0) + c.Assert(err, IsNil) + store.regionCache.storeMu.stores[2] = &Store{ + storeID: 2, + storeType: tikvrpc.TiKV, + state: uint64(resolved), + labels: []*metapb.StoreLabel{ + { + Key: DCLabelKey, + Value: oracle.LocalTxnScope, + }, + }, + } + store.regionCache.storeMu.stores[3] = &Store{ + storeID: 3, + storeType: tikvrpc.TiKV, + state: uint64(resolved), + labels: []*metapb.StoreLabel{{ + Key: DCLabelKey, + Value: "Some Random Label", + }}, + } + store.resolveTSMu.resolveTS[2] = 102 + store.resolveTSMu.resolveTS[3] = 101 + s.store = store +} + +func (s *extractStartTsSuite) TestExtractStartTs(c *C) { + i := uint64(100) + cases := []kv.TransactionOption{ + // StartTS setted + {TxnScope: oracle.GlobalTxnScope, StartTS: &i, PrevSec: nil, MinStartTS: nil, MaxPrevSec: nil}, + // PrevSec setted + {TxnScope: oracle.GlobalTxnScope, StartTS: nil, PrevSec: &i, MinStartTS: nil, MaxPrevSec: nil}, + // MinStartTS setted, global + {TxnScope: oracle.GlobalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: &i, MaxPrevSec: nil}, + // MinStartTS setted, local + {TxnScope: oracle.LocalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: &i, MaxPrevSec: nil}, + // MaxPrevSec setted + // however we need to add more cases to check the behavior when it fall backs to MinStartTS setted + // see `TestMaxPrevSecFallback` + {TxnScope: oracle.GlobalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: nil, MaxPrevSec: &i}, + // nothing setted + {TxnScope: oracle.GlobalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: nil, MaxPrevSec: nil}, + } + bo := NewBackofferWithVars(context.Background(), tsoMaxBackoff, nil) + stalenessTimestamp, _ := s.store.getStalenessTimestamp(bo, oracle.GlobalTxnScope, 100) + expectedTs := []uint64{ + 100, + stalenessTimestamp, + + 101, + 102, + + stalenessTimestamp, + // it's too hard to figure out the value `getTimestampWithRetry` returns + // so we just check whether it is greater than stalenessTimestamp + 0, + } + for i, cs := range cases { + expected := expectedTs[i] + result, _ := extractStartTs(s.store, cs) + if expected == 0 { + c.Assert(result, Greater, stalenessTimestamp) + } else { + c.Assert(result, Equals, expected) + } + } +} + +func (s *extractStartTsSuite) TestMaxPrevSecFallback(c *C) { + s.store.resolveTSMu.resolveTS[2] = 0x8000000000000002 + s.store.resolveTSMu.resolveTS[3] = 0x8000000000000001 + + i := uint64(100) + cases := []kv.TransactionOption{ + {TxnScope: oracle.GlobalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: nil, MaxPrevSec: &i}, + {TxnScope: oracle.LocalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: nil, MaxPrevSec: &i}, + } + expectedTs := []uint64{0x8000000000000001, 0x8000000000000002} + for i, cs := range cases { + expected := expectedTs[i] + result, _ := extractStartTs(s.store, cs) + c.Assert(result, Equals, expected) + } +} diff --git a/store/tikv/kv.go b/store/tikv/kv.go index 0f4824a785ecb..5ddca52726a04 100644 --- a/store/tikv/kv.go +++ b/store/tikv/kv.go @@ -26,7 +26,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/kvrpcpb" - "github.com/pingcap/kvproto/pkg/metapb" + tidbkv "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/store/tikv/config" tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/store/tikv/kv" @@ -184,72 +184,12 @@ func (s *KVStore) runSafePointChecker() { // Begin a global transaction. func (s *KVStore) Begin() (*KVTxn, error) { - return s.BeginWithTxnScope(oracle.GlobalTxnScope) + return s.BeginWithOption(tidbkv.DefaultTransactionOption()) } -// BeginWithTxnScope begins a transaction with the given txnScope (local or global) -func (s *KVStore) BeginWithTxnScope(txnScope string) (*KVTxn, error) { - txn, err := newTiKVTxn(s, txnScope) - if err != nil { - return nil, errors.Trace(err) - } - return txn, nil -} - -// BeginWithStartTS begins a transaction with startTS. -func (s *KVStore) BeginWithStartTS(txnScope string, startTS uint64) (*KVTxn, error) { - txn, err := newTiKVTxnWithStartTS(s, txnScope, startTS, s.nextReplicaReadSeed()) - if err != nil { - return nil, errors.Trace(err) - } - return txn, nil -} - -// BeginWithExactStaleness begins transaction with given staleness -func (s *KVStore) BeginWithExactStaleness(txnScope string, prevSec uint64) (*KVTxn, error) { - txn, err := newTiKVTxnWithExactStaleness(s, txnScope, prevSec) - if err != nil { - return nil, errors.Trace(err) - } - return txn, nil -} - -// BeginWithMinStartTS begins transaction with the least startTS -func (s *KVStore) BeginWithMinStartTS(txnScope string, minStartTS uint64) (*KVTxn, error) { - stores := make([]*Store, 0) - allStores := s.regionCache.getStoresByType(tikvrpc.TiKV) - if txnScope != oracle.GlobalTxnScope { - for _, store := range allStores { - if store.IsLabelsMatch([]*metapb.StoreLabel{ - { - Key: DCLabelKey, - Value: txnScope, - }, - }) { - stores = append(stores, store) - } - } - } else { - stores = allStores - } - resolveTS := s.getMinResolveTSByStores(stores) - startTS := minStartTS - // If the resolveTS is larger than the minStartTS, we will use resolveTS as StartTS, otherwise we will use - // minStartTS directly. - if oracle.CompareTS(startTS, resolveTS) < 0 { - startTS = resolveTS - } - return s.BeginWithStartTS(txnScope, startTS) -} - -// BeginWithMaxPrevSec begins transaction with given max previous seconds for startTS -func (s *KVStore) BeginWithMaxPrevSec(txnScope string, maxPrevSec uint64) (*KVTxn, error) { - bo := NewBackofferWithVars(context.Background(), tsoMaxBackoff, nil) - minStartTS, err := s.getStalenessTimestamp(bo, txnScope, maxPrevSec) - if err != nil { - return nil, errors.Trace(err) - } - return s.BeginWithMinStartTS(txnScope, minStartTS) +// BeginWithOption begins a transaction with the given TransactionOption +func (s *KVStore) BeginWithOption(options tidbkv.TransactionOption) (*KVTxn, error) { + return newTiKVTxnWithOptions(s, options) } // GetSnapshot gets a snapshot that is able to read any data which data is <= ver. diff --git a/store/tikv/tests/2pc_test.go b/store/tikv/tests/2pc_test.go index 6d7d7e89d1a8e..8043ccdab89f0 100644 --- a/store/tikv/tests/2pc_test.go +++ b/store/tikv/tests/2pc_test.go @@ -29,6 +29,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/kvrpcpb" pb "github.com/pingcap/kvproto/pkg/kvrpcpb" + tidbkv "github.com/pingcap/tidb/kv" drivertxn "github.com/pingcap/tidb/store/driver/txn" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/config" @@ -602,12 +603,12 @@ func (s *testCommitterSuite) TestRejectCommitTS(c *C) { // Use max.Uint64 to read the data and success. // That means the final commitTS > startTS+2, it's not the one we provide. // So we cover the rety commitTS logic. - txn1, err := s.store.BeginWithStartTS(oracle.GlobalTxnScope, committer.GetStartTS()+2) + txn1, err := s.store.BeginWithOption(tidbkv.DefaultTransactionOption().SetStartTs(committer.GetStartTS() + 2)) c.Assert(err, IsNil) _, err = txn1.Get(bo.GetCtx(), []byte("x")) c.Assert(tikverr.IsErrNotFound(err), IsTrue) - txn2, err := s.store.BeginWithStartTS(oracle.GlobalTxnScope, math.MaxUint64) + txn2, err := s.store.BeginWithOption(tidbkv.DefaultTransactionOption().SetStartTs(math.MaxUint64)) c.Assert(err, IsNil) val, err := txn2.Get(bo.GetCtx(), []byte("x")) c.Assert(err, IsNil) diff --git a/store/tikv/txn.go b/store/tikv/txn.go index 4e462653c415c..0ae2df13c12e9 100644 --- a/store/tikv/txn.go +++ b/store/tikv/txn.go @@ -30,10 +30,14 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/kvrpcpb" + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/pingcap/tidb/kv" tikverr "github.com/pingcap/tidb/store/tikv/error" - "github.com/pingcap/tidb/store/tikv/kv" + tikv "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/logutil" "github.com/pingcap/tidb/store/tikv/metrics" + "github.com/pingcap/tidb/store/tikv/oracle" + "github.com/pingcap/tidb/store/tikv/tikvrpc" "github.com/pingcap/tidb/store/tikv/unionstore" "github.com/pingcap/tidb/store/tikv/util" "go.uber.org/zap" @@ -60,7 +64,7 @@ type KVTxn struct { commitTS uint64 mu sync.Mutex // For thread-safe LockKeys function. setCnt int64 - vars *kv.Variables + vars *tikv.Variables committer *twoPhaseCommitter lockedCnt int @@ -83,44 +87,80 @@ type KVTxn struct { kvFilter KVFilter } -func newTiKVTxn(store *KVStore, txnScope string) (*KVTxn, error) { - bo := NewBackofferWithVars(context.Background(), tsoMaxBackoff, nil) - startTS, err := store.getTimestampWithRetry(bo, txnScope) - if err != nil { - return nil, errors.Trace(err) +func extractStartTs(store *KVStore, options kv.TransactionOption) (uint64, error) { + var startTs uint64 + var err error + if options.StartTS != nil { + startTs = *options.StartTS + } else if options.PrevSec != nil { + bo := NewBackofferWithVars(context.Background(), tsoMaxBackoff, nil) + startTs, err = store.getStalenessTimestamp(bo, options.TxnScope, *options.PrevSec) + } else if options.MinStartTS != nil { + stores := make([]*Store, 0) + allStores := store.regionCache.getStoresByType(tikvrpc.TiKV) + if options.TxnScope != oracle.GlobalTxnScope { + for _, store := range allStores { + if store.IsLabelsMatch([]*metapb.StoreLabel{ + { + Key: DCLabelKey, + Value: options.TxnScope, + }, + }) { + stores = append(stores, store) + } + } + } else { + stores = allStores + } + resolveTS := store.getMinResolveTSByStores(stores) + startTs = *options.MinStartTS + // If the resolveTS is larger than the minStartTS, we will use resolveTS as StartTS, otherwise we will use + // minStartTS directly. + if oracle.CompareTS(startTs, resolveTS) < 0 { + startTs = resolveTS + } + } else if options.MaxPrevSec != nil { + bo := NewBackofferWithVars(context.Background(), tsoMaxBackoff, nil) + minStartTS, err := store.getStalenessTimestamp(bo, options.TxnScope, *options.MaxPrevSec) + if err != nil { + return 0, errors.Trace(err) + } + options.MinStartTS = &minStartTS + return extractStartTs(store, options) + } else { + bo := NewBackofferWithVars(context.Background(), tsoMaxBackoff, nil) + startTs, err = store.getTimestampWithRetry(bo, options.TxnScope) } - return newTiKVTxnWithStartTS(store, txnScope, startTS, store.nextReplicaReadSeed()) + return startTs, err } -// newTiKVTxnWithStartTS creates a txn with startTS. -func newTiKVTxnWithStartTS(store *KVStore, txnScope string, startTS uint64, replicaReadSeed uint32) (*KVTxn, error) { - snapshot := newTiKVSnapshot(store, startTS, replicaReadSeed) - return &KVTxn{ +func newTiKVTxnWithOptions(store *KVStore, options kv.TransactionOption) (*KVTxn, error) { + if options.TxnScope == "" { + options.TxnScope = oracle.GlobalTxnScope + } + startTs, err := extractStartTs(store, options) + if err != nil { + return nil, errors.Trace(err) + } + snapshot := newTiKVSnapshot(store, startTs, store.nextReplicaReadSeed()) + newTiKVTxn := &KVTxn{ snapshot: snapshot, us: unionstore.NewUnionStore(snapshot), store: store, - startTS: startTS, + startTS: startTs, startTime: time.Now(), valid: true, - vars: kv.DefaultVars, - scope: txnScope, - }, nil -} - -func newTiKVTxnWithExactStaleness(store *KVStore, txnScope string, prevSec uint64) (*KVTxn, error) { - bo := NewBackofferWithVars(context.Background(), tsoMaxBackoff, nil) - startTS, err := store.getStalenessTimestamp(bo, txnScope, prevSec) - if err != nil { - return nil, errors.Trace(err) + vars: tikv.DefaultVars, + scope: options.TxnScope, } - return newTiKVTxnWithStartTS(store, txnScope, startTS, store.nextReplicaReadSeed()) + return newTiKVTxn, nil } // SetSuccess is used to probe if kv variables are set or not. It is ONLY used in test cases. var SetSuccess = false // SetVars sets variables to the transaction. -func (txn *KVTxn) SetVars(vars *kv.Variables) { +func (txn *KVTxn) SetVars(vars *tikv.Variables) { txn.vars = vars txn.snapshot.vars = vars failpoint.Inject("probeSetVars", func(val failpoint.Value) { @@ -131,7 +171,7 @@ func (txn *KVTxn) SetVars(vars *kv.Variables) { } // GetVars gets variables from the transaction. -func (txn *KVTxn) GetVars() *kv.Variables { +func (txn *KVTxn) GetVars() *tikv.Variables { return txn.vars } @@ -184,7 +224,7 @@ func (txn *KVTxn) SetOption(opt int, val interface{}) { txn.us.SetOption(opt, val) txn.snapshot.SetOption(opt, val) switch opt { - case kv.SchemaAmender: + case tikv.SchemaAmender: txn.schemaAmender = val.(SchemaAmender) } } @@ -442,8 +482,8 @@ func (txn *KVTxn) onCommitted(err error) { } // LockKeys tries to lock the entries with the keys in KV store. -// lockWaitTime in ms, except that tidbkv.LockAlwaysWait(0) means always wait lock, tidbkv.LockNowait(-1) means nowait lock -func (txn *KVTxn) LockKeys(ctx context.Context, lockCtx *kv.LockCtx, keysInput ...[]byte) error { +// lockWaitTime in ms, except that kv.LockAlwaysWait(0) means always wait lock, kv.LockNowait(-1) means nowait lock +func (txn *KVTxn) LockKeys(ctx context.Context, lockCtx *tikv.LockCtx, keysInput ...[]byte) error { // Exclude keys that are already locked. var err error keys := make([][]byte, 0, len(keysInput)) @@ -494,7 +534,7 @@ func (txn *KVTxn) LockKeys(ctx context.Context, lockCtx *kv.LockCtx, keysInput . if lockCtx.ReturnValues && locked { // An already locked key can not return values, we add an entry to let the caller get the value // in other ways. - lockCtx.Values[string(key)] = kv.ReturnedValue{AlreadyLocked: true} + lockCtx.Values[string(key)] = tikv.ReturnedValue{AlreadyLocked: true} } } if len(keys) == 0 { @@ -574,16 +614,16 @@ func (txn *KVTxn) LockKeys(ctx context.Context, lockCtx *kv.LockCtx, keysInput . } } for _, key := range keys { - valExists := kv.SetKeyLockedValueExists + valExists := tikv.SetKeyLockedValueExists // PointGet and BatchPointGet will return value in pessimistic lock response, the value may not exist. // For other lock modes, the locked key values always exist. if lockCtx.ReturnValues { val, _ := lockCtx.Values[string(key)] if len(val.Value) == 0 { - valExists = kv.SetKeyLockedValueNotExists + valExists = tikv.SetKeyLockedValueNotExists } } - memBuf.UpdateFlags(key, kv.SetKeyLocked, kv.DelNeedCheckExists, valExists) + memBuf.UpdateFlags(key, tikv.SetKeyLocked, tikv.DelNeedCheckExists, valExists) } txn.lockedCnt += len(keys) return nil From d27fda32fb20a6db930ea41a71053eb1a96a25d2 Mon Sep 17 00:00:00 2001 From: ZhuoZhi <517770911@qq.com> Date: Tue, 11 May 2021 11:59:39 +0800 Subject: [PATCH 51/67] executor: add correctness tests about direct reading with aggregations (#24491) --- executor/partition_table_test.go | 144 +++++++++++++++++++++++++++++++ 1 file changed, 144 insertions(+) diff --git a/executor/partition_table_test.go b/executor/partition_table_test.go index 555e152265349..8ce126972ec73 100644 --- a/executor/partition_table_test.go +++ b/executor/partition_table_test.go @@ -434,6 +434,150 @@ func (s *partitionTableSuite) TestGlobalStatsAndSQLBinding(c *C) { tk.MustIndexLookup("select * from tlist where a<1") } +func (s *partitionTableSuite) TestDirectReadingWithAgg(c *C) { + if israce.RaceEnabled { + c.Skip("exhaustive types test, skip race test") + } + + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create database test_dr_agg") + tk.MustExec("use test_dr_agg") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + + // list partition table + tk.MustExec(`create table tlist(a int, b int, index idx_a(a), index idx_b(b)) partition by list(a)( + partition p0 values in (1, 2, 3, 4), + partition p1 values in (5, 6, 7, 8), + partition p2 values in (9, 10, 11, 12));`) + + // range partition table + tk.MustExec(`create table trange(a int, b int, index idx_a(a), index idx_b(b)) partition by range(a) ( + partition p0 values less than(300), + partition p1 values less than (500), + partition p2 values less than(1100));`) + + // hash partition table + tk.MustExec(`create table thash(a int, b int) partition by hash(a) partitions 4;`) + + // regular table + tk.MustExec("create table tregular1(a int, b int, index idx_a(a))") + tk.MustExec("create table tregular2(a int, b int, index idx_a(a))") + + // generate some random data to be inserted + vals := make([]string, 0, 2000) + for i := 0; i < 2000; i++ { + vals = append(vals, fmt.Sprintf("(%v, %v)", rand.Intn(1100), rand.Intn(2000))) + } + + tk.MustExec("insert into trange values " + strings.Join(vals, ",")) + tk.MustExec("insert into thash values " + strings.Join(vals, ",")) + tk.MustExec("insert into tregular1 values " + strings.Join(vals, ",")) + + vals = make([]string, 0, 2000) + for i := 0; i < 2000; i++ { + vals = append(vals, fmt.Sprintf("(%v, %v)", rand.Intn(12)+1, rand.Intn(20))) + } + + tk.MustExec("insert into tlist values " + strings.Join(vals, ",")) + tk.MustExec("insert into tregular2 values " + strings.Join(vals, ",")) + + // test range partition + for i := 0; i < 2000; i++ { + // select /*+ stream_agg() */ a from t where a > ? group by a; + // select /*+ hash_agg() */ a from t where a > ? group by a; + // select /*+ stream_agg() */ a from t where a in(?, ?, ?) group by a; + // select /*+ hash_agg() */ a from t where a in (?, ?, ?) group by a; + x := rand.Intn(1099) + + queryPartition1 := fmt.Sprintf("select /*+ stream_agg() */ count(*), sum(b), max(b), a from trange where a > %v group by a;", x) + queryRegular1 := fmt.Sprintf("select /*+ stream_agg() */ count(*), sum(b), max(b), a from tregular1 where a > %v group by a;", x) + c.Assert(tk.HasPlan(queryPartition1, "StreamAgg"), IsTrue) // check if IndexLookUp is used + tk.MustQuery(queryPartition1).Sort().Check(tk.MustQuery(queryRegular1).Sort().Rows()) + + queryPartition2 := fmt.Sprintf("select /*+ hash_agg() */ count(*), sum(b), max(b), a from trange where a > %v group by a;", x) + queryRegular2 := fmt.Sprintf("select /*+ hash_agg() */ count(*), sum(b), max(b), a from tregular1 where a > %v group by a;", x) + c.Assert(tk.HasPlan(queryPartition2, "HashAgg"), IsTrue) // check if IndexLookUp is used + tk.MustQuery(queryPartition2).Sort().Check(tk.MustQuery(queryRegular2).Sort().Rows()) + + y := rand.Intn(1099) + z := rand.Intn(1099) + + queryPartition3 := fmt.Sprintf("select /*+ stream_agg() */ count(*), sum(b), max(b), a from trange where a in(%v, %v, %v) group by a;", x, y, z) + queryRegular3 := fmt.Sprintf("select /*+ stream_agg() */ count(*), sum(b), max(b), a from tregular1 where a in(%v, %v, %v) group by a;", x, y, z) + c.Assert(tk.HasPlan(queryPartition3, "StreamAgg"), IsTrue) // check if IndexLookUp is used + tk.MustQuery(queryPartition3).Sort().Check(tk.MustQuery(queryRegular3).Sort().Rows()) + + queryPartition4 := fmt.Sprintf("select /*+ hash_agg() */ count(*), sum(b), max(b), a from trange where a in (%v, %v, %v) group by a;", x, y, z) + queryRegular4 := fmt.Sprintf("select /*+ hash_agg() */ count(*), sum(b), max(b), a from tregular1 where a in (%v, %v, %v) group by a;", x, y, z) + c.Assert(tk.HasPlan(queryPartition4, "HashAgg"), IsTrue) // check if IndexLookUp is used + tk.MustQuery(queryPartition4).Sort().Check(tk.MustQuery(queryRegular4).Sort().Rows()) + } + + // test hash partition + for i := 0; i < 2000; i++ { + // select /*+ stream_agg() */ a from t where a > ? group by a; + // select /*+ hash_agg() */ a from t where a > ? group by a; + // select /*+ stream_agg() */ a from t where a in(?, ?, ?) group by a; + // select /*+ hash_agg() */ a from t where a in (?, ?, ?) group by a; + x := rand.Intn(1099) + + queryPartition1 := fmt.Sprintf("select /*+ stream_agg() */ count(*), sum(b), max(b), a from thash where a > %v group by a;", x) + queryRegular1 := fmt.Sprintf("select /*+ stream_agg() */ count(*), sum(b), max(b), a from tregular1 where a > %v group by a;", x) + c.Assert(tk.HasPlan(queryPartition1, "StreamAgg"), IsTrue) // check if IndexLookUp is used + tk.MustQuery(queryPartition1).Sort().Check(tk.MustQuery(queryRegular1).Sort().Rows()) + + queryPartition2 := fmt.Sprintf("select /*+ hash_agg() */ count(*), sum(b), max(b), a from thash where a > %v group by a;", x) + queryRegular2 := fmt.Sprintf("select /*+ hash_agg() */ count(*), sum(b), max(b), a from tregular1 where a > %v group by a;", x) + c.Assert(tk.HasPlan(queryPartition2, "HashAgg"), IsTrue) // check if IndexLookUp is used + tk.MustQuery(queryPartition2).Sort().Check(tk.MustQuery(queryRegular2).Sort().Rows()) + + y := rand.Intn(1099) + z := rand.Intn(1099) + + queryPartition3 := fmt.Sprintf("select /*+ stream_agg() */ count(*), sum(b), max(b), a from thash where a in(%v, %v, %v) group by a;", x, y, z) + queryRegular3 := fmt.Sprintf("select /*+ stream_agg() */ count(*), sum(b), max(b), a from tregular1 where a in(%v, %v, %v) group by a;", x, y, z) + c.Assert(tk.HasPlan(queryPartition3, "StreamAgg"), IsTrue) // check if IndexLookUp is used + tk.MustQuery(queryPartition3).Sort().Check(tk.MustQuery(queryRegular3).Sort().Rows()) + + queryPartition4 := fmt.Sprintf("select /*+ hash_agg() */ count(*), sum(b), max(b), a from thash where a in (%v, %v, %v) group by a;", x, y, z) + queryRegular4 := fmt.Sprintf("select /*+ hash_agg() */ count(*), sum(b), max(b), a from tregular1 where a in (%v, %v, %v) group by a;", x, y, z) + c.Assert(tk.HasPlan(queryPartition4, "HashAgg"), IsTrue) // check if IndexLookUp is used + tk.MustQuery(queryPartition4).Sort().Check(tk.MustQuery(queryRegular4).Sort().Rows()) + } + + // test list partition + for i := 0; i < 2000; i++ { + // select /*+ stream_agg() */ a from t where a > ? group by a; + // select /*+ hash_agg() */ a from t where a > ? group by a; + // select /*+ stream_agg() */ a from t where a in(?, ?, ?) group by a; + // select /*+ hash_agg() */ a from t where a in (?, ?, ?) group by a; + x := rand.Intn(12) + 1 + + queryPartition1 := fmt.Sprintf("select /*+ stream_agg() */ count(*), sum(b), max(b), a from tlist where a > %v group by a;", x) + queryRegular1 := fmt.Sprintf("select /*+ stream_agg() */ count(*), sum(b), max(b), a from tregular2 where a > %v group by a;", x) + c.Assert(tk.HasPlan(queryPartition1, "StreamAgg"), IsTrue) // check if IndexLookUp is used + tk.MustQuery(queryPartition1).Sort().Check(tk.MustQuery(queryRegular1).Sort().Rows()) + + queryPartition2 := fmt.Sprintf("select /*+ hash_agg() */ count(*), sum(b), max(b), a from tlist where a > %v group by a;", x) + queryRegular2 := fmt.Sprintf("select /*+ hash_agg() */ count(*), sum(b), max(b), a from tregular2 where a > %v group by a;", x) + c.Assert(tk.HasPlan(queryPartition2, "HashAgg"), IsTrue) // check if IndexLookUp is used + tk.MustQuery(queryPartition2).Sort().Check(tk.MustQuery(queryRegular2).Sort().Rows()) + + y := rand.Intn(12) + 1 + z := rand.Intn(12) + 1 + + queryPartition3 := fmt.Sprintf("select /*+ stream_agg() */ count(*), sum(b), max(b), a from tlist where a in(%v, %v, %v) group by a;", x, y, z) + queryRegular3 := fmt.Sprintf("select /*+ stream_agg() */ count(*), sum(b), max(b), a from tregular2 where a in(%v, %v, %v) group by a;", x, y, z) + c.Assert(tk.HasPlan(queryPartition3, "StreamAgg"), IsTrue) // check if IndexLookUp is used + tk.MustQuery(queryPartition3).Sort().Check(tk.MustQuery(queryRegular3).Sort().Rows()) + + queryPartition4 := fmt.Sprintf("select /*+ hash_agg() */ count(*), sum(b), max(b), a from tlist where a in (%v, %v, %v) group by a;", x, y, z) + queryRegular4 := fmt.Sprintf("select /*+ hash_agg() */ count(*), sum(b), max(b), a from tregular2 where a in (%v, %v, %v) group by a;", x, y, z) + c.Assert(tk.HasPlan(queryPartition4, "HashAgg"), IsTrue) // check if IndexLookUp is used + tk.MustQuery(queryPartition4).Sort().Check(tk.MustQuery(queryRegular4).Sort().Rows()) + } +} + func (s *globalIndexSuite) TestGlobalIndexScan(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) tk.MustExec("drop table if exists p") From daf133cf81bff352a2c0732cc3e8b20a0e6fe111 Mon Sep 17 00:00:00 2001 From: Shirly Date: Tue, 11 May 2021 12:13:39 +0800 Subject: [PATCH 52/67] store/copr: removes the unused fields in batchCopIterator (#24446) --- store/copr/batch_coprocessor.go | 18 +++++------------- 1 file changed, 5 insertions(+), 13 deletions(-) diff --git a/store/copr/batch_coprocessor.go b/store/copr/batch_coprocessor.go index 5506c9d497ac2..3231f95bbc824 100644 --- a/store/copr/batch_coprocessor.go +++ b/store/copr/batch_coprocessor.go @@ -33,8 +33,6 @@ import ( "github.com/pingcap/tidb/store/tikv/logutil" "github.com/pingcap/tidb/store/tikv/metrics" "github.com/pingcap/tidb/store/tikv/tikvrpc" - "github.com/pingcap/tidb/store/tikv/util" - "github.com/pingcap/tidb/util/memory" "go.uber.org/zap" ) @@ -184,13 +182,11 @@ func (c *CopClient) sendBatch(ctx context.Context, req *kv.Request, vars *tikv.V return copErrorResponse{err} } it := &batchCopIterator{ - store: c.store.kvStore, - req: req, - finishCh: make(chan struct{}), - vars: vars, - memTracker: req.MemTracker, - ClientHelper: tikv.NewClientHelper(c.store.kvStore.store, util.NewTSSet(5)), - rpcCancel: tikv.NewRPCanceller(), + store: c.store.kvStore, + req: req, + finishCh: make(chan struct{}), + vars: vars, + rpcCancel: tikv.NewRPCanceller(), } ctx = context.WithValue(ctx, tikv.RPCCancellerCtxKey{}, it.rpcCancel) it.tasks = tasks @@ -200,8 +196,6 @@ func (c *CopClient) sendBatch(ctx context.Context, req *kv.Request, vars *tikv.V } type batchCopIterator struct { - *tikv.ClientHelper - store *kvStore req *kv.Request finishCh chan struct{} @@ -213,8 +207,6 @@ type batchCopIterator struct { vars *tikv.Variables - memTracker *memory.Tracker - rpcCancel *tikv.RPCCanceller wg sync.WaitGroup From 9b86513a4dc9cf22f4fe92a6b4faabad1233160b Mon Sep 17 00:00:00 2001 From: Shirly Date: Tue, 11 May 2021 12:25:38 +0800 Subject: [PATCH 53/67] =?UTF-8?q?store/tikv:=20make=20tikv.ErrLockWaitTime?= =?UTF-8?q?out=20as=20a=20normal=20error=20instead=20of=E2=80=A6=20(#24424?= =?UTF-8?q?)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- session/pessimistic_test.go | 19 ++++++++++--------- store/driver/txn/error.go | 6 ++++++ store/tikv/error/error.go | 3 ++- store/tikv/tests/2pc_test.go | 2 +- 4 files changed, 19 insertions(+), 11 deletions(-) diff --git a/session/pessimistic_test.go b/session/pessimistic_test.go index 05a93e3aee30d..8fdd635b51bc1 100644 --- a/session/pessimistic_test.go +++ b/session/pessimistic_test.go @@ -31,6 +31,7 @@ import ( plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx/variable" + txndriver "github.com/pingcap/tidb/store/driver/txn" "github.com/pingcap/tidb/store/tikv" tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/store/tikv/oracle" @@ -733,10 +734,10 @@ func (s *testPessimisticSuite) TestInnodbLockWaitTimeout(c *C) { timeoutErr := <-timeoutErrCh c.Assert(timeoutErr, NotNil) - c.Assert(timeoutErr.Error(), Equals, tikverr.ErrLockWaitTimeout.Error()) + c.Assert(timeoutErr.Error(), Equals, txndriver.ErrLockWaitTimeout.Error()) timeoutErr = <-timeoutErrCh c.Assert(timeoutErr, NotNil) - c.Assert(timeoutErr.Error(), Equals, tikverr.ErrLockWaitTimeout.Error()) + c.Assert(timeoutErr.Error(), Equals, txndriver.ErrLockWaitTimeout.Error()) // tk4 lock c1 = 2 tk4.MustExec("begin pessimistic") @@ -749,7 +750,7 @@ func (s *testPessimisticSuite) TestInnodbLockWaitTimeout(c *C) { _, err := tk2.Exec("delete from tk where c1 = 2") c.Check(time.Since(start), GreaterEqual, 1000*time.Millisecond) c.Check(time.Since(start), Less, 3000*time.Millisecond) // unit test diff should not be too big - c.Check(err.Error(), Equals, tikverr.ErrLockWaitTimeout.Error()) + c.Check(err.Error(), Equals, txndriver.ErrLockWaitTimeout.Error()) tk4.MustExec("commit") @@ -767,7 +768,7 @@ func (s *testPessimisticSuite) TestInnodbLockWaitTimeout(c *C) { _, err = tk2.Exec("delete from tk where c1 = 3") // tk2 tries to lock c1 = 3 fail, this delete should be rollback, but previous update should be keeped c.Check(time.Since(start), GreaterEqual, 1000*time.Millisecond) c.Check(time.Since(start), Less, 3000*time.Millisecond) // unit test diff should not be too big - c.Check(err.Error(), Equals, tikverr.ErrLockWaitTimeout.Error()) + c.Check(err.Error(), Equals, txndriver.ErrLockWaitTimeout.Error()) tk2.MustExec("commit") tk3.MustExec("commit") @@ -841,7 +842,7 @@ func (s *testPessimisticSuite) TestInnodbLockWaitTimeoutWaitStart(c *C) { c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/PessimisticLockErrWriteConflict"), IsNil) waitErr := <-done c.Assert(waitErr, NotNil) - c.Check(waitErr.Error(), Equals, tikverr.ErrLockWaitTimeout.Error()) + c.Check(waitErr.Error(), Equals, txndriver.ErrLockWaitTimeout.Error()) c.Check(duration, GreaterEqual, 1000*time.Millisecond) c.Check(duration, LessEqual, 3000*time.Millisecond) tk2.MustExec("rollback") @@ -1279,7 +1280,7 @@ func (s *testPessimisticSuite) TestBatchPointGetLockIndex(c *C) { tk2.MustExec("begin pessimistic") err := tk2.ExecToErr("insert into t1 values(2, 2, 2)") c.Assert(err, NotNil) - c.Assert(tikverr.ErrLockWaitTimeout.Equal(err), IsTrue) + c.Assert(txndriver.ErrLockWaitTimeout.Equal(err), IsTrue) err = tk2.ExecToErr("select * from t1 where c2 = 3 for update nowait") c.Assert(err, NotNil) c.Assert(tikverr.ErrLockAcquireFailAndNoWaitSet.Equal(err), IsTrue) @@ -1996,11 +1997,11 @@ func (s *testPessimisticSuite) TestSelectForUpdateWaitSeconds(c *C) { waitErr2 := <-errCh waitErr3 := <-errCh c.Assert(waitErr, NotNil) - c.Check(waitErr.Error(), Equals, tikverr.ErrLockWaitTimeout.Error()) + c.Check(waitErr.Error(), Equals, txndriver.ErrLockWaitTimeout.Error()) c.Assert(waitErr2, NotNil) - c.Check(waitErr2.Error(), Equals, tikverr.ErrLockWaitTimeout.Error()) + c.Check(waitErr2.Error(), Equals, txndriver.ErrLockWaitTimeout.Error()) c.Assert(waitErr3, NotNil) - c.Check(waitErr3.Error(), Equals, tikverr.ErrLockWaitTimeout.Error()) + c.Check(waitErr3.Error(), Equals, txndriver.ErrLockWaitTimeout.Error()) c.Assert(time.Since(start).Seconds(), Less, 45.0) tk2.MustExec("commit") tk3.MustExec("rollback") diff --git a/store/driver/txn/error.go b/store/driver/txn/error.go index c026b8899ae72..5bbdc01024947 100644 --- a/store/driver/txn/error.go +++ b/store/driver/txn/error.go @@ -50,6 +50,8 @@ var ( // ErrTiKVMaxTimestampNotSynced is the error that tikv's max timestamp is not synced. ErrTiKVMaxTimestampNotSynced = dbterror.ClassTiKV.NewStd(errno.ErrTiKVMaxTimestampNotSynced) ErrResolveLockTimeout = dbterror.ClassTiKV.NewStd(errno.ErrResolveLockTimeout) + // ErrLockWaitTimeout is the error that wait for the lock is timeout. + ErrLockWaitTimeout = dbterror.ClassTiKV.NewStd(errno.ErrLockWaitTimeout) // ErrTiKVServerBusy is the error when tikv server is busy. ErrTiKVServerBusy = dbterror.ClassTiKV.NewStd(errno.ErrTiKVServerBusy) // ErrTiFlashServerBusy is the error that tiflash server is busy. @@ -251,6 +253,10 @@ func ToTiDBErr(err error) error { return ErrResolveLockTimeout } + if errors.ErrorEqual(err, tikverr.ErrLockWaitTimeout) { + return ErrLockWaitTimeout + } + if errors.ErrorEqual(err, tikverr.ErrRegionUnavailable) { return ErrRegionUnavailable } diff --git a/store/tikv/error/error.go b/store/tikv/error/error.go index 70ec995e88489..bde97e8b2e5d4 100644 --- a/store/tikv/error/error.go +++ b/store/tikv/error/error.go @@ -44,6 +44,8 @@ var ( ErrTiKVMaxTimestampNotSynced = errors.New("tikv max timestamp not synced") // ErrResolveLockTimeout is the error that resolve lock timeout. ErrResolveLockTimeout = errors.New("resolve lock timeout") + // ErrLockWaitTimeout is the error that wait for the lock is timeout. + ErrLockWaitTimeout = errors.New("lock wait timeout") // ErrTiKVServerBusy is the error when tikv server is busy. ErrTiKVServerBusy = errors.New("tikv server busy") // ErrTiFlashServerBusy is the error that tiflash server is busy. @@ -59,7 +61,6 @@ const MismatchClusterID = "mismatch cluster id" var ( ErrQueryInterrupted = dbterror.ClassTiKV.NewStd(CodeQueryInterrupted) ErrLockAcquireFailAndNoWaitSet = dbterror.ClassTiKV.NewStd(CodeLockAcquireFailAndNoWaitSet) - ErrLockWaitTimeout = dbterror.ClassTiKV.NewStd(CodeLockWaitTimeout) ErrUnknown = dbterror.ClassTiKV.NewStd(CodeUnknown) ) diff --git a/store/tikv/tests/2pc_test.go b/store/tikv/tests/2pc_test.go index 8043ccdab89f0..d1e635f205efa 100644 --- a/store/tikv/tests/2pc_test.go +++ b/store/tikv/tests/2pc_test.go @@ -1025,7 +1025,7 @@ func (s *testCommitterSuite) TestPessimisticLockPrimary(c *C) { c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/txnNotFoundRetTTL"), IsNil) c.Assert(err, IsNil) waitErr := <-doneCh - c.Assert(tikverr.ErrLockWaitTimeout.Equal(waitErr), IsTrue) + c.Assert(tikverr.ErrLockWaitTimeout, Equals, waitErr) } func (s *testCommitterSuite) TestResolvePessimisticLock(c *C) { From 3fed33fc17354038cebc121669b67e3dcecd21ad Mon Sep 17 00:00:00 2001 From: rebelice Date: Tue, 11 May 2021 12:47:38 +0800 Subject: [PATCH 54/67] statistics: dump FMSketch to KV only for partition table with dynamic prune mode (#24453) --- executor/analyze.go | 7 +-- statistics/handle/ddl.go | 6 ++- statistics/handle/dump.go | 6 ++- statistics/handle/dump_test.go | 2 +- statistics/handle/handle.go | 4 +- statistics/handle/handle_test.go | 85 +++++++++++++++++++++++--------- statistics/handle/update.go | 11 +++-- 7 files changed, 83 insertions(+), 38 deletions(-) diff --git a/executor/analyze.go b/executor/analyze.go index b7e9e51fec7da..8345382939865 100644 --- a/executor/analyze.go +++ b/executor/analyze.go @@ -160,9 +160,9 @@ func (e *AnalyzeExec) Next(ctx context.Context, req *chunk.Chunk) error { } var err1 error if result.StatsVer == statistics.Version3 { - err1 = statsHandle.SaveStatsToStorage(statisticsID, result.Count, result.IsIndex, hg, nil, result.TopNs[i], result.Fms[i], result.StatsVer, 1) + err1 = statsHandle.SaveStatsToStorage(statisticsID, result.Count, result.IsIndex, hg, nil, result.TopNs[i], result.Fms[i], result.StatsVer, 1, result.TableID.IsPartitionTable() && needGlobalStats) } else { - err1 = statsHandle.SaveStatsToStorage(statisticsID, result.Count, result.IsIndex, hg, result.Cms[i], result.TopNs[i], result.Fms[i], result.StatsVer, 1) + err1 = statsHandle.SaveStatsToStorage(statisticsID, result.Count, result.IsIndex, hg, result.Cms[i], result.TopNs[i], result.Fms[i], result.StatsVer, 1, result.TableID.IsPartitionTable() && needGlobalStats) } if err1 != nil { err = err1 @@ -198,7 +198,8 @@ func (e *AnalyzeExec) Next(ctx context.Context, req *chunk.Chunk) error { } for i := 0; i < globalStats.Num; i++ { hg, cms, topN, fms := globalStats.Hg[i], globalStats.Cms[i], globalStats.TopN[i], globalStats.Fms[i] - err = statsHandle.SaveStatsToStorage(globalStatsID.tableID, globalStats.Count, info.isIndex, hg, cms, topN, fms, info.statsVersion, 1) + // fms for global stats doesn't need to dump to kv. + err = statsHandle.SaveStatsToStorage(globalStatsID.tableID, globalStats.Count, info.isIndex, hg, cms, topN, fms, info.statsVersion, 1, false) if err != nil { logutil.Logger(ctx).Error("save global-level stats to storage failed", zap.Error(err)) } diff --git a/statistics/handle/ddl.go b/statistics/handle/ddl.go index 34f30c1241e1c..2ddc460ff690b 100644 --- a/statistics/handle/ddl.go +++ b/statistics/handle/ddl.go @@ -112,7 +112,8 @@ func (h *Handle) updateGlobalStats(tblInfo *model.TableInfo) error { } for i := 0; i < newColGlobalStats.Num; i++ { hg, cms, topN, fms := newColGlobalStats.Hg[i], newColGlobalStats.Cms[i], newColGlobalStats.TopN[i], newColGlobalStats.Fms[i] - err = h.SaveStatsToStorage(tableID, newColGlobalStats.Count, 0, hg, cms, topN, fms, 2, 1) + // fms for global stats doesn't need to dump to kv. + err = h.SaveStatsToStorage(tableID, newColGlobalStats.Count, 0, hg, cms, topN, fms, 2, 1, false) if err != nil { return err } @@ -141,7 +142,8 @@ func (h *Handle) updateGlobalStats(tblInfo *model.TableInfo) error { } for i := 0; i < newIndexGlobalStats.Num; i++ { hg, cms, topN, fms := newIndexGlobalStats.Hg[i], newIndexGlobalStats.Cms[i], newIndexGlobalStats.TopN[i], newIndexGlobalStats.Fms[i] - err = h.SaveStatsToStorage(tableID, newIndexGlobalStats.Count, 1, hg, cms, topN, fms, 2, 1) + // fms for global stats doesn't need to dump to kv. + err = h.SaveStatsToStorage(tableID, newIndexGlobalStats.Count, 1, hg, cms, topN, fms, 2, 1, false) if err != nil { return err } diff --git a/statistics/handle/dump.go b/statistics/handle/dump.go index 1f20855742a76..36971076b4644 100644 --- a/statistics/handle/dump.go +++ b/statistics/handle/dump.go @@ -230,13 +230,15 @@ func (h *Handle) loadStatsFromJSON(tableInfo *model.TableInfo, physicalID int64, } for _, col := range tbl.Columns { - err = h.SaveStatsToStorage(tbl.PhysicalID, tbl.Count, 0, &col.Histogram, col.CMSketch, col.TopN, col.FMSketch, int(col.StatsVer), 1) + // loadStatsFromJSON doesn't support partition table now. + err = h.SaveStatsToStorage(tbl.PhysicalID, tbl.Count, 0, &col.Histogram, col.CMSketch, col.TopN, col.FMSketch, int(col.StatsVer), 1, false) if err != nil { return errors.Trace(err) } } for _, idx := range tbl.Indices { - err = h.SaveStatsToStorage(tbl.PhysicalID, tbl.Count, 1, &idx.Histogram, idx.CMSketch, idx.TopN, nil, int(idx.StatsVer), 1) + // loadStatsFromJSON doesn't support partition table now. + err = h.SaveStatsToStorage(tbl.PhysicalID, tbl.Count, 1, &idx.Histogram, idx.CMSketch, idx.TopN, nil, int(idx.StatsVer), 1, false) if err != nil { return errors.Trace(err) } diff --git a/statistics/handle/dump_test.go b/statistics/handle/dump_test.go index ef02403c48ac5..f0c7c2fb09110 100644 --- a/statistics/handle/dump_test.go +++ b/statistics/handle/dump_test.go @@ -215,7 +215,7 @@ func (s *testStatsSuite) TestDumpCMSketchWithTopN(c *C) { cms, _, _, _ := statistics.NewCMSketchAndTopN(5, 2048, fakeData, 20, 100) stat := h.GetTableStats(tableInfo) - err = h.SaveStatsToStorage(tableInfo.ID, 1, 0, &stat.Columns[tableInfo.Columns[0].ID].Histogram, cms, nil, nil, statistics.Version2, 1) + err = h.SaveStatsToStorage(tableInfo.ID, 1, 0, &stat.Columns[tableInfo.Columns[0].ID].Histogram, cms, nil, nil, statistics.Version2, 1, false) c.Assert(err, IsNil) c.Assert(h.Update(is), IsNil) diff --git a/statistics/handle/handle.go b/statistics/handle/handle.go index 7036242b4e3a7..a8f51f8924e81 100644 --- a/statistics/handle/handle.go +++ b/statistics/handle/handle.go @@ -952,7 +952,7 @@ func (h *Handle) extendedStatsFromStorage(reader *statsReader, table *statistics } // SaveStatsToStorage saves the stats to storage. -func (h *Handle) SaveStatsToStorage(tableID int64, count int64, isIndex int, hg *statistics.Histogram, cms *statistics.CMSketch, topN *statistics.TopN, fms *statistics.FMSketch, statsVersion int, isAnalyzed int64) (err error) { +func (h *Handle) SaveStatsToStorage(tableID int64, count int64, isIndex int, hg *statistics.Histogram, cms *statistics.CMSketch, topN *statistics.TopN, fms *statistics.FMSketch, statsVersion int, isAnalyzed int64, needDumpFMS bool) (err error) { h.mu.Lock() defer h.mu.Unlock() ctx := context.TODO() @@ -1001,7 +1001,7 @@ func (h *Handle) SaveStatsToStorage(tableID int64, count int64, isIndex int, hg if _, err := exec.ExecuteInternal(ctx, "delete from mysql.stats_fm_sketch where table_id = %? and is_index = %? and hist_id = %?", tableID, isIndex, hg.ID); err != nil { return err } - if fmSketch != nil { + if fmSketch != nil && needDumpFMS { if _, err = exec.ExecuteInternal(ctx, "insert into mysql.stats_fm_sketch (table_id, is_index, hist_id, value) values (%?, %?, %?, %?)", tableID, isIndex, hg.ID, fmSketch); err != nil { return err } diff --git a/statistics/handle/handle_test.go b/statistics/handle/handle_test.go index 3bb95e8219478..2bf1d93ff4246 100644 --- a/statistics/handle/handle_test.go +++ b/statistics/handle/handle_test.go @@ -2332,16 +2332,18 @@ func (s *testStatsSuite) TestDuplicateFMSketch(c *C) { defer cleanEnv(c, s.store, s.do) tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") - tk.MustExec("create table t(a int, b int, c int)") + tk.MustExec("set @@tidb_partition_prune_mode='dynamic'") + defer tk.MustExec("set @@tidb_partition_prune_mode='static'") + tk.MustExec("create table t(a int, b int, c int) partition by hash(a) partitions 3") tk.MustExec("insert into t values (1, 1, 1)") tk.MustExec("analyze table t") - tk.MustQuery("select count(*) from mysql.stats_fm_sketch").Check(testkit.Rows("3")) + tk.MustQuery("select count(*) from mysql.stats_fm_sketch").Check(testkit.Rows("9")) tk.MustExec("analyze table t") - tk.MustQuery("select count(*) from mysql.stats_fm_sketch").Check(testkit.Rows("3")) + tk.MustQuery("select count(*) from mysql.stats_fm_sketch").Check(testkit.Rows("9")) - tk.MustExec("alter table t drop column a") + tk.MustExec("alter table t drop column b") c.Assert(s.do.StatsHandle().GCStats(s.do.InfoSchema(), time.Duration(0)), IsNil) - tk.MustQuery("select count(*) from mysql.stats_fm_sketch").Check(testkit.Rows("2")) + tk.MustQuery("select count(*) from mysql.stats_fm_sketch").Check(testkit.Rows("6")) } func (s *testStatsSuite) TestIndexFMSketch(c *C) { @@ -2349,24 +2351,26 @@ func (s *testStatsSuite) TestIndexFMSketch(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t") - tk.MustExec("create table t(a int, b int, c int, index ia(a), index ibc(b, c))") + tk.MustExec("create table t(a int, b int, c int, index ia(a), index ibc(b, c)) partition by hash(a) partitions 3") tk.MustExec("insert into t values (1, 1, 1)") + tk.MustExec("set @@tidb_partition_prune_mode='dynamic'") + defer tk.MustExec("set @@tidb_partition_prune_mode='static'") tk.MustExec("analyze table t index ia") - tk.MustQuery("select count(*) from mysql.stats_fm_sketch").Check(testkit.Rows("1")) + tk.MustQuery("select count(*) from mysql.stats_fm_sketch").Check(testkit.Rows("3")) tk.MustExec("analyze table t index ibc") - tk.MustQuery("select count(*) from mysql.stats_fm_sketch").Check(testkit.Rows("2")) + tk.MustQuery("select count(*) from mysql.stats_fm_sketch").Check(testkit.Rows("6")) tk.MustExec("analyze table t") - tk.MustQuery("select count(*) from mysql.stats_fm_sketch").Check(testkit.Rows("5")) + tk.MustQuery("select count(*) from mysql.stats_fm_sketch").Check(testkit.Rows("15")) tk.MustExec("drop table if exists t") c.Assert(s.do.StatsHandle().GCStats(s.do.InfoSchema(), 0), IsNil) // clustered index tk.MustExec("drop table if exists t") tk.MustExec("set @@tidb_enable_clustered_index=ON") - tk.MustExec("create table t (a datetime, b datetime, primary key (a))") + tk.MustExec("create table t (a datetime, b datetime, primary key (a)) partition by hash(year(a)) partitions 3") tk.MustExec("insert into t values ('2000-01-01', '2000-01-01')") tk.MustExec("analyze table t") - tk.MustQuery("select count(*) from mysql.stats_fm_sketch").Check(testkit.Rows("2")) + tk.MustQuery("select count(*) from mysql.stats_fm_sketch").Check(testkit.Rows("6")) tk.MustExec("drop table if exists t") c.Assert(s.do.StatsHandle().GCStats(s.do.InfoSchema(), 0), IsNil) @@ -2383,25 +2387,23 @@ func (s *testStatsSuite) TestIndexFMSketch(c *C) { } tk.MustExec("set @@tidb_enable_clustered_index=OFF") - tk.MustExec("create table t(a int, key(a))") + tk.MustExec("create table t(a int, key(a)) partition by hash(a) partitions 3") tk.MustExec("insert into t values (1), (2), (2), (3)") - checkNDV(2, 3) - tk.MustExec("insert into t values (4), (5)") - checkNDV(2, 5) + checkNDV(6, 1) + tk.MustExec("insert into t values (4), (5), (6)") + checkNDV(6, 2) tk.MustExec("insert into t values (2), (5)") - checkNDV(2, 5) + checkNDV(6, 2) tk.MustExec("drop table if exists t") c.Assert(s.do.StatsHandle().GCStats(s.do.InfoSchema(), 0), IsNil) // clustered index tk.MustExec("set @@tidb_enable_clustered_index=ON") - tk.MustExec("create table t (a datetime, b datetime, primary key (a))") - tk.MustExec("insert into t values ('2000-01-01', '2000-01-01')") - checkNDV(2, 1) - tk.MustExec("insert into t values ('2020-01-01', '2020-01-01')") - checkNDV(2, 2) - tk.MustExec("insert into t values ('1999-01-01', '1999-01-01'), ('1999-01-02', '1999-01-02'), ('1999-01-03', '1999-01-03')") - checkNDV(2, 5) + tk.MustExec("create table t (a datetime, b datetime, primary key (a)) partition by hash(year(a)) partitions 3") + tk.MustExec("insert into t values ('2000-01-01', '2001-01-01'), ('2001-01-01', '2001-01-01'), ('2002-01-01', '2001-01-01')") + checkNDV(6, 1) + tk.MustExec("insert into t values ('1999-01-01', '1998-01-01'), ('1997-01-02', '1999-01-02'), ('1998-01-03', '1999-01-03')") + checkNDV(6, 2) } func (s *testStatsSuite) TestShowExtendedStats4DropColumn(c *C) { @@ -2802,3 +2804,40 @@ func (s *testSerialStatsSuite) TestIssues24349(c *C) { "test t global b 0 1 10 1 4 4 0", )) } + +func (s *testStatsSuite) TestIssues24401(c *C) { + defer cleanEnv(c, s.store, s.do) + testKit := testkit.NewTestKit(c, s.store) + testKit.MustExec("use test") + + // normal table with static prune mode + testKit.MustExec("set @@tidb_partition_prune_mode='static'") + testKit.MustExec("create table t(a int, index(a))") + testKit.MustExec("insert into t values (1), (2), (3)") + testKit.MustExec("analyze table t") + testKit.MustQuery("select * from mysql.stats_fm_sketch").Check(testkit.Rows()) + + // partition table with static prune mode + testKit.MustExec("create table tp(a int, index(a)) partition by hash(a) partitions 3") + testKit.MustExec("insert into tp values (1), (2), (3)") + testKit.MustExec("analyze table tp") + testKit.MustQuery("select * from mysql.stats_fm_sketch").Check(testkit.Rows()) + + // normal table with dynamic prune mode + testKit.MustExec("set @@tidb_partition_prune_mode='dynamic'") + defer testKit.MustExec("set @@tidb_partition_prune_mode='static'") + testKit.MustExec("analyze table t") + testKit.MustQuery("select * from mysql.stats_fm_sketch").Check(testkit.Rows()) + + // partition table with dynamic prune mode + testKit.MustExec("analyze table tp") + rows := testKit.MustQuery("select * from mysql.stats_fm_sketch").Rows() + lenRows := len(rows) + c.Assert(lenRows, Equals, 6) + + // check fm-sketch won't increase infinitely + testKit.MustExec("insert into t values (10), (20), (30), (12), (23), (23), (4344)") + testKit.MustExec("analyze table tp") + rows = testKit.MustQuery("select * from mysql.stats_fm_sketch").Rows() + c.Assert(len(rows), Equals, lenRows) +} diff --git a/statistics/handle/update.go b/statistics/handle/update.go index 6f472fc61fdc7..21438adc5b29d 100644 --- a/statistics/handle/update.go +++ b/statistics/handle/update.go @@ -750,11 +750,11 @@ func (h *Handle) handleSingleHistogramUpdate(is infoschema.InfoSchema, rows []ch return nil } var tbl *statistics.Table - if table.Meta().GetPartitionInfo() == nil || h.CurrentPruneMode() == variable.Dynamic { - tbl = h.GetTableStats(table.Meta()) - } else { - tbl = h.GetPartitionStats(table.Meta(), physicalTableID) + // feedback for partition is not ready + if table.Meta().GetPartitionInfo() != nil { + return nil } + tbl = h.GetTableStats(table.Meta()) var cms *statistics.CMSketch var hist *statistics.Histogram var topN *statistics.TopN @@ -822,7 +822,8 @@ func (h *Handle) deleteOutdatedFeedback(tableID, histID, isIndex int64) error { func (h *Handle) dumpStatsUpdateToKV(tableID, isIndex int64, q *statistics.QueryFeedback, hist *statistics.Histogram, cms *statistics.CMSketch, topN *statistics.TopN, fms *statistics.FMSketch, statsVersion int64) error { hist = statistics.UpdateHistogram(hist, q, int(statsVersion)) - err := h.SaveStatsToStorage(tableID, -1, int(isIndex), hist, cms, topN, fms, int(statsVersion), 0) + // feedback for partition is not ready. + err := h.SaveStatsToStorage(tableID, -1, int(isIndex), hist, cms, topN, fms, int(statsVersion), 0, false) metrics.UpdateStatsCounter.WithLabelValues(metrics.RetLabel(err)).Inc() return errors.Trace(err) } From 24e9c1342b1b637c8cc56fa20eded5ea2f6c8510 Mon Sep 17 00:00:00 2001 From: jianzhiyao <739319867@qq.com> Date: Tue, 11 May 2021 12:59:38 +0800 Subject: [PATCH 55/67] ddl: speedup test case TestModifyColumnTime (#24469) --- ddl/db_test.go | 188 +++++++++++++++++++++++++++++++++++++------------ 1 file changed, 143 insertions(+), 45 deletions(-) diff --git a/ddl/db_test.go b/ddl/db_test.go index e865de39d3248..eddad6d0d635d 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -4801,52 +4801,9 @@ func (s *testSerialDBSuite) TestModifyColumnCharset(c *C) { } -func (s *testDBSuite1) TestModifyColumnTime(c *C) { - limit := variable.GetDDLErrorCountLimit() - variable.SetDDLErrorCountLimit(3) - - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test_db") - enableChangeColumnType := tk.Se.GetSessionVars().EnableChangeColumnType - tk.Se.GetSessionVars().EnableChangeColumnType = true - - // Set time zone to UTC. - originalTz := tk.Se.GetSessionVars().TimeZone - tk.Se.GetSessionVars().TimeZone = time.UTC - defer func() { - variable.SetDDLErrorCountLimit(limit) - tk.Se.GetSessionVars().EnableChangeColumnType = enableChangeColumnType - tk.Se.GetSessionVars().TimeZone = originalTz - }() - - now := time.Now().UTC() - now = time.Date(now.Year(), now.Month(), now.Day(), 0, 0, 0, 0, time.UTC) - timeToDate1 := now.Format("2006-01-02") - timeToDate2 := now.AddDate(0, 0, 30).Format("2006-01-02") - - timeToDatetime1 := now.Add(20 * time.Hour).Add(12 * time.Second).Format("2006-01-02 15:04:05") - timeToDatetime2 := now.Add(20 * time.Hour).Format("2006-01-02 15:04:05") - timeToDatetime3 := now.Add(12 * time.Second).Format("2006-01-02 15:04:05") - timeToDatetime4 := now.AddDate(0, 0, 30).Add(20 * time.Hour).Add(12 * time.Second).Format("2006-01-02 15:04:05") - timeToDatetime5 := now.AddDate(0, 0, 30).Add(20 * time.Hour).Format("2006-01-02 15:04:05") - - timeToTimestamp1 := now.Add(20 * time.Hour).Add(12 * time.Second).Format("2006-01-02 15:04:05") - timeToTimestamp2 := now.Add(20 * time.Hour).Format("2006-01-02 15:04:05") - timeToTimestamp3 := now.Add(12 * time.Second).Format("2006-01-02 15:04:05") - timeToTimestamp4 := now.AddDate(0, 0, 30).Add(20 * time.Hour).Add(12 * time.Second).Format("2006-01-02 15:04:05") - timeToTimestamp5 := now.AddDate(0, 0, 30).Add(20 * time.Hour).Format("2006-01-02 15:04:05") +func (s *testDBSuite1) TestModifyColumnTime_TimeToYear(c *C) { currentYear := strconv.Itoa(time.Now().Year()) - - // 1. In conversion between date/time, fraction parts are taken into account - // Refer to doc: https://dev.mysql.com/doc/refman/5.7/en/date-and-time-type-conversion.html - // 2. Failed tests are commentd to pass unit-test - tests := []struct { - from string - value string - to string - expect string - err uint16 - }{ + tests := []testModifyColumnTimeCase{ // time to year, it's reasonable to return current year and discard the time (even if MySQL may get data out of range error). {"time", `"30 20:00:12"`, "year", currentYear, 0}, {"time", `"30 20:00"`, "year", currentYear, 0}, @@ -4862,7 +4819,16 @@ func (s *testDBSuite1) TestModifyColumnTime(c *C) { {"time", `"20:00:12.498"`, "year", currentYear, 0}, {"time", `"200012.498"`, "year", currentYear, 0}, {"time", `200012.498`, "year", currentYear, 0}, + } + testModifyColumnTime(c, s.store, tests) +} +func (s *testDBSuite1) TestModifyColumnTime_TimeToDate(c *C) { + now := time.Now().UTC() + now = time.Date(now.Year(), now.Month(), now.Day(), 0, 0, 0, 0, time.UTC) + timeToDate1 := now.Format("2006-01-02") + timeToDate2 := now.AddDate(0, 0, 30).Format("2006-01-02") + tests := []testModifyColumnTimeCase{ // time to date {"time", `"30 20:00:12"`, "date", timeToDate2, 0}, {"time", `"30 20:00"`, "date", timeToDate2, 0}, @@ -4878,7 +4844,19 @@ func (s *testDBSuite1) TestModifyColumnTime(c *C) { {"time", `"20:00:12.498"`, "date", timeToDate1, 0}, {"time", `"200012.498"`, "date", timeToDate1, 0}, {"time", `200012.498`, "date", timeToDate1, 0}, + } + testModifyColumnTime(c, s.store, tests) +} +func (s *testDBSuite1) TestModifyColumnTime_TimeToDatetime(c *C) { + now := time.Now().UTC() + now = time.Date(now.Year(), now.Month(), now.Day(), 0, 0, 0, 0, time.UTC) + timeToDatetime1 := now.Add(20 * time.Hour).Add(12 * time.Second).Format("2006-01-02 15:04:05") + timeToDatetime2 := now.Add(20 * time.Hour).Format("2006-01-02 15:04:05") + timeToDatetime3 := now.Add(12 * time.Second).Format("2006-01-02 15:04:05") + timeToDatetime4 := now.AddDate(0, 0, 30).Add(20 * time.Hour).Add(12 * time.Second).Format("2006-01-02 15:04:05") + timeToDatetime5 := now.AddDate(0, 0, 30).Add(20 * time.Hour).Format("2006-01-02 15:04:05") + tests := []testModifyColumnTimeCase{ // time to datetime {"time", `"30 20:00:12"`, "datetime", timeToDatetime4, 0}, {"time", `"30 20:00"`, "datetime", timeToDatetime5, 0}, @@ -4894,7 +4872,19 @@ func (s *testDBSuite1) TestModifyColumnTime(c *C) { {"time", `"20:00:12.498"`, "datetime", timeToDatetime1, 0}, {"time", `"200012.498"`, "datetime", timeToDatetime1, 0}, {"time", `200012.498`, "datetime", timeToDatetime1, 0}, + } + testModifyColumnTime(c, s.store, tests) +} +func (s *testDBSuite1) TestModifyColumnTime_TimeToTimestamp(c *C) { + now := time.Now().UTC() + now = time.Date(now.Year(), now.Month(), now.Day(), 0, 0, 0, 0, time.UTC) + timeToTimestamp1 := now.Add(20 * time.Hour).Add(12 * time.Second).Format("2006-01-02 15:04:05") + timeToTimestamp2 := now.Add(20 * time.Hour).Format("2006-01-02 15:04:05") + timeToTimestamp3 := now.Add(12 * time.Second).Format("2006-01-02 15:04:05") + timeToTimestamp4 := now.AddDate(0, 0, 30).Add(20 * time.Hour).Add(12 * time.Second).Format("2006-01-02 15:04:05") + timeToTimestamp5 := now.AddDate(0, 0, 30).Add(20 * time.Hour).Format("2006-01-02 15:04:05") + tests := []testModifyColumnTimeCase{ // time to timestamp {"time", `"30 20:00:12"`, "timestamp", timeToTimestamp4, 0}, {"time", `"30 20:00"`, "timestamp", timeToTimestamp5, 0}, @@ -4910,7 +4900,12 @@ func (s *testDBSuite1) TestModifyColumnTime(c *C) { {"time", `"20:00:12.498"`, "timestamp", timeToTimestamp1, 0}, {"time", `"200012.498"`, "timestamp", timeToTimestamp1, 0}, {"time", `200012.498`, "timestamp", timeToTimestamp1, 0}, + } + testModifyColumnTime(c, s.store, tests) +} +func (s *testDBSuite1) TestModifyColumnTime_DateToTime(c *C) { + tests := []testModifyColumnTimeCase{ // date to time {"date", `"2019-01-02"`, "time", "00:00:00", 0}, {"date", `"19-01-02"`, "time", "00:00:00", 0}, @@ -4918,7 +4913,12 @@ func (s *testDBSuite1) TestModifyColumnTime(c *C) { {"date", `"190102"`, "time", "00:00:00", 0}, {"date", `20190102`, "time", "00:00:00", 0}, {"date", `190102`, "time", "00:00:00", 0}, + } + testModifyColumnTime(c, s.store, tests) +} +func (s *testDBSuite1) TestModifyColumnTime_DateToYear(c *C) { + tests := []testModifyColumnTimeCase{ // date to year {"date", `"2019-01-02"`, "year", "2019", 0}, {"date", `"19-01-02"`, "year", "2019", 0}, @@ -4926,7 +4926,12 @@ func (s *testDBSuite1) TestModifyColumnTime(c *C) { {"date", `"190102"`, "year", "2019", 0}, {"date", `20190102`, "year", "2019", 0}, {"date", `190102`, "year", "2019", 0}, + } + testModifyColumnTime(c, s.store, tests) +} +func (s *testDBSuite1) TestModifyColumnTime_DateToDatetime(c *C) { + tests := []testModifyColumnTimeCase{ // date to datetime {"date", `"2019-01-02"`, "datetime", "2019-01-02 00:00:00", 0}, {"date", `"19-01-02"`, "datetime", "2019-01-02 00:00:00", 0}, @@ -4934,7 +4939,12 @@ func (s *testDBSuite1) TestModifyColumnTime(c *C) { {"date", `"190102"`, "datetime", "2019-01-02 00:00:00", 0}, {"date", `20190102`, "datetime", "2019-01-02 00:00:00", 0}, {"date", `190102`, "datetime", "2019-01-02 00:00:00", 0}, + } + testModifyColumnTime(c, s.store, tests) +} +func (s *testDBSuite1) TestModifyColumnTime_DateToTimestamp(c *C) { + tests := []testModifyColumnTimeCase{ // date to timestamp {"date", `"2019-01-02"`, "timestamp", "2019-01-02 00:00:00", 0}, {"date", `"19-01-02"`, "timestamp", "2019-01-02 00:00:00", 0}, @@ -4942,7 +4952,12 @@ func (s *testDBSuite1) TestModifyColumnTime(c *C) { {"date", `"190102"`, "timestamp", "2019-01-02 00:00:00", 0}, {"date", `20190102`, "timestamp", "2019-01-02 00:00:00", 0}, {"date", `190102`, "timestamp", "2019-01-02 00:00:00", 0}, + } + testModifyColumnTime(c, s.store, tests) +} +func (s *testDBSuite1) TestModifyColumnTime_TimestampToYear(c *C) { + tests := []testModifyColumnTimeCase{ // timestamp to year {"timestamp", `"2006-01-02 15:04:05"`, "year", "2006", 0}, {"timestamp", `"06-01-02 15:04:05"`, "year", "2006", 0}, @@ -4951,7 +4966,12 @@ func (s *testDBSuite1) TestModifyColumnTime(c *C) { {"timestamp", `20060102150405`, "year", "2006", 0}, {"timestamp", `060102150405`, "year", "2006", 0}, {"timestamp", `"2006-01-02 23:59:59.506"`, "year", "2006", 0}, + } + testModifyColumnTime(c, s.store, tests) +} +func (s *testDBSuite1) TestModifyColumnTime_TimestampToTime(c *C) { + tests := []testModifyColumnTimeCase{ // timestamp to time {"timestamp", `"2006-01-02 15:04:05"`, "time", "15:04:05", 0}, {"timestamp", `"06-01-02 15:04:05"`, "time", "15:04:05", 0}, @@ -4960,7 +4980,12 @@ func (s *testDBSuite1) TestModifyColumnTime(c *C) { {"timestamp", `20060102150405`, "time", "15:04:05", 0}, {"timestamp", `060102150405`, "time", "15:04:05", 0}, {"timestamp", `"2006-01-02 23:59:59.506"`, "time", "00:00:00", 0}, + } + testModifyColumnTime(c, s.store, tests) +} +func (s *testDBSuite1) TestModifyColumnTime_TimestampToDate(c *C) { + tests := []testModifyColumnTimeCase{ // timestamp to date {"timestamp", `"2006-01-02 15:04:05"`, "date", "2006-01-02", 0}, {"timestamp", `"06-01-02 15:04:05"`, "date", "2006-01-02", 0}, @@ -4969,7 +4994,12 @@ func (s *testDBSuite1) TestModifyColumnTime(c *C) { {"timestamp", `20060102150405`, "date", "2006-01-02", 0}, {"timestamp", `060102150405`, "date", "2006-01-02", 0}, {"timestamp", `"2006-01-02 23:59:59.506"`, "date", "2006-01-03", 0}, + } + testModifyColumnTime(c, s.store, tests) +} +func (s *testDBSuite1) TestModifyColumnTime_TimestampToDatetime(c *C) { + tests := []testModifyColumnTimeCase{ // timestamp to datetime {"timestamp", `"2006-01-02 15:04:05"`, "datetime", "2006-01-02 15:04:05", 0}, {"timestamp", `"06-01-02 15:04:05"`, "datetime", "2006-01-02 15:04:05", 0}, @@ -4978,7 +5008,12 @@ func (s *testDBSuite1) TestModifyColumnTime(c *C) { {"timestamp", `20060102150405`, "datetime", "2006-01-02 15:04:05", 0}, {"timestamp", `060102150405`, "datetime", "2006-01-02 15:04:05", 0}, {"timestamp", `"2006-01-02 23:59:59.506"`, "datetime", "2006-01-03 00:00:00", 0}, + } + testModifyColumnTime(c, s.store, tests) +} +func (s *testDBSuite1) TestModifyColumnTime_DatetimeToYear(c *C) { + tests := []testModifyColumnTimeCase{ // datetime to year {"datetime", `"2006-01-02 15:04:05"`, "year", "2006", 0}, {"datetime", `"06-01-02 15:04:05"`, "year", "2006", 0}, @@ -4990,7 +5025,12 @@ func (s *testDBSuite1) TestModifyColumnTime(c *C) { // MySQL will get "Data truncation: Out of range value for column 'a' at row 1. {"datetime", `"1000-01-02 23:59:59"`, "year", "", errno.ErrInvalidYear}, {"datetime", `"9999-01-02 23:59:59"`, "year", "", errno.ErrInvalidYear}, + } + testModifyColumnTime(c, s.store, tests) +} +func (s *testDBSuite1) TestModifyColumnTime_DatetimeToTime(c *C) { + tests := []testModifyColumnTimeCase{ // datetime to time {"datetime", `"2006-01-02 15:04:05"`, "time", "15:04:05", 0}, {"datetime", `"06-01-02 15:04:05"`, "time", "15:04:05", 0}, @@ -5001,7 +5041,12 @@ func (s *testDBSuite1) TestModifyColumnTime(c *C) { {"datetime", `"2006-01-02 23:59:59.506"`, "time", "00:00:00", 0}, {"datetime", `"1000-01-02 23:59:59"`, "time", "23:59:59", 0}, {"datetime", `"9999-01-02 23:59:59"`, "time", "23:59:59", 0}, + } + testModifyColumnTime(c, s.store, tests) +} +func (s *testDBSuite1) TestModifyColumnTime_DatetimeToDate(c *C) { + tests := []testModifyColumnTimeCase{ // datetime to date {"datetime", `"2006-01-02 15:04:05"`, "date", "2006-01-02", 0}, {"datetime", `"06-01-02 15:04:05"`, "date", "2006-01-02", 0}, @@ -5012,7 +5057,12 @@ func (s *testDBSuite1) TestModifyColumnTime(c *C) { {"datetime", `"2006-01-02 23:59:59.506"`, "date", "2006-01-03", 0}, {"datetime", `"1000-01-02 23:59:59"`, "date", "1000-01-02", 0}, {"datetime", `"9999-01-02 23:59:59"`, "date", "9999-01-02", 0}, + } + testModifyColumnTime(c, s.store, tests) +} +func (s *testDBSuite1) TestModifyColumnTime_DatetimeToTimestamp(c *C) { + tests := []testModifyColumnTimeCase{ // datetime to timestamp {"datetime", `"2006-01-02 15:04:05"`, "timestamp", "2006-01-02 15:04:05", 0}, {"datetime", `"06-01-02 15:04:05"`, "timestamp", "2006-01-02 15:04:05", 0}, @@ -5023,7 +5073,12 @@ func (s *testDBSuite1) TestModifyColumnTime(c *C) { {"datetime", `"2006-01-02 23:59:59.506"`, "timestamp", "2006-01-03 00:00:00", 0}, {"datetime", `"1000-01-02 23:59:59"`, "timestamp", "1000-01-02 23:59:59", 0}, {"datetime", `"9999-01-02 23:59:59"`, "timestamp", "9999-01-02 23:59:59", 0}, + } + testModifyColumnTime(c, s.store, tests) +} +func (s *testDBSuite1) TestModifyColumnTime_YearToTime(c *C) { + tests := []testModifyColumnTimeCase{ // year to time // failed cases are not handled by TiDB {"year", `"2019"`, "time", "00:20:19", 0}, @@ -5036,7 +5091,12 @@ func (s *testDBSuite1) TestModifyColumnTime(c *C) { {"year", `69`, "time", "", errno.ErrTruncatedWrongValue}, {"year", `70`, "time", "", errno.ErrTruncatedWrongValue}, {"year", `99`, "time", "", errno.ErrTruncatedWrongValue}, + } + testModifyColumnTime(c, s.store, tests) +} +func (s *testDBSuite1) TestModifyColumnTime_YearToDate(c *C) { + tests := []testModifyColumnTimeCase{ // year to date {"year", `"2019"`, "date", "", errno.ErrTruncatedWrongValue}, {"year", `2019`, "date", "", errno.ErrTruncatedWrongValue}, @@ -5049,7 +5109,12 @@ func (s *testDBSuite1) TestModifyColumnTime(c *C) { {"year", `69`, "date", "", errno.ErrTruncatedWrongValue}, {"year", `70`, "date", "", errno.ErrTruncatedWrongValue}, {"year", `99`, "date", "", errno.ErrTruncatedWrongValue}, + } + testModifyColumnTime(c, s.store, tests) +} +func (s *testDBSuite1) TestModifyColumnTime_YearToDatetime(c *C) { + tests := []testModifyColumnTimeCase{ // year to datetime {"year", `"2019"`, "datetime", "", errno.ErrTruncatedWrongValue}, {"year", `2019`, "datetime", "", errno.ErrTruncatedWrongValue}, @@ -5062,7 +5127,12 @@ func (s *testDBSuite1) TestModifyColumnTime(c *C) { {"year", `69`, "datetime", "", errno.ErrTruncatedWrongValue}, {"year", `70`, "datetime", "", errno.ErrTruncatedWrongValue}, {"year", `99`, "datetime", "", errno.ErrTruncatedWrongValue}, + } + testModifyColumnTime(c, s.store, tests) +} +func (s *testDBSuite1) TestModifyColumnTime_YearToTimestamp(c *C) { + tests := []testModifyColumnTimeCase{ // year to timestamp {"year", `"2019"`, "timestamp", "", errno.ErrTruncatedWrongValue}, {"year", `2019`, "timestamp", "", errno.ErrTruncatedWrongValue}, @@ -5076,6 +5146,34 @@ func (s *testDBSuite1) TestModifyColumnTime(c *C) { {"year", `70`, "timestamp", "", errno.ErrTruncatedWrongValue}, {"year", `99`, "timestamp", "", errno.ErrTruncatedWrongValue}, } + testModifyColumnTime(c, s.store, tests) +} + +type testModifyColumnTimeCase struct { + from string + value string + to string + expect string + err uint16 +} + +func testModifyColumnTime(c *C, store kv.Storage, tests []testModifyColumnTimeCase) { + limit := variable.GetDDLErrorCountLimit() + variable.SetDDLErrorCountLimit(3) + + tk := testkit.NewTestKit(c, store) + tk.MustExec("use test_db") + enableChangeColumnType := tk.Se.GetSessionVars().EnableChangeColumnType + tk.Se.GetSessionVars().EnableChangeColumnType = true + + // Set time zone to UTC. + originalTz := tk.Se.GetSessionVars().TimeZone + tk.Se.GetSessionVars().TimeZone = time.UTC + defer func() { + variable.SetDDLErrorCountLimit(limit) + tk.Se.GetSessionVars().EnableChangeColumnType = enableChangeColumnType + tk.Se.GetSessionVars().TimeZone = originalTz + }() for _, t := range tests { tk.MustExec("drop table if exists t_mc") From e26b8470c7f8b2b6e319e5d2c7b72e6293992720 Mon Sep 17 00:00:00 2001 From: Ryan Leung Date: Tue, 11 May 2021 13:11:38 +0800 Subject: [PATCH 56/67] variable: use the defined variable (#24509) --- sessionctx/variable/noop.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sessionctx/variable/noop.go b/sessionctx/variable/noop.go index a3382fc1159d1..05553c6f7eec1 100644 --- a/sessionctx/variable/noop.go +++ b/sessionctx/variable/noop.go @@ -156,7 +156,7 @@ var noopSysVars = []*SysVar{ {Scope: ScopeNone, Name: "myisam_mmap_size", Value: "18446744073709551615"}, {Scope: ScopeNone, Name: "innodb_buffer_pool_instances", Value: "8"}, {Scope: ScopeGlobal | ScopeSession, Name: "max_length_for_sort_data", Value: "1024", IsHintUpdatable: true}, - {Scope: ScopeNone, Name: "character_set_system", Value: "utf8"}, + {Scope: ScopeNone, Name: CharacterSetSystem, Value: "utf8"}, {Scope: ScopeGlobal, Name: InnodbOptimizeFullTextOnly, Value: "0"}, {Scope: ScopeNone, Name: "character_sets_dir", Value: "/usr/local/mysql-5.6.25-osx10.8-x86_64/share/charsets/"}, {Scope: ScopeGlobal | ScopeSession, Name: QueryCacheType, Value: Off, Type: TypeEnum, PossibleValues: []string{Off, On, "DEMAND"}}, From c8bc701170165e82cc7893ab68523f9a2059a7e6 Mon Sep 17 00:00:00 2001 From: Morgan Tocker Date: Mon, 10 May 2021 23:23:38 -0600 Subject: [PATCH 57/67] *: misc cleanups to variable package (#24507) --- ddl/util/util.go | 4 +- executor/aggfuncs/builder.go | 2 +- executor/analyze.go | 2 +- executor/checksum.go | 2 +- executor/infoschema_reader.go | 2 +- executor/set_test.go | 2 +- executor/show.go | 2 +- expression/aggregation/descriptor.go | 2 +- expression/helper.go | 2 +- planner/core/expression_rewriter.go | 36 ++++++++------ sessionctx/variable/noop.go | 2 +- sessionctx/variable/session.go | 31 +----------- sessionctx/variable/sysvar.go | 23 +++++++-- sessionctx/variable/sysvar_test.go | 54 ++++++++++++--------- sessionctx/variable/varsutil.go | 29 +++-------- sessionctx/variable/varsutil_test.go | 72 ++++++++++++++-------------- 16 files changed, 127 insertions(+), 140 deletions(-) diff --git a/ddl/util/util.go b/ddl/util/util.go index 0c3ec2608b9eb..0e5eb8fe2051d 100644 --- a/ddl/util/util.go +++ b/ddl/util/util.go @@ -186,7 +186,9 @@ func LoadGlobalVars(ctx sessionctx.Context, varNames []string) error { for _, row := range rows { varName := row.GetString(0) varValue := row.GetString(1) - variable.SetLocalSystemVar(varName, varValue) + if err = ctx.GetSessionVars().SetSystemVar(varName, varValue); err != nil { + return err + } } } return nil diff --git a/executor/aggfuncs/builder.go b/executor/aggfuncs/builder.go index 666743e4c137c..c914ea4838f2d 100644 --- a/executor/aggfuncs/builder.go +++ b/executor/aggfuncs/builder.go @@ -479,7 +479,7 @@ func buildGroupConcat(ctx sessionctx.Context, aggFuncDesc *aggregation.AggFuncDe panic(fmt.Sprintf("Error happened when buildGroupConcat: %s", err.Error())) } var s string - s, err = variable.GetSessionSystemVar(ctx.GetSessionVars(), variable.GroupConcatMaxLen) + s, err = variable.GetSessionOrGlobalSystemVar(ctx.GetSessionVars(), variable.GroupConcatMaxLen) if err != nil { panic(fmt.Sprintf("Error happened when buildGroupConcat: no system variable named '%s'", variable.GroupConcatMaxLen)) } diff --git a/executor/analyze.go b/executor/analyze.go index 8345382939865..9cf9c75b1261c 100644 --- a/executor/analyze.go +++ b/executor/analyze.go @@ -211,7 +211,7 @@ func (e *AnalyzeExec) Next(ctx context.Context, req *chunk.Chunk) error { func getBuildStatsConcurrency(ctx sessionctx.Context) (int, error) { sessionVars := ctx.GetSessionVars() - concurrency, err := variable.GetSessionSystemVar(sessionVars, variable.TiDBBuildStatsConcurrency) + concurrency, err := variable.GetSessionOrGlobalSystemVar(sessionVars, variable.TiDBBuildStatsConcurrency) if err != nil { return 0, err } diff --git a/executor/checksum.go b/executor/checksum.go index 419c76042ad15..63f622d2f8140 100644 --- a/executor/checksum.go +++ b/executor/checksum.go @@ -269,7 +269,7 @@ func (c *checksumContext) HandleResponse(update *tipb.ChecksumResponse) { func getChecksumTableConcurrency(ctx sessionctx.Context) (int, error) { sessionVars := ctx.GetSessionVars() - concurrency, err := variable.GetSessionSystemVar(sessionVars, variable.TiDBChecksumTableConcurrency) + concurrency, err := variable.GetSessionOrGlobalSystemVar(sessionVars, variable.TiDBChecksumTableConcurrency) if err != nil { return 0, err } diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index a39992799b27a..0ec0c48885ecf 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -1675,7 +1675,7 @@ func (e *memtableRetriever) setDataFromSessionVar(ctx sessionctx.Context) error sessionVars := ctx.GetSessionVars() for _, v := range variable.GetSysVars() { var value string - value, err = variable.GetSessionSystemVar(sessionVars, v.Name) + value, err = variable.GetSessionOrGlobalSystemVar(sessionVars, v.Name) if err != nil { return err } diff --git a/executor/set_test.go b/executor/set_test.go index ec97914cc12a5..4f24ca83519de 100644 --- a/executor/set_test.go +++ b/executor/set_test.go @@ -570,7 +570,7 @@ func (s *testSuite5) TestSetCharset(c *C) { check := func(args ...string) { for i, v := range characterSetVariables { - sVar, err := variable.GetSessionSystemVar(sessionVars, v) + sVar, err := variable.GetSessionOrGlobalSystemVar(sessionVars, v) c.Assert(err, IsNil) c.Assert(sVar, Equals, args[i], Commentf("%d: %s", i, characterSetVariables[i])) } diff --git a/executor/show.go b/executor/show.go index 725c0e0e28ec2..c5df3bb290268 100644 --- a/executor/show.go +++ b/executor/show.go @@ -693,7 +693,7 @@ func (e *ShowExec) fetchShowVariables() (err error) { if v.Hidden { continue } - value, err = variable.GetSessionSystemVar(sessionVars, v.Name) + value, err = variable.GetSessionOrGlobalSystemVar(sessionVars, v.Name) if err != nil { return errors.Trace(err) } diff --git a/expression/aggregation/descriptor.go b/expression/aggregation/descriptor.go index af16d26b1f81a..4415b0688ce09 100644 --- a/expression/aggregation/descriptor.go +++ b/expression/aggregation/descriptor.go @@ -210,7 +210,7 @@ func (a *AggFuncDesc) GetAggFunc(ctx sessionctx.Context) Aggregation { var s string var err error var maxLen uint64 - s, err = variable.GetSessionSystemVar(ctx.GetSessionVars(), variable.GroupConcatMaxLen) + s, err = variable.GetSessionOrGlobalSystemVar(ctx.GetSessionVars(), variable.GroupConcatMaxLen) if err != nil { panic(fmt.Sprintf("Error happened when GetAggFunc: no system variable named '%s'", variable.GroupConcatMaxLen)) } diff --git a/expression/helper.go b/expression/helper.go index 1e89c86e705f2..c5f91dbd090b5 100644 --- a/expression/helper.go +++ b/expression/helper.go @@ -139,7 +139,7 @@ func getStmtTimestamp(ctx sessionctx.Context) (time.Time, error) { } sessionVars := ctx.GetSessionVars() - timestampStr, err := variable.GetSessionSystemVar(sessionVars, "timestamp") + timestampStr, err := variable.GetSessionOrGlobalSystemVar(sessionVars, "timestamp") if err != nil { return now, err } diff --git a/planner/core/expression_rewriter.go b/planner/core/expression_rewriter.go index 25ba4a21460fe..8b52318a260a1 100644 --- a/planner/core/expression_rewriter.go +++ b/planner/core/expression_rewriter.go @@ -1215,25 +1215,33 @@ func (er *expressionRewriter) rewriteVariable(v *ast.VariableExpr) { er.ctxStackAppend(f, types.EmptyName) return } - var val string - var err error - if v.ExplicitScope { - err = variable.ValidateGetSystemVar(name, v.IsGlobal) - if err != nil { - er.err = err - return - } - } sysVar := variable.GetSysVar(name) if sysVar == nil { er.err = variable.ErrUnknownSystemVar.GenWithStackByArgs(name) return } - // Variable is @@gobal.variable_name or variable is only global scope variable. - if v.IsGlobal || sysVar.Scope == variable.ScopeGlobal { + if v.ExplicitScope && !sysVar.HasNoneScope() { + if v.IsGlobal && !sysVar.HasGlobalScope() { + er.err = variable.ErrIncorrectScope.GenWithStackByArgs(name, "GLOBAL") + return + } + if !v.IsGlobal && !sysVar.HasSessionScope() { + er.err = variable.ErrIncorrectScope.GenWithStackByArgs(name, "SESSION") + return + } + } + var val string + var err error + if sysVar.HasNoneScope() { + val = sysVar.Value + } else if v.IsGlobal || !sysVar.HasSessionScope() { + // The condition "|| !sysVar.HasSessionScope()" is a workaround + // for issue https://github.com/pingcap/tidb/issues/24368 + // Where global values are cached incorrectly. When this issue closes, + // the if statement here can be simplified. val, err = variable.GetGlobalSystemVar(sessionVars, name) } else { - val, err = variable.GetSessionSystemVar(sessionVars, name) + val, err = variable.GetSessionOrGlobalSystemVar(sessionVars, name) } if err != nil { er.err = err @@ -1241,8 +1249,8 @@ func (er *expressionRewriter) rewriteVariable(v *ast.VariableExpr) { } nativeVal, nativeType, nativeFlag := sysVar.GetNativeValType(val) e := expression.DatumToConstant(nativeVal, nativeType, nativeFlag) - e.GetType().Charset, _ = er.sctx.GetSessionVars().GetSystemVar(variable.CharacterSetConnection) - e.GetType().Collate, _ = er.sctx.GetSessionVars().GetSystemVar(variable.CollationConnection) + e.GetType().Charset, _ = sessionVars.GetSystemVar(variable.CharacterSetConnection) + e.GetType().Collate, _ = sessionVars.GetSystemVar(variable.CollationConnection) er.ctxStackAppend(e, types.EmptyName) } diff --git a/sessionctx/variable/noop.go b/sessionctx/variable/noop.go index 05553c6f7eec1..c510d9c73ce3a 100644 --- a/sessionctx/variable/noop.go +++ b/sessionctx/variable/noop.go @@ -38,7 +38,7 @@ var noopSysVars = []*SysVar{ {Scope: ScopeGlobal, Name: SuperReadOnly, Value: Off, Type: TypeBool, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { return checkReadOnly(vars, normalizedValue, originalValue, scope, false) }}, - {Scope: ScopeGlobal, Name: serverReadOnly, Value: Off, Type: TypeBool, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + {Scope: ScopeGlobal, Name: ReadOnly, Value: Off, Type: TypeBool, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { return checkReadOnly(vars, normalizedValue, originalValue, scope, false) }}, {Scope: ScopeGlobal, Name: ConnectTimeout, Value: "10", Type: TypeUnsigned, MinValue: 2, MaxValue: secondsPerYear, AutoConvertOutOfRange: true}, diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 7db9de383ba55..618120b5da6e6 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -1445,20 +1445,6 @@ func (s *SessionVars) LazyCheckKeyNotExists() bool { return s.PresumeKeyNotExists || (s.TxnCtx.IsPessimistic && !s.StmtCtx.DupKeyAsWarning) } -// SetLocalSystemVar sets values of the local variables which in "server" scope. -func SetLocalSystemVar(name string, val string) { - switch name { - case TiDBDDLReorgWorkerCount: - SetDDLReorgWorkerCounter(int32(tidbOptPositiveInt32(val, DefTiDBDDLReorgWorkerCount))) - case TiDBDDLReorgBatchSize: - SetDDLReorgBatchSize(int32(tidbOptPositiveInt32(val, DefTiDBDDLReorgBatchSize))) - case TiDBDDLErrorCountLimit: - SetDDLErrorCountLimit(tidbOptInt64(val, DefTiDBDDLErrorCountLimit)) - case TiDBRowFormatVersion: - SetDDLReorgRowFormat(tidbOptInt64(val, DefTiDBRowFormatV2)) - } -} - // special session variables. const ( SQLModeVar = "sql_mode" @@ -1469,22 +1455,7 @@ const ( TransactionIsolation = "transaction_isolation" TxnIsolationOneShot = "tx_isolation_one_shot" MaxExecutionTime = "max_execution_time" -) - -// these variables are useless for TiDB, but still need to validate their values for some compatible issues. -// TODO: some more variables need to be added here. -const ( - serverReadOnly = "read_only" -) - -var ( - // TxIsolationNames are the valid values of the variable "tx_isolation" or "transaction_isolation". - TxIsolationNames = map[string]struct{}{ - "READ-UNCOMMITTED": {}, - "READ-COMMITTED": {}, - "REPEATABLE-READ": {}, - "SERIALIZABLE": {}, - } + ReadOnly = "read_only" ) // TableDelta stands for the changed count for one table or partition. diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index c17238c9ae9c5..9a01c19470722 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -171,6 +171,11 @@ func (sv *SysVar) SetGlobalFromHook(s *SessionVars, val string, skipAliases bool return nil } +// HasNoneScope returns true if the scope for the sysVar is None. +func (sv *SysVar) HasNoneScope() bool { + return sv.Scope == ScopeNone +} + // HasSessionScope returns true if the scope for the sysVar includes session. func (sv *SysVar) HasSessionScope() bool { return sv.Scope&ScopeSession != 0 @@ -1060,6 +1065,7 @@ var defaultSysVars = []*SysVar{ } else if formatVersion == DefTiDBRowFormatV2 { s.RowEncoder.Enable = true } + SetDDLReorgRowFormat(tidbOptInt64(val, DefTiDBRowFormatV2)) return nil }}, {Scope: ScopeSession, Name: TiDBOptimizerSelectivityLevel, Value: strconv.Itoa(DefTiDBOptimizerSelectivityLevel), Type: TypeUnsigned, MinValue: 1, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { @@ -1138,9 +1144,18 @@ var defaultSysVars = []*SysVar{ return nil }}, {Scope: ScopeSession, Name: TiDBConfig, Value: "", ReadOnly: true}, - {Scope: ScopeGlobal, Name: TiDBDDLReorgWorkerCount, Value: strconv.Itoa(DefTiDBDDLReorgWorkerCount), Type: TypeUnsigned, MinValue: 1, MaxValue: uint64(maxDDLReorgWorkerCount)}, - {Scope: ScopeGlobal, Name: TiDBDDLReorgBatchSize, Value: strconv.Itoa(DefTiDBDDLReorgBatchSize), Type: TypeUnsigned, MinValue: int64(MinDDLReorgBatchSize), MaxValue: uint64(MaxDDLReorgBatchSize), AutoConvertOutOfRange: true}, - {Scope: ScopeGlobal, Name: TiDBDDLErrorCountLimit, Value: strconv.Itoa(DefTiDBDDLErrorCountLimit), Type: TypeUnsigned, MinValue: 0, MaxValue: uint64(math.MaxInt64), AutoConvertOutOfRange: true}, + {Scope: ScopeGlobal, Name: TiDBDDLReorgWorkerCount, Value: strconv.Itoa(DefTiDBDDLReorgWorkerCount), Type: TypeUnsigned, MinValue: 1, MaxValue: uint64(maxDDLReorgWorkerCount), SetSession: func(s *SessionVars, val string) error { + SetDDLReorgWorkerCounter(int32(tidbOptPositiveInt32(val, DefTiDBDDLReorgWorkerCount))) + return nil + }}, + {Scope: ScopeGlobal, Name: TiDBDDLReorgBatchSize, Value: strconv.Itoa(DefTiDBDDLReorgBatchSize), Type: TypeUnsigned, MinValue: int64(MinDDLReorgBatchSize), MaxValue: uint64(MaxDDLReorgBatchSize), AutoConvertOutOfRange: true, SetSession: func(s *SessionVars, val string) error { + SetDDLReorgBatchSize(int32(tidbOptPositiveInt32(val, DefTiDBDDLReorgBatchSize))) + return nil + }}, + {Scope: ScopeGlobal, Name: TiDBDDLErrorCountLimit, Value: strconv.Itoa(DefTiDBDDLErrorCountLimit), Type: TypeUnsigned, MinValue: 0, MaxValue: uint64(math.MaxInt64), AutoConvertOutOfRange: true, SetSession: func(s *SessionVars, val string) error { + SetDDLErrorCountLimit(tidbOptInt64(val, DefTiDBDDLErrorCountLimit)) + return nil + }}, {Scope: ScopeSession, Name: TiDBDDLReorgPriority, Value: "PRIORITY_LOW", SetSession: func(s *SessionVars, val string) error { s.setDDLReorgPriority(val) return nil @@ -1210,7 +1225,7 @@ var defaultSysVars = []*SysVar{ // To prevent this strange position, prevent setting to OFF when any of these sysVars are ON of the same scope. if normalizedValue == Off { - for _, potentialIncompatibleSysVar := range []string{TxReadOnly, TransactionReadOnly, OfflineMode, SuperReadOnly, serverReadOnly} { + for _, potentialIncompatibleSysVar := range []string{TxReadOnly, TransactionReadOnly, OfflineMode, SuperReadOnly, ReadOnly} { val, _ := vars.GetSystemVar(potentialIncompatibleSysVar) // session scope if scope == ScopeGlobal { // global scope var err error diff --git a/sessionctx/variable/sysvar_test.go b/sessionctx/variable/sysvar_test.go index cc765854a5c0b..71979a57b7eef 100644 --- a/sessionctx/variable/sysvar_test.go +++ b/sessionctx/variable/sysvar_test.go @@ -223,14 +223,22 @@ func (*testSysVarSuite) TestScope(c *C) { sv := SysVar{Scope: ScopeGlobal | ScopeSession, Name: "mynewsysvar", Value: On, Type: TypeEnum, PossibleValues: []string{"OFF", "ON", "AUTO"}} c.Assert(sv.HasSessionScope(), IsTrue) c.Assert(sv.HasGlobalScope(), IsTrue) + c.Assert(sv.HasNoneScope(), IsFalse) sv = SysVar{Scope: ScopeGlobal, Name: "mynewsysvar", Value: On, Type: TypeEnum, PossibleValues: []string{"OFF", "ON", "AUTO"}} c.Assert(sv.HasSessionScope(), IsFalse) c.Assert(sv.HasGlobalScope(), IsTrue) + c.Assert(sv.HasNoneScope(), IsFalse) + + sv = SysVar{Scope: ScopeSession, Name: "mynewsysvar", Value: On, Type: TypeEnum, PossibleValues: []string{"OFF", "ON", "AUTO"}} + c.Assert(sv.HasSessionScope(), IsTrue) + c.Assert(sv.HasGlobalScope(), IsFalse) + c.Assert(sv.HasNoneScope(), IsFalse) sv = SysVar{Scope: ScopeNone, Name: "mynewsysvar", Value: On, Type: TypeEnum, PossibleValues: []string{"OFF", "ON", "AUTO"}} c.Assert(sv.HasSessionScope(), IsFalse) c.Assert(sv.HasGlobalScope(), IsFalse) + c.Assert(sv.HasNoneScope(), IsTrue) } func (*testSysVarSuite) TestBuiltInCase(c *C) { @@ -444,31 +452,31 @@ func (*testSysVarSuite) TestGetScopeNoneSystemVar(c *C) { } func (*testSysVarSuite) TestInstanceScopedVars(c *C) { - // This tests instance scoped variables through GetSessionSystemVar(). + // This tests instance scoped variables through GetSessionOrGlobalSystemVar(). // Eventually these should be changed to use getters so that the switch // statement in GetSessionOnlySysVars can be removed. vars := NewSessionVars() - val, err := GetSessionSystemVar(vars, TiDBCurrentTS) + val, err := GetSessionOrGlobalSystemVar(vars, TiDBCurrentTS) c.Assert(err, IsNil) c.Assert(val, Equals, fmt.Sprintf("%d", vars.TxnCtx.StartTS)) - val, err = GetSessionSystemVar(vars, TiDBLastTxnInfo) + val, err = GetSessionOrGlobalSystemVar(vars, TiDBLastTxnInfo) c.Assert(err, IsNil) c.Assert(val, Equals, vars.LastTxnInfo) - val, err = GetSessionSystemVar(vars, TiDBLastQueryInfo) + val, err = GetSessionOrGlobalSystemVar(vars, TiDBLastQueryInfo) c.Assert(err, IsNil) info, err := json.Marshal(vars.LastQueryInfo) c.Assert(err, IsNil) c.Assert(val, Equals, string(info)) - val, err = GetSessionSystemVar(vars, TiDBGeneralLog) + val, err = GetSessionOrGlobalSystemVar(vars, TiDBGeneralLog) c.Assert(err, IsNil) c.Assert(val, Equals, BoolToOnOff(ProcessGeneralLog.Load())) - val, err = GetSessionSystemVar(vars, TiDBPProfSQLCPU) + val, err = GetSessionOrGlobalSystemVar(vars, TiDBPProfSQLCPU) c.Assert(err, IsNil) expected := "0" if EnablePProfSQLCPU.Load() { @@ -476,74 +484,74 @@ func (*testSysVarSuite) TestInstanceScopedVars(c *C) { } c.Assert(val, Equals, expected) - val, err = GetSessionSystemVar(vars, TiDBExpensiveQueryTimeThreshold) + val, err = GetSessionOrGlobalSystemVar(vars, TiDBExpensiveQueryTimeThreshold) c.Assert(err, IsNil) c.Assert(val, Equals, fmt.Sprintf("%d", atomic.LoadUint64(&ExpensiveQueryTimeThreshold))) - val, err = GetSessionSystemVar(vars, TiDBMemoryUsageAlarmRatio) + val, err = GetSessionOrGlobalSystemVar(vars, TiDBMemoryUsageAlarmRatio) c.Assert(err, IsNil) c.Assert(val, Equals, fmt.Sprintf("%g", MemoryUsageAlarmRatio.Load())) - val, err = GetSessionSystemVar(vars, TiDBConfig) + val, err = GetSessionOrGlobalSystemVar(vars, TiDBConfig) c.Assert(err, IsNil) conf := config.GetGlobalConfig() j, err := json.MarshalIndent(conf, "", "\t") c.Assert(err, IsNil) c.Assert(val, Equals, config.HideConfig(string(j))) - val, err = GetSessionSystemVar(vars, TiDBForcePriority) + val, err = GetSessionOrGlobalSystemVar(vars, TiDBForcePriority) c.Assert(err, IsNil) c.Assert(val, Equals, mysql.Priority2Str[mysql.PriorityEnum(atomic.LoadInt32(&ForcePriority))]) - val, err = GetSessionSystemVar(vars, TiDBDDLSlowOprThreshold) + val, err = GetSessionOrGlobalSystemVar(vars, TiDBDDLSlowOprThreshold) c.Assert(err, IsNil) c.Assert(val, Equals, strconv.FormatUint(uint64(atomic.LoadUint32(&DDLSlowOprThreshold)), 10)) - val, err = GetSessionSystemVar(vars, PluginDir) + val, err = GetSessionOrGlobalSystemVar(vars, PluginDir) c.Assert(err, IsNil) c.Assert(val, Equals, config.GetGlobalConfig().Plugin.Dir) - val, err = GetSessionSystemVar(vars, PluginLoad) + val, err = GetSessionOrGlobalSystemVar(vars, PluginLoad) c.Assert(err, IsNil) c.Assert(val, Equals, config.GetGlobalConfig().Plugin.Load) - val, err = GetSessionSystemVar(vars, TiDBSlowLogThreshold) + val, err = GetSessionOrGlobalSystemVar(vars, TiDBSlowLogThreshold) c.Assert(err, IsNil) c.Assert(val, Equals, strconv.FormatUint(atomic.LoadUint64(&config.GetGlobalConfig().Log.SlowThreshold), 10)) - val, err = GetSessionSystemVar(vars, TiDBRecordPlanInSlowLog) + val, err = GetSessionOrGlobalSystemVar(vars, TiDBRecordPlanInSlowLog) c.Assert(err, IsNil) c.Assert(val, Equals, strconv.FormatUint(uint64(atomic.LoadUint32(&config.GetGlobalConfig().Log.RecordPlanInSlowLog)), 10)) - val, err = GetSessionSystemVar(vars, TiDBEnableSlowLog) + val, err = GetSessionOrGlobalSystemVar(vars, TiDBEnableSlowLog) c.Assert(err, IsNil) c.Assert(val, Equals, BoolToOnOff(config.GetGlobalConfig().Log.EnableSlowLog)) - val, err = GetSessionSystemVar(vars, TiDBQueryLogMaxLen) + val, err = GetSessionOrGlobalSystemVar(vars, TiDBQueryLogMaxLen) c.Assert(err, IsNil) c.Assert(val, Equals, strconv.FormatUint(atomic.LoadUint64(&config.GetGlobalConfig().Log.QueryLogMaxLen), 10)) - val, err = GetSessionSystemVar(vars, TiDBCheckMb4ValueInUTF8) + val, err = GetSessionOrGlobalSystemVar(vars, TiDBCheckMb4ValueInUTF8) c.Assert(err, IsNil) c.Assert(val, Equals, BoolToOnOff(config.GetGlobalConfig().CheckMb4ValueInUTF8)) - val, err = GetSessionSystemVar(vars, TiDBCapturePlanBaseline) + val, err = GetSessionOrGlobalSystemVar(vars, TiDBCapturePlanBaseline) c.Assert(err, IsNil) c.Assert(val, Equals, CapturePlanBaseline.GetVal()) - val, err = GetSessionSystemVar(vars, TiDBFoundInPlanCache) + val, err = GetSessionOrGlobalSystemVar(vars, TiDBFoundInPlanCache) c.Assert(err, IsNil) c.Assert(val, Equals, BoolToOnOff(vars.PrevFoundInPlanCache)) - val, err = GetSessionSystemVar(vars, TiDBFoundInBinding) + val, err = GetSessionOrGlobalSystemVar(vars, TiDBFoundInBinding) c.Assert(err, IsNil) c.Assert(val, Equals, BoolToOnOff(vars.PrevFoundInBinding)) - val, err = GetSessionSystemVar(vars, TiDBEnableCollectExecutionInfo) + val, err = GetSessionOrGlobalSystemVar(vars, TiDBEnableCollectExecutionInfo) c.Assert(err, IsNil) c.Assert(val, Equals, BoolToOnOff(config.GetGlobalConfig().EnableCollectExecutionInfo)) - val, err = GetSessionSystemVar(vars, TiDBTxnScope) + val, err = GetSessionOrGlobalSystemVar(vars, TiDBTxnScope) c.Assert(err, IsNil) c.Assert(val, Equals, vars.TxnScope.GetVarValue()) } diff --git a/sessionctx/variable/varsutil.go b/sessionctx/variable/varsutil.go index d43a45b1cdbc7..809420ca8ee6c 100644 --- a/sessionctx/variable/varsutil.go +++ b/sessionctx/variable/varsutil.go @@ -161,10 +161,9 @@ func checkIsolationLevel(vars *SessionVars, normalizedValue string, originalValu return normalizedValue, nil } -// GetSessionSystemVar gets a system variable. -// If it is a session only variable, use the default value defined in code. -// Returns error if there is no such variable. -func GetSessionSystemVar(s *SessionVars, key string) (string, error) { +// GetSessionOrGlobalSystemVar gets a system variable of session or global scope. +// It also respects TIDB's special "instance" scope in GetSessionOnlySysVars. +func GetSessionOrGlobalSystemVar(s *SessionVars, key string) (string, error) { key = strings.ToLower(key) gVal, ok, err := GetSessionOnlySysVars(s, key) if err != nil || ok { @@ -174,6 +173,9 @@ func GetSessionSystemVar(s *SessionVars, key string) (string, error) { if err != nil { return "", err } + // This cache results in incorrect behavior since changes to global + // variables will not be picked up. It should be removed once + // https://github.com/pingcap/tidb/issues/24368 is closed. s.systems[key] = gVal return gVal, nil } @@ -313,25 +315,6 @@ func SetStmtVar(vars *SessionVars, name string, value string) error { return vars.SetStmtVar(name, sVal) } -// ValidateGetSystemVar checks if system variable exists and validates its scope when get system variable. -func ValidateGetSystemVar(name string, isGlobal bool) error { - sysVar := GetSysVar(name) - if sysVar == nil { - return ErrUnknownSystemVar.GenWithStackByArgs(name) - } - switch sysVar.Scope { - case ScopeGlobal: - if !isGlobal { - return ErrIncorrectScope.GenWithStackByArgs(name, "GLOBAL") - } - case ScopeSession: - if isGlobal { - return ErrIncorrectScope.GenWithStackByArgs(name, "SESSION") - } - } - return nil -} - const ( // initChunkSizeUpperBound indicates upper bound value of tidb_init_chunk_size. initChunkSizeUpperBound = 32 diff --git a/sessionctx/variable/varsutil_test.go b/sessionctx/variable/varsutil_test.go index 5757eaeeac403..a96897140b9b4 100644 --- a/sessionctx/variable/varsutil_test.go +++ b/sessionctx/variable/varsutil_test.go @@ -124,7 +124,7 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) { err := SetSessionSystemVar(v, "autocommit", "1") c.Assert(err, IsNil) - val, err := GetSessionSystemVar(v, "autocommit") + val, err := GetSessionOrGlobalSystemVar(v, "autocommit") c.Assert(err, IsNil) c.Assert(val, Equals, "ON") c.Assert(SetSessionSystemVar(v, "autocommit", ""), NotNil) @@ -132,20 +132,20 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) { // 0 converts to OFF err = SetSessionSystemVar(v, "foreign_key_checks", "0") c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, "foreign_key_checks") + val, err = GetSessionOrGlobalSystemVar(v, "foreign_key_checks") c.Assert(err, IsNil) c.Assert(val, Equals, "OFF") // 1/ON is not supported (generates a warning and sets to OFF) err = SetSessionSystemVar(v, "foreign_key_checks", "1") c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, "foreign_key_checks") + val, err = GetSessionOrGlobalSystemVar(v, "foreign_key_checks") c.Assert(err, IsNil) c.Assert(val, Equals, "OFF") err = SetSessionSystemVar(v, "sql_mode", "strict_trans_tables") c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, "sql_mode") + val, err = GetSessionOrGlobalSystemVar(v, "sql_mode") c.Assert(err, IsNil) c.Assert(val, Equals, "STRICT_TRANS_TABLES") c.Assert(v.StrictSQLMode, IsTrue) @@ -253,7 +253,7 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) { // Test case for TiDBConfig session variable. err = SetSessionSystemVar(v, TiDBConfig, "abc") c.Assert(terror.ErrorEqual(err, ErrIncorrectScope), IsTrue) - val, err = GetSessionSystemVar(v, TiDBConfig) + val, err = GetSessionOrGlobalSystemVar(v, TiDBConfig) c.Assert(err, IsNil) bVal, err := json.MarshalIndent(config.GetGlobalConfig(), "", "\t") c.Assert(err, IsNil) @@ -261,13 +261,13 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) { err = SetSessionSystemVar(v, TiDBEnableStreaming, "1") c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, TiDBEnableStreaming) + val, err = GetSessionOrGlobalSystemVar(v, TiDBEnableStreaming) c.Assert(err, IsNil) c.Assert(val, Equals, "ON") c.Assert(v.EnableStreaming, Equals, true) err = SetSessionSystemVar(v, TiDBEnableStreaming, "0") c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, TiDBEnableStreaming) + val, err = GetSessionOrGlobalSystemVar(v, TiDBEnableStreaming) c.Assert(err, IsNil) c.Assert(val, Equals, "OFF") c.Assert(v.EnableStreaming, Equals, false) @@ -282,7 +282,7 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) { err = SetSessionSystemVar(v, TiDBRetryLimit, "3") c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, TiDBRetryLimit) + val, err = GetSessionOrGlobalSystemVar(v, TiDBRetryLimit) c.Assert(err, IsNil) c.Assert(val, Equals, "3") c.Assert(v.RetryLimit, Equals, int64(3)) @@ -290,7 +290,7 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) { c.Assert(v.EnableTablePartition, Equals, "") err = SetSessionSystemVar(v, TiDBEnableTablePartition, "on") c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, TiDBEnableTablePartition) + val, err = GetSessionOrGlobalSystemVar(v, TiDBEnableTablePartition) c.Assert(err, IsNil) c.Assert(val, Equals, "ON") c.Assert(v.EnableTablePartition, Equals, "ON") @@ -298,7 +298,7 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) { c.Assert(v.EnableListTablePartition, Equals, false) err = SetSessionSystemVar(v, TiDBEnableListTablePartition, "on") c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, TiDBEnableListTablePartition) + val, err = GetSessionOrGlobalSystemVar(v, TiDBEnableListTablePartition) c.Assert(err, IsNil) c.Assert(val, Equals, "ON") c.Assert(v.EnableListTablePartition, Equals, true) @@ -306,33 +306,33 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) { c.Assert(v.TiDBOptJoinReorderThreshold, Equals, DefTiDBOptJoinReorderThreshold) err = SetSessionSystemVar(v, TiDBOptJoinReorderThreshold, "5") c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, TiDBOptJoinReorderThreshold) + val, err = GetSessionOrGlobalSystemVar(v, TiDBOptJoinReorderThreshold) c.Assert(err, IsNil) c.Assert(val, Equals, "5") c.Assert(v.TiDBOptJoinReorderThreshold, Equals, 5) err = SetSessionSystemVar(v, TiDBCheckMb4ValueInUTF8, "1") c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, TiDBCheckMb4ValueInUTF8) + val, err = GetSessionOrGlobalSystemVar(v, TiDBCheckMb4ValueInUTF8) c.Assert(err, IsNil) c.Assert(val, Equals, "ON") c.Assert(config.GetGlobalConfig().CheckMb4ValueInUTF8, Equals, true) err = SetSessionSystemVar(v, TiDBCheckMb4ValueInUTF8, "0") c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, TiDBCheckMb4ValueInUTF8) + val, err = GetSessionOrGlobalSystemVar(v, TiDBCheckMb4ValueInUTF8) c.Assert(err, IsNil) c.Assert(val, Equals, "OFF") c.Assert(config.GetGlobalConfig().CheckMb4ValueInUTF8, Equals, false) err = SetSessionSystemVar(v, TiDBLowResolutionTSO, "1") c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, TiDBLowResolutionTSO) + val, err = GetSessionOrGlobalSystemVar(v, TiDBLowResolutionTSO) c.Assert(err, IsNil) c.Assert(val, Equals, "ON") c.Assert(v.LowResolutionTSO, Equals, true) err = SetSessionSystemVar(v, TiDBLowResolutionTSO, "0") c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, TiDBLowResolutionTSO) + val, err = GetSessionOrGlobalSystemVar(v, TiDBLowResolutionTSO) c.Assert(err, IsNil) c.Assert(val, Equals, "OFF") c.Assert(v.LowResolutionTSO, Equals, false) @@ -340,7 +340,7 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) { c.Assert(v.CorrelationThreshold, Equals, 0.9) err = SetSessionSystemVar(v, TiDBOptCorrelationThreshold, "0") c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, TiDBOptCorrelationThreshold) + val, err = GetSessionOrGlobalSystemVar(v, TiDBOptCorrelationThreshold) c.Assert(err, IsNil) c.Assert(val, Equals, "0") c.Assert(v.CorrelationThreshold, Equals, float64(0)) @@ -348,7 +348,7 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) { c.Assert(v.CPUFactor, Equals, 3.0) err = SetSessionSystemVar(v, TiDBOptCPUFactor, "5.0") c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, TiDBOptCPUFactor) + val, err = GetSessionOrGlobalSystemVar(v, TiDBOptCPUFactor) c.Assert(err, IsNil) c.Assert(val, Equals, "5.0") c.Assert(v.CPUFactor, Equals, 5.0) @@ -356,7 +356,7 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) { c.Assert(v.CopCPUFactor, Equals, 3.0) err = SetSessionSystemVar(v, TiDBOptCopCPUFactor, "5.0") c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, TiDBOptCopCPUFactor) + val, err = GetSessionOrGlobalSystemVar(v, TiDBOptCopCPUFactor) c.Assert(err, IsNil) c.Assert(val, Equals, "5.0") c.Assert(v.CopCPUFactor, Equals, 5.0) @@ -364,7 +364,7 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) { c.Assert(v.CopTiFlashConcurrencyFactor, Equals, 24.0) err = SetSessionSystemVar(v, TiDBOptTiFlashConcurrencyFactor, "5.0") c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, TiDBOptTiFlashConcurrencyFactor) + val, err = GetSessionOrGlobalSystemVar(v, TiDBOptTiFlashConcurrencyFactor) c.Assert(err, IsNil) c.Assert(val, Equals, "5.0") c.Assert(v.CopCPUFactor, Equals, 5.0) @@ -372,7 +372,7 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) { c.Assert(v.NetworkFactor, Equals, 1.0) err = SetSessionSystemVar(v, TiDBOptNetworkFactor, "3.0") c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, TiDBOptNetworkFactor) + val, err = GetSessionOrGlobalSystemVar(v, TiDBOptNetworkFactor) c.Assert(err, IsNil) c.Assert(val, Equals, "3.0") c.Assert(v.NetworkFactor, Equals, 3.0) @@ -380,7 +380,7 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) { c.Assert(v.ScanFactor, Equals, 1.5) err = SetSessionSystemVar(v, TiDBOptScanFactor, "3.0") c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, TiDBOptScanFactor) + val, err = GetSessionOrGlobalSystemVar(v, TiDBOptScanFactor) c.Assert(err, IsNil) c.Assert(val, Equals, "3.0") c.Assert(v.ScanFactor, Equals, 3.0) @@ -388,7 +388,7 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) { c.Assert(v.DescScanFactor, Equals, 3.0) err = SetSessionSystemVar(v, TiDBOptDescScanFactor, "5.0") c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, TiDBOptDescScanFactor) + val, err = GetSessionOrGlobalSystemVar(v, TiDBOptDescScanFactor) c.Assert(err, IsNil) c.Assert(val, Equals, "5.0") c.Assert(v.DescScanFactor, Equals, 5.0) @@ -396,7 +396,7 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) { c.Assert(v.SeekFactor, Equals, 20.0) err = SetSessionSystemVar(v, TiDBOptSeekFactor, "50.0") c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, TiDBOptSeekFactor) + val, err = GetSessionOrGlobalSystemVar(v, TiDBOptSeekFactor) c.Assert(err, IsNil) c.Assert(val, Equals, "50.0") c.Assert(v.SeekFactor, Equals, 50.0) @@ -404,7 +404,7 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) { c.Assert(v.MemoryFactor, Equals, 0.001) err = SetSessionSystemVar(v, TiDBOptMemoryFactor, "1.0") c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, TiDBOptMemoryFactor) + val, err = GetSessionOrGlobalSystemVar(v, TiDBOptMemoryFactor) c.Assert(err, IsNil) c.Assert(val, Equals, "1.0") c.Assert(v.MemoryFactor, Equals, 1.0) @@ -412,7 +412,7 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) { c.Assert(v.DiskFactor, Equals, 1.5) err = SetSessionSystemVar(v, TiDBOptDiskFactor, "1.1") c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, TiDBOptDiskFactor) + val, err = GetSessionOrGlobalSystemVar(v, TiDBOptDiskFactor) c.Assert(err, IsNil) c.Assert(val, Equals, "1.1") c.Assert(v.DiskFactor, Equals, 1.1) @@ -420,57 +420,57 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) { c.Assert(v.ConcurrencyFactor, Equals, 3.0) err = SetSessionSystemVar(v, TiDBOptConcurrencyFactor, "5.0") c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, TiDBOptConcurrencyFactor) + val, err = GetSessionOrGlobalSystemVar(v, TiDBOptConcurrencyFactor) c.Assert(err, IsNil) c.Assert(val, Equals, "5.0") c.Assert(v.ConcurrencyFactor, Equals, 5.0) err = SetSessionSystemVar(v, TiDBReplicaRead, "follower") c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, TiDBReplicaRead) + val, err = GetSessionOrGlobalSystemVar(v, TiDBReplicaRead) c.Assert(err, IsNil) c.Assert(val, Equals, "follower") c.Assert(v.GetReplicaRead(), Equals, tikvstore.ReplicaReadFollower) err = SetSessionSystemVar(v, TiDBReplicaRead, "leader") c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, TiDBReplicaRead) + val, err = GetSessionOrGlobalSystemVar(v, TiDBReplicaRead) c.Assert(err, IsNil) c.Assert(val, Equals, "leader") c.Assert(v.GetReplicaRead(), Equals, tikvstore.ReplicaReadLeader) err = SetSessionSystemVar(v, TiDBReplicaRead, "leader-and-follower") c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, TiDBReplicaRead) + val, err = GetSessionOrGlobalSystemVar(v, TiDBReplicaRead) c.Assert(err, IsNil) c.Assert(val, Equals, "leader-and-follower") c.Assert(v.GetReplicaRead(), Equals, tikvstore.ReplicaReadMixed) err = SetSessionSystemVar(v, TiDBEnableStmtSummary, "ON") c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, TiDBEnableStmtSummary) + val, err = GetSessionOrGlobalSystemVar(v, TiDBEnableStmtSummary) c.Assert(err, IsNil) c.Assert(val, Equals, "ON") err = SetSessionSystemVar(v, TiDBRedactLog, "ON") c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, TiDBRedactLog) + val, err = GetSessionOrGlobalSystemVar(v, TiDBRedactLog) c.Assert(err, IsNil) c.Assert(val, Equals, "ON") err = SetSessionSystemVar(v, TiDBStmtSummaryRefreshInterval, "10") c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, TiDBStmtSummaryRefreshInterval) + val, err = GetSessionOrGlobalSystemVar(v, TiDBStmtSummaryRefreshInterval) c.Assert(err, IsNil) c.Assert(val, Equals, "10") err = SetSessionSystemVar(v, TiDBStmtSummaryHistorySize, "10") c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, TiDBStmtSummaryHistorySize) + val, err = GetSessionOrGlobalSystemVar(v, TiDBStmtSummaryHistorySize) c.Assert(err, IsNil) c.Assert(val, Equals, "10") err = SetSessionSystemVar(v, TiDBStmtSummaryMaxStmtCount, "10") c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, TiDBStmtSummaryMaxStmtCount) + val, err = GetSessionOrGlobalSystemVar(v, TiDBStmtSummaryMaxStmtCount) c.Assert(err, IsNil) c.Assert(val, Equals, "10") err = SetSessionSystemVar(v, TiDBStmtSummaryMaxStmtCount, "a") @@ -478,7 +478,7 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) { err = SetSessionSystemVar(v, TiDBStmtSummaryMaxSQLLength, "10") c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, TiDBStmtSummaryMaxSQLLength) + val, err = GetSessionOrGlobalSystemVar(v, TiDBStmtSummaryMaxSQLLength) c.Assert(err, IsNil) c.Assert(val, Equals, "10") err = SetSessionSystemVar(v, TiDBStmtSummaryMaxSQLLength, "a") @@ -492,7 +492,7 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) { err = SetSessionSystemVar(v, TiDBEnableChangeColumnType, "ON") c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, TiDBEnableChangeColumnType) + val, err = GetSessionOrGlobalSystemVar(v, TiDBEnableChangeColumnType) c.Assert(err, IsNil) c.Assert(val, Equals, "ON") c.Assert(v.systems[TiDBEnableChangeColumnType], Equals, "ON") From 81ce996c413abf926375e99d1a8bd8ef7696aa84 Mon Sep 17 00:00:00 2001 From: mmyj Date: Tue, 11 May 2021 13:35:38 +0800 Subject: [PATCH 58/67] session: add a test case to cover batch point get for temporary table (#24519) --- session/session_test.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/session/session_test.go b/session/session_test.go index 84442a8a16956..9ed2f9759243b 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -4296,6 +4296,8 @@ func (s *testSessionSuite3) TestGlobalTemporaryTable(c *C) { tk.MustQuery("select c from g_tmp where b = 3").Check(testkit.Rows("3")) // Cover point get. tk.MustQuery("select * from g_tmp where a = 3").Check(testkit.Rows("3 3 3")) + // Cover batch point get. + tk.MustQuery("select * from g_tmp where a in (2,3,4)").Check(testkit.Rows("3 3 3", "4 7 9")) tk.MustExec("commit") // The global temporary table data is discard after the transaction commit. From 5460b5c77f0fd13a56f6bee965508a42d9c36d31 Mon Sep 17 00:00:00 2001 From: JmPotato Date: Tue, 11 May 2021 14:03:38 +0800 Subject: [PATCH 59/67] store, executor: rename resolveTS to safeTS to be consistent with TiKV (#24538) --- executor/stale_txn_test.go | 44 ++++++++++++++--------------- store/tikv/extract_start_ts_test.go | 8 +++--- store/tikv/kv.go | 39 ++++++++++++++----------- store/tikv/txn.go | 8 +++--- 4 files changed, 53 insertions(+), 46 deletions(-) diff --git a/executor/stale_txn_test.go b/executor/stale_txn_test.go index c68b8a5bfa511..ce5202ae58a75 100644 --- a/executor/stale_txn_test.go +++ b/executor/stale_txn_test.go @@ -256,65 +256,65 @@ func (s *testStaleTxnSerialSuite) TestTimeBoundedStalenessTxn(c *C) { defer tk.MustExec(`drop table if exists t`) tk.MustExec(`START TRANSACTION READ ONLY WITH TIMESTAMP BOUND MAX STALENESS '00:00:10'`) testcases := []struct { - name string - sql string - injectResolveTS uint64 - useResolveTS bool + name string + sql string + injectSafeTS uint64 + useSafeTS bool }{ { - name: "max 20 seconds ago, resolveTS 10 secs ago", + name: "max 20 seconds ago, safeTS 10 secs ago", sql: `START TRANSACTION READ ONLY WITH TIMESTAMP BOUND MAX STALENESS '00:00:20'`, - injectResolveTS: func() uint64 { + injectSafeTS: func() uint64 { phy := time.Now().Add(-10*time.Second).Unix() * 1000 return oracle.ComposeTS(phy, 0) }(), - useResolveTS: true, + useSafeTS: true, }, { - name: "max 10 seconds ago, resolveTS 20 secs ago", + name: "max 10 seconds ago, safeTS 20 secs ago", sql: `START TRANSACTION READ ONLY WITH TIMESTAMP BOUND MAX STALENESS '00:00:10'`, - injectResolveTS: func() uint64 { + injectSafeTS: func() uint64 { phy := time.Now().Add(-20*time.Second).Unix() * 1000 return oracle.ComposeTS(phy, 0) }(), - useResolveTS: false, + useSafeTS: false, }, { - name: "max 20 seconds ago, resolveTS 10 secs ago", + name: "max 20 seconds ago, safeTS 10 secs ago", sql: func() string { return fmt.Sprintf(`START TRANSACTION READ ONLY WITH TIMESTAMP BOUND MIN READ TIMESTAMP '%v'`, time.Now().Add(-20*time.Second).Format("2006-01-02 15:04:05")) }(), - injectResolveTS: func() uint64 { + injectSafeTS: func() uint64 { phy := time.Now().Add(-10*time.Second).Unix() * 1000 return oracle.ComposeTS(phy, 0) }(), - useResolveTS: true, + useSafeTS: true, }, { - name: "max 10 seconds ago, resolveTS 20 secs ago", + name: "max 10 seconds ago, safeTS 20 secs ago", sql: func() string { return fmt.Sprintf(`START TRANSACTION READ ONLY WITH TIMESTAMP BOUND MIN READ TIMESTAMP '%v'`, time.Now().Add(-10*time.Second).Format("2006-01-02 15:04:05")) }(), - injectResolveTS: func() uint64 { + injectSafeTS: func() uint64 { phy := time.Now().Add(-20*time.Second).Unix() * 1000 return oracle.ComposeTS(phy, 0) }(), - useResolveTS: false, + useSafeTS: false, }, } for _, testcase := range testcases { c.Log(testcase.name) - c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/injectResolveTS", - fmt.Sprintf("return(%v)", testcase.injectResolveTS)), IsNil) + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/injectSafeTS", + fmt.Sprintf("return(%v)", testcase.injectSafeTS)), IsNil) tk.MustExec(testcase.sql) - if testcase.useResolveTS { - c.Assert(tk.Se.GetSessionVars().TxnCtx.StartTS, Equals, testcase.injectResolveTS) + if testcase.useSafeTS { + c.Assert(tk.Se.GetSessionVars().TxnCtx.StartTS, Equals, testcase.injectSafeTS) } else { - c.Assert(oracle.CompareTS(tk.Se.GetSessionVars().TxnCtx.StartTS, testcase.injectResolveTS), Equals, 1) + c.Assert(oracle.CompareTS(tk.Se.GetSessionVars().TxnCtx.StartTS, testcase.injectSafeTS), Equals, 1) } tk.MustExec("commit") - failpoint.Disable("github.com/pingcap/tidb/store/tikv/injectResolveTS") + failpoint.Disable("github.com/pingcap/tidb/store/tikv/injectSafeTS") } } diff --git a/store/tikv/extract_start_ts_test.go b/store/tikv/extract_start_ts_test.go index 1422e387bfda5..a108a0f7e41cb 100644 --- a/store/tikv/extract_start_ts_test.go +++ b/store/tikv/extract_start_ts_test.go @@ -56,8 +56,8 @@ func (s *extractStartTsSuite) SetUpTest(c *C) { Value: "Some Random Label", }}, } - store.resolveTSMu.resolveTS[2] = 102 - store.resolveTSMu.resolveTS[3] = 101 + store.setSafeTS(2, 102) + store.setSafeTS(3, 101) s.store = store } @@ -105,8 +105,8 @@ func (s *extractStartTsSuite) TestExtractStartTs(c *C) { } func (s *extractStartTsSuite) TestMaxPrevSecFallback(c *C) { - s.store.resolveTSMu.resolveTS[2] = 0x8000000000000002 - s.store.resolveTSMu.resolveTS[3] = 0x8000000000000001 + s.store.setSafeTS(2, 0x8000000000000002) + s.store.setSafeTS(3, 0x8000000000000001) i := uint64(100) cases := []kv.TransactionOption{ diff --git a/store/tikv/kv.go b/store/tikv/kv.go index 5ddca52726a04..a487b0024e3e9 100644 --- a/store/tikv/kv.go +++ b/store/tikv/kv.go @@ -82,10 +82,10 @@ type KVStore struct { spMutex sync.RWMutex // this is used to update safePoint and spTime closed chan struct{} // this is used to nofity when the store is closed - resolveTSMu struct { - sync.RWMutex - resolveTS map[uint64]uint64 // storeID -> resolveTS - } + // storeID -> safeTS, stored as map[uint64]uint64 + // safeTS here will be used during the Stale Read process, + // it indicates the safe timestamp point that can be used to read consistent but may not the latest data. + safeTSMap sync.Map replicaReadSeed uint32 // this is used to load balance followers / learners when replica read is enabled } @@ -142,7 +142,6 @@ func NewKVStore(uuid string, pdClient pd.Client, spkv SafePointKV, client Client replicaReadSeed: rand.Uint32(), } store.lockResolver = newLockResolver(store) - store.resolveTSMu.resolveTS = make(map[uint64]uint64) go store.runSafePointChecker() go store.safeTSUpdater() @@ -337,20 +336,30 @@ func (s *KVStore) GetTiKVClient() (client Client) { return s.client } -func (s *KVStore) getMinResolveTSByStores(stores []*Store) uint64 { - failpoint.Inject("injectResolveTS", func(val failpoint.Value) { +func (s *KVStore) getSafeTS(storeID uint64) uint64 { + safeTS, ok := s.safeTSMap.Load(storeID) + if !ok { + return 0 + } + return safeTS.(uint64) +} + +func (s *KVStore) setSafeTS(storeID, safeTS uint64) { + s.safeTSMap.Store(storeID, safeTS) +} + +func (s *KVStore) getMinSafeTSByStores(stores []*Store) uint64 { + failpoint.Inject("injectSafeTS", func(val failpoint.Value) { injectTS := val.(int) failpoint.Return(uint64(injectTS)) }) minSafeTS := uint64(math.MaxUint64) - s.resolveTSMu.RLock() - defer s.resolveTSMu.RUnlock() // when there is no store, return 0 in order to let minStartTS become startTS directly if len(stores) < 1 { return 0 } for _, store := range stores { - safeTS := s.resolveTSMu.resolveTS[store.storeID] + safeTS := s.getSafeTS(store.storeID) if safeTS < minSafeTS { minSafeTS = safeTS } @@ -368,12 +377,12 @@ func (s *KVStore) safeTSUpdater() { case <-s.Closed(): return case <-t.C: - s.updateResolveTS(ctx) + s.updateSafeTS(ctx) } } } -func (s *KVStore) updateResolveTS(ctx context.Context) { +func (s *KVStore) updateSafeTS(ctx context.Context) { stores := s.regionCache.getStoresByType(tikvrpc.TiKV) tikvClient := s.GetTiKVClient() wg := &sync.WaitGroup{} @@ -389,13 +398,11 @@ func (s *KVStore) updateResolveTS(ctx context.Context) { EndKey: []byte(""), }}), ReadTimeoutShort) if err != nil { - logutil.BgLogger().Debug("update resolveTS failed", zap.Error(err), zap.Uint64("store-id", storeID)) + logutil.BgLogger().Debug("update safeTS failed", zap.Error(err), zap.Uint64("store-id", storeID)) return } safeTSResp := resp.Resp.(*kvrpcpb.StoreSafeTSResponse) - s.resolveTSMu.Lock() - s.resolveTSMu.resolveTS[storeID] = safeTSResp.GetSafeTs() - s.resolveTSMu.Unlock() + s.setSafeTS(storeID, safeTSResp.GetSafeTs()) }(ctx, wg, storeID, storeAddr) } wg.Wait() diff --git a/store/tikv/txn.go b/store/tikv/txn.go index 0ae2df13c12e9..1d678d010957e 100644 --- a/store/tikv/txn.go +++ b/store/tikv/txn.go @@ -112,12 +112,12 @@ func extractStartTs(store *KVStore, options kv.TransactionOption) (uint64, error } else { stores = allStores } - resolveTS := store.getMinResolveTSByStores(stores) + safeTS := store.getMinSafeTSByStores(stores) startTs = *options.MinStartTS - // If the resolveTS is larger than the minStartTS, we will use resolveTS as StartTS, otherwise we will use + // If the safeTS is larger than the minStartTS, we will use safeTS as StartTS, otherwise we will use // minStartTS directly. - if oracle.CompareTS(startTs, resolveTS) < 0 { - startTs = resolveTS + if oracle.CompareTS(startTs, safeTS) < 0 { + startTs = safeTS } } else if options.MaxPrevSec != nil { bo := NewBackofferWithVars(context.Background(), tsoMaxBackoff, nil) From 6e348f44d2b5dc68a9aa13fa72d8847218909337 Mon Sep 17 00:00:00 2001 From: xiongjiwei Date: Tue, 11 May 2021 14:41:39 +0800 Subject: [PATCH 60/67] parser: update parser to fix Can't recognize numeric literals when set 'ANSI_QUOTES' sql_mode (#24522) --- expression/integration_test.go | 9 +++++++++ go.mod | 2 +- go.sum | 4 ++-- 3 files changed, 12 insertions(+), 3 deletions(-) diff --git a/expression/integration_test.go b/expression/integration_test.go index d4b6a031087e4..f15dc5822be15 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -9156,6 +9156,15 @@ func (s *testIntegrationSerialSuite) TestIssue23805(c *C) { tk.MustExec("insert ignore into tbl_5 set col_28 = 'ZmZIdSnq' , col_25 = '18:50:52.00' on duplicate key update col_26 = 'y';\n") } +func (s *testIntegrationSuite) TestIssue24429(c *C) { + tk := testkit.NewTestKit(c, s.store) + + tk.MustExec("set @@sql_mode = ANSI_QUOTES;") + tk.MustExec("use test") + tk.MustExec("create table t (a int);") + tk.MustQuery(`select t."a"=10 from t;`).Check(testkit.Rows()) +} + func (s *testIntegrationSuite) TestVitessHash(c *C) { defer s.cleanEnv(c) tk := testkit.NewTestKit(c, s.store) diff --git a/go.mod b/go.mod index bf927f9cc55ce..cd8becf5a757c 100644 --- a/go.mod +++ b/go.mod @@ -47,7 +47,7 @@ require ( github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 github.com/pingcap/kvproto v0.0.0-20210429093846-65f54a202d7e github.com/pingcap/log v0.0.0-20210317133921-96f4fcab92a4 - github.com/pingcap/parser v0.0.0-20210427084954-8e8ed7927bde + github.com/pingcap/parser v0.0.0-20210508071014-cd9cd78e230c github.com/pingcap/sysutil v0.0.0-20210315073920-cc0985d983a3 github.com/pingcap/tidb-tools v4.0.9-0.20201127090955-2707c97b3853+incompatible github.com/pingcap/tipb v0.0.0-20210422074242-57dd881b81b1 diff --git a/go.sum b/go.sum index a3ebad580db64..4e1030039a04f 100644 --- a/go.sum +++ b/go.sum @@ -443,8 +443,8 @@ github.com/pingcap/log v0.0.0-20200511115504-543df19646ad/go.mod h1:4rbK1p9ILyIf github.com/pingcap/log v0.0.0-20201112100606-8f1e84a3abc8/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20210317133921-96f4fcab92a4 h1:ERrF0fTuIOnwfGbt71Ji3DKbOEaP189tjym50u8gpC8= github.com/pingcap/log v0.0.0-20210317133921-96f4fcab92a4/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= -github.com/pingcap/parser v0.0.0-20210427084954-8e8ed7927bde h1:CcGOCE3kr8aYBy6rRcWWldidL1X5smQxV79nlnzOk+o= -github.com/pingcap/parser v0.0.0-20210427084954-8e8ed7927bde/go.mod h1:xZC8I7bug4GJ5KtHhgAikjTfU4kBv1Sbo3Pf1MZ6lVw= +github.com/pingcap/parser v0.0.0-20210508071014-cd9cd78e230c h1:GLFd+wBN7EsV6ad/tVGFCD37taOyzIMVs3SdiWZF18I= +github.com/pingcap/parser v0.0.0-20210508071014-cd9cd78e230c/go.mod h1:xZC8I7bug4GJ5KtHhgAikjTfU4kBv1Sbo3Pf1MZ6lVw= github.com/pingcap/sysutil v0.0.0-20200206130906-2bfa6dc40bcd/go.mod h1:EB/852NMQ+aRKioCpToQ94Wl7fktV+FNnxf3CX/TTXI= github.com/pingcap/sysutil v0.0.0-20210315073920-cc0985d983a3 h1:A9KL9R+lWSVPH8IqUuH1QSTRJ5FGoY1bT2IcfPKsWD8= github.com/pingcap/sysutil v0.0.0-20210315073920-cc0985d983a3/go.mod h1:tckvA041UWP+NqYzrJ3fMgC/Hw9wnmQ/tUkp/JaHly8= From c1a1550ba0c80726ec51b226a2f68e572a835901 Mon Sep 17 00:00:00 2001 From: Shirly Date: Tue, 11 May 2021 14:59:38 +0800 Subject: [PATCH 61/67] store/tikv: make tikv.ErrUnknown as a normal error instead of terror (#24486) --- store/driver/txn/error.go | 6 ++++++ store/tikv/error/errcode.go | 1 - store/tikv/error/error.go | 3 ++- 3 files changed, 8 insertions(+), 2 deletions(-) diff --git a/store/driver/txn/error.go b/store/driver/txn/error.go index 5bbdc01024947..2a69a6567a489 100644 --- a/store/driver/txn/error.go +++ b/store/driver/txn/error.go @@ -60,6 +60,8 @@ var ( ErrPDServerTimeout = dbterror.ClassTiKV.NewStd(errno.ErrPDServerTimeout) // ErrRegionUnavailable is the error when region is not available. ErrRegionUnavailable = dbterror.ClassTiKV.NewStd(errno.ErrRegionUnavailable) + // ErrUnknown is the unknow error. + ErrUnknown = dbterror.ClassTiKV.NewStd(errno.ErrUnknown) ) // Registers error returned from TiKV. @@ -265,6 +267,10 @@ func ToTiDBErr(err error) error { return ErrTokenLimit.GenWithStackByArgs(e.StoreID) } + if errors.ErrorEqual(err, tikverr.ErrUnknown) { + return ErrUnknown + } + return errors.Trace(originErr) } diff --git a/store/tikv/error/errcode.go b/store/tikv/error/errcode.go index 01e8db4d12473..17ee9a3ccac2c 100644 --- a/store/tikv/error/errcode.go +++ b/store/tikv/error/errcode.go @@ -16,7 +16,6 @@ package error // MySQL error code. // This value is numeric. It is not portable to other database systems. const ( - CodeUnknown = 1105 CodeLockWaitTimeout = 1205 CodeQueryInterrupted = 1317 CodeLockAcquireFailAndNoWaitSet = 3572 diff --git a/store/tikv/error/error.go b/store/tikv/error/error.go index bde97e8b2e5d4..d83482c679449 100644 --- a/store/tikv/error/error.go +++ b/store/tikv/error/error.go @@ -52,6 +52,8 @@ var ( ErrTiFlashServerBusy = errors.New("tiflash server busy") // ErrRegionUnavailable is the error when region is not available. ErrRegionUnavailable = errors.New("region unavailable") + // ErrUnknown is the unknow error. + ErrUnknown = errors.New("unknow") ) // MismatchClusterID represents the message that the cluster ID of the PD client does not match the PD. @@ -61,7 +63,6 @@ const MismatchClusterID = "mismatch cluster id" var ( ErrQueryInterrupted = dbterror.ClassTiKV.NewStd(CodeQueryInterrupted) ErrLockAcquireFailAndNoWaitSet = dbterror.ClassTiKV.NewStd(CodeLockAcquireFailAndNoWaitSet) - ErrUnknown = dbterror.ClassTiKV.NewStd(CodeUnknown) ) // IsErrNotFound checks if err is a kind of NotFound error. From b639ff55fdb3e534ecaa0f52bb52e257eb4d934e Mon Sep 17 00:00:00 2001 From: Shirly Date: Tue, 11 May 2021 15:19:40 +0800 Subject: [PATCH 62/67] store/tikv: make tikv.ErrQueryInterrupted as a normal error (#24474) --- session/pessimistic_test.go | 2 +- store/copr/batch_coprocessor.go | 3 +-- store/copr/coprocessor.go | 3 +-- store/copr/mpp.go | 3 +-- store/driver/txn/error.go | 6 ++++++ store/tikv/error/errcode.go | 2 -- store/tikv/error/error.go | 3 ++- 7 files changed, 12 insertions(+), 10 deletions(-) diff --git a/session/pessimistic_test.go b/session/pessimistic_test.go index 8fdd635b51bc1..fa79cfae8ff02 100644 --- a/session/pessimistic_test.go +++ b/session/pessimistic_test.go @@ -612,7 +612,7 @@ func (s *testPessimisticSuite) TestWaitLockKill(c *C) { _, err := tk2.Exec("update test_kill set c = c + 1 where id = 1") wg.Done() c.Assert(err, NotNil) - c.Assert(terror.ErrorEqual(err, tikverr.ErrQueryInterrupted), IsTrue) + c.Assert(terror.ErrorEqual(err, txndriver.ErrQueryInterrupted), IsTrue) tk.MustExec("rollback") } diff --git a/store/copr/batch_coprocessor.go b/store/copr/batch_coprocessor.go index 3231f95bbc824..af224c59fc38a 100644 --- a/store/copr/batch_coprocessor.go +++ b/store/copr/batch_coprocessor.go @@ -28,7 +28,6 @@ import ( "github.com/pingcap/tidb/kv" txndriver "github.com/pingcap/tidb/store/driver/txn" "github.com/pingcap/tidb/store/tikv" - tikverr "github.com/pingcap/tidb/store/tikv/error" tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/logutil" "github.com/pingcap/tidb/store/tikv/metrics" @@ -262,7 +261,7 @@ func (b *batchCopIterator) recvFromRespCh(ctx context.Context) (resp *batchCopRe return case <-ticker.C: if atomic.LoadUint32(b.vars.Killed) == 1 { - resp = &batchCopResponse{err: tikverr.ErrQueryInterrupted} + resp = &batchCopResponse{err: txndriver.ErrQueryInterrupted} ok = true return } diff --git a/store/copr/coprocessor.go b/store/copr/coprocessor.go index cb799edb16f70..5e7eab303e84f 100644 --- a/store/copr/coprocessor.go +++ b/store/copr/coprocessor.go @@ -37,7 +37,6 @@ import ( tidbmetrics "github.com/pingcap/tidb/metrics" txndriver "github.com/pingcap/tidb/store/driver/txn" "github.com/pingcap/tidb/store/tikv" - tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/store/tikv/logutil" "github.com/pingcap/tidb/store/tikv/metrics" "github.com/pingcap/tidb/store/tikv/tikvrpc" @@ -477,7 +476,7 @@ func (it *copIterator) recvFromRespCh(ctx context.Context, respCh <-chan *copRes return case <-ticker.C: if atomic.LoadUint32(it.vars.Killed) == 1 { - resp = &copResponse{err: tikverr.ErrQueryInterrupted} + resp = &copResponse{err: txndriver.ErrQueryInterrupted} ok = true return } diff --git a/store/copr/mpp.go b/store/copr/mpp.go index 3ea07e744f9b9..10784912faa9b 100644 --- a/store/copr/mpp.go +++ b/store/copr/mpp.go @@ -29,7 +29,6 @@ import ( "github.com/pingcap/tidb/kv" txndriver "github.com/pingcap/tidb/store/driver/txn" "github.com/pingcap/tidb/store/tikv" - tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/store/tikv/logutil" "github.com/pingcap/tidb/store/tikv/tikvrpc" "go.uber.org/zap" @@ -406,7 +405,7 @@ func (m *mppIterator) nextImpl(ctx context.Context) (resp *mppResponse, ok bool, return case <-ticker.C: if m.vars != nil && m.vars.Killed != nil && atomic.LoadUint32(m.vars.Killed) == 1 { - err = tikverr.ErrQueryInterrupted + err = txndriver.ErrQueryInterrupted exit = true return } diff --git a/store/driver/txn/error.go b/store/driver/txn/error.go index 2a69a6567a489..456a9f118c9cd 100644 --- a/store/driver/txn/error.go +++ b/store/driver/txn/error.go @@ -47,6 +47,8 @@ var ( ErrGCTooEarly = dbterror.ClassTiKV.NewStd(errno.ErrGCTooEarly) // ErrTiKVStaleCommand is the error that the command is stale in tikv. ErrTiKVStaleCommand = dbterror.ClassTiKV.NewStd(errno.ErrTiKVStaleCommand) + // ErrQueryInterrupted is the error when the query is interrupted. + ErrQueryInterrupted = dbterror.ClassTiKV.NewStd(errno.ErrQueryInterrupted) // ErrTiKVMaxTimestampNotSynced is the error that tikv's max timestamp is not synced. ErrTiKVMaxTimestampNotSynced = dbterror.ClassTiKV.NewStd(errno.ErrTiKVMaxTimestampNotSynced) ErrResolveLockTimeout = dbterror.ClassTiKV.NewStd(errno.ErrResolveLockTimeout) @@ -231,6 +233,10 @@ func ToTiDBErr(err error) error { return ErrTiFlashServerTimeout } + if errors.ErrorEqual(err, tikverr.ErrQueryInterrupted) { + return ErrQueryInterrupted + } + if errors.ErrorEqual(err, tikverr.ErrTiKVServerBusy) { return ErrTiKVServerBusy } diff --git a/store/tikv/error/errcode.go b/store/tikv/error/errcode.go index 17ee9a3ccac2c..5455c75da2ed4 100644 --- a/store/tikv/error/errcode.go +++ b/store/tikv/error/errcode.go @@ -16,7 +16,5 @@ package error // MySQL error code. // This value is numeric. It is not portable to other database systems. const ( - CodeLockWaitTimeout = 1205 - CodeQueryInterrupted = 1317 CodeLockAcquireFailAndNoWaitSet = 3572 ) diff --git a/store/tikv/error/error.go b/store/tikv/error/error.go index d83482c679449..d8d5122fad774 100644 --- a/store/tikv/error/error.go +++ b/store/tikv/error/error.go @@ -38,6 +38,8 @@ var ( ErrTiKVServerTimeout = errors.New("tikv server timeout") // ErrTiFlashServerTimeout is the error when tiflash server is timeout. ErrTiFlashServerTimeout = errors.New("tiflash server timeout") + // ErrQueryInterrupted is the error when the query is interrupted. + ErrQueryInterrupted = errors.New("query interruppted") // ErrTiKVStaleCommand is the error that the command is stale in tikv. ErrTiKVStaleCommand = errors.New("tikv stale command") // ErrTiKVMaxTimestampNotSynced is the error that tikv's max timestamp is not synced. @@ -61,7 +63,6 @@ const MismatchClusterID = "mismatch cluster id" // error instances. var ( - ErrQueryInterrupted = dbterror.ClassTiKV.NewStd(CodeQueryInterrupted) ErrLockAcquireFailAndNoWaitSet = dbterror.ClassTiKV.NewStd(CodeLockAcquireFailAndNoWaitSet) ) From 263a47e85ce04f74ec80d1d35b426618bc89b5a3 Mon Sep 17 00:00:00 2001 From: rebelice Date: Tue, 11 May 2021 15:51:39 +0800 Subject: [PATCH 63/67] executor: add a test for dynamic partition prune mode with equal expression (#24520) --- executor/partition_table_test.go | 66 ++++++++++++++++++++++++++++++++ util/testkit/testkit.go | 11 ++++++ 2 files changed, 77 insertions(+) diff --git a/executor/partition_table_test.go b/executor/partition_table_test.go index 8ce126972ec73..0c9e63129ebf1 100644 --- a/executor/partition_table_test.go +++ b/executor/partition_table_test.go @@ -434,6 +434,72 @@ func (s *partitionTableSuite) TestGlobalStatsAndSQLBinding(c *C) { tk.MustIndexLookup("select * from tlist where a<1") } +func createTable4DynamicPruneModeTestWithExpression(tk *testkit.TestKit) { + tk.MustExec("create table trange(a int) partition by range(a) (partition p0 values less than(3), partition p1 values less than (5), partition p2 values less than(11));") + tk.MustExec("create table thash(a int) partition by hash(a) partitions 4;") + tk.MustExec("create table t(a int)") + tk.MustExec("insert into trange values(1), (1), (1), (2), (3), (4), (5), (6), (7), (7), (10), (NULL), (NULL);") + tk.MustExec("insert into thash values(1), (1), (1), (2), (3), (4), (5), (6), (7), (7), (10), (NULL), (NULL);") + tk.MustExec("insert into t values(1), (1), (1), (2), (3), (4), (5), (6), (7), (7), (10), (NULL), (NULL);") + tk.MustExec("set session tidb_partition_prune_mode='dynamic'") + tk.MustExec("analyze table trange") + tk.MustExec("analyze table thash") + tk.MustExec("analyze table t") +} + +type testData4Expression struct { + sql string + partitions []string +} + +func (s *partitionTableSuite) TestDynamicPruneModeWithEqualExpression(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("drop database if exists db_equal_expression") + tk.MustExec("create database db_equal_expression") + tk.MustExec("use db_equal_expression") + createTable4DynamicPruneModeTestWithExpression(tk) + + tables := []string{"trange", "thash"} + tests := []testData4Expression{ + { + sql: "select * from %s where a = 2", + partitions: []string{ + "p0", + "p2", + }, + }, + { + sql: "select * from %s where a = 4 or a = 1", + partitions: []string{ + "p0,p1", + "p0,p1", + }, + }, + { + sql: "select * from %s where a = -1", + partitions: []string{ + "p0", + "p1", + }, + }, + { + sql: "select * from %s where a is NULL", + partitions: []string{ + "p0", + "p0", + }, + }, + } + + for _, t := range tests { + for i := range t.partitions { + sql := fmt.Sprintf(t.sql, tables[i]) + c.Assert(tk.MustPartition(sql, t.partitions[i]), IsTrue) + tk.MustQuery(sql).Sort().Check(tk.MustQuery(fmt.Sprintf(t.sql, "t")).Sort().Rows()) + } + } +} + func (s *partitionTableSuite) TestDirectReadingWithAgg(c *C) { if israce.RaceEnabled { c.Skip("exhaustive types test, skip race test") diff --git a/util/testkit/testkit.go b/util/testkit/testkit.go index 4992e28663b1a..06eb826c56b78 100644 --- a/util/testkit/testkit.go +++ b/util/testkit/testkit.go @@ -255,6 +255,17 @@ func (tk *TestKit) MustNoGlobalStats(table string) bool { return true } +// MustPartition checks if the result execution plan must read specific partitions. +func (tk *TestKit) MustPartition(sql string, partitions string, args ...interface{}) bool { + rs := tk.MustQuery("explain "+sql, args...) + for i := range rs.rows { + if strings.Compare(rs.rows[i][3], "partition:"+partitions) == 0 { + return true + } + } + return false +} + // MustUseIndex checks if the result execution plan contains specific index(es). func (tk *TestKit) MustUseIndex(sql string, index string, args ...interface{}) bool { rs := tk.MustQuery("explain "+sql, args...) From 7d41c868128d07bcbc2359533d39e7736b180620 Mon Sep 17 00:00:00 2001 From: Mingcong Han Date: Tue, 11 May 2021 16:21:39 +0800 Subject: [PATCH 64/67] planner: create new column slice in PreparePossibleProperties (#24342) --- planner/core/integration_test.go | 17 +++++++++++++++++ planner/core/property_cols_prune.go | 15 ++++++++------- 2 files changed, 25 insertions(+), 7 deletions(-) diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index 4ae7342a4f7d7..7f58d2db6d466 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -3597,6 +3597,23 @@ func (s *testIntegrationSuite) TestIssue24095(c *C) { } } +func (s *testIntegrationSuite) TestIssue24281(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists member, agent, deposit, view_member_agents") + tk.MustExec("create table member(login varchar(50) NOT NULL, agent_login varchar(100) DEFAULT NULL, PRIMARY KEY(login))") + tk.MustExec("create table agent(login varchar(50) NOT NULL, data varchar(100) DEFAULT NULL, share_login varchar(50) NOT NULL, PRIMARY KEY(login))") + tk.MustExec("create table deposit(id varchar(50) NOT NULL, member_login varchar(50) NOT NULL, transfer_amount int NOT NULL, PRIMARY KEY(id), KEY midx(member_login, transfer_amount))") + tk.MustExec("create definer='root'@'localhost' view view_member_agents (member, share_login) as select m.login as member, a.share_login AS share_login from member as m join agent as a on m.agent_login = a.login") + + tk.MustExec(" select s.member_login as v1, SUM(s.transfer_amount) AS v2 " + + "FROM deposit AS s " + + "JOIN view_member_agents AS v ON s.member_login = v.member " + + "WHERE 1 = 1 AND v.share_login = 'somevalue' " + + "GROUP BY s.member_login " + + "UNION select 1 as v1, 2 as v2") +} + func (s *testIntegrationSuite) TestConflictReadFromStorage(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") diff --git a/planner/core/property_cols_prune.go b/planner/core/property_cols_prune.go index 91a9f34fb9017..9cd83adc412de 100644 --- a/planner/core/property_cols_prune.go +++ b/planner/core/property_cols_prune.go @@ -148,21 +148,22 @@ func (p *LogicalProjection) PreparePossibleProperties(schema *expression.Schema, } } tmpSchema := expression.NewSchema(oldCols...) - for i := len(childProperties) - 1; i >= 0; i-- { - for j, col := range childProperties[i] { + newProperties := make([][]*expression.Column, 0, len(childProperties)) + for _, childProperty := range childProperties { + newChildProperty := make([]*expression.Column, 0, len(childProperty)) + for _, col := range childProperty { pos := tmpSchema.ColumnIndex(col) if pos >= 0 { - childProperties[i][j] = newCols[pos] + newChildProperty = append(newChildProperty, newCols[pos]) } else { - childProperties[i] = childProperties[i][:j] break } } - if len(childProperties[i]) == 0 { - childProperties = append(childProperties[:i], childProperties[i+1:]...) + if len(newChildProperty) != 0 { + newProperties = append(newProperties, newChildProperty) } } - return childProperties + return newProperties } func clonePossibleProperties(props [][]*expression.Column) [][]*expression.Column { From 7611952a09f84637456e180274606d803ed69868 Mon Sep 17 00:00:00 2001 From: Shirly Date: Tue, 11 May 2021 17:11:39 +0800 Subject: [PATCH 65/67] store/tikv: make tikv.ErrLockAcquireFailAndNoWaitSet as a normal error (#24475) --- session/pessimistic_test.go | 21 ++++++++++----------- store/driver/txn/error.go | 8 +++++++- store/tikv/error/errcode.go | 20 -------------------- store/tikv/error/error.go | 8 ++------ 4 files changed, 19 insertions(+), 38 deletions(-) delete mode 100644 store/tikv/error/errcode.go diff --git a/session/pessimistic_test.go b/session/pessimistic_test.go index fa79cfae8ff02..83f0057384aea 100644 --- a/session/pessimistic_test.go +++ b/session/pessimistic_test.go @@ -33,7 +33,6 @@ import ( "github.com/pingcap/tidb/sessionctx/variable" txndriver "github.com/pingcap/tidb/store/driver/txn" "github.com/pingcap/tidb/store/tikv" - tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" @@ -1132,11 +1131,11 @@ func (s *testPessimisticSuite) TestPessimisticLockNonExistsKey(c *C) { tk1.MustExec("begin pessimistic") err := tk1.ExecToErr("select * from t where k = 2 for update nowait") - c.Check(tikverr.ErrLockAcquireFailAndNoWaitSet.Equal(err), IsTrue) + c.Check(txndriver.ErrLockAcquireFailAndNoWaitSet.Equal(err), IsTrue) err = tk1.ExecToErr("select * from t where k = 4 for update nowait") - c.Check(tikverr.ErrLockAcquireFailAndNoWaitSet.Equal(err), IsTrue) + c.Check(txndriver.ErrLockAcquireFailAndNoWaitSet.Equal(err), IsTrue) err = tk1.ExecToErr("select * from t where k = 7 for update nowait") - c.Check(tikverr.ErrLockAcquireFailAndNoWaitSet.Equal(err), IsTrue) + c.Check(txndriver.ErrLockAcquireFailAndNoWaitSet.Equal(err), IsTrue) tk.MustExec("rollback") tk1.MustExec("rollback") @@ -1148,9 +1147,9 @@ func (s *testPessimisticSuite) TestPessimisticLockNonExistsKey(c *C) { tk1.MustExec("begin pessimistic") err = tk1.ExecToErr("select * from t where k = 2 for update nowait") - c.Check(tikverr.ErrLockAcquireFailAndNoWaitSet.Equal(err), IsTrue) + c.Check(txndriver.ErrLockAcquireFailAndNoWaitSet.Equal(err), IsTrue) err = tk1.ExecToErr("select * from t where k = 6 for update nowait") - c.Check(tikverr.ErrLockAcquireFailAndNoWaitSet.Equal(err), IsTrue) + c.Check(txndriver.ErrLockAcquireFailAndNoWaitSet.Equal(err), IsTrue) tk.MustExec("rollback") tk1.MustExec("rollback") } @@ -1283,7 +1282,7 @@ func (s *testPessimisticSuite) TestBatchPointGetLockIndex(c *C) { c.Assert(txndriver.ErrLockWaitTimeout.Equal(err), IsTrue) err = tk2.ExecToErr("select * from t1 where c2 = 3 for update nowait") c.Assert(err, NotNil) - c.Assert(tikverr.ErrLockAcquireFailAndNoWaitSet.Equal(err), IsTrue) + c.Assert(txndriver.ErrLockAcquireFailAndNoWaitSet.Equal(err), IsTrue) tk.MustExec("rollback") tk2.MustExec("rollback") } @@ -1430,12 +1429,12 @@ func (s *testPessimisticSuite) TestGenerateColPointGet(c *C) { tk2.MustExec("begin pessimistic") err := tk2.ExecToErr("select * from tu where z = 3 for update nowait") c.Assert(err, NotNil) - c.Assert(terror.ErrorEqual(err, tikverr.ErrLockAcquireFailAndNoWaitSet), IsTrue) + c.Assert(terror.ErrorEqual(err, txndriver.ErrLockAcquireFailAndNoWaitSet), IsTrue) tk.MustExec("begin pessimistic") tk.MustExec("insert into tu(x, y) values(2, 2);") err = tk2.ExecToErr("select * from tu where z = 4 for update nowait") c.Assert(err, NotNil) - c.Assert(terror.ErrorEqual(err, tikverr.ErrLockAcquireFailAndNoWaitSet), IsTrue) + c.Assert(terror.ErrorEqual(err, txndriver.ErrLockAcquireFailAndNoWaitSet), IsTrue) // test batch point get lock tk.MustExec("begin pessimistic") @@ -1444,12 +1443,12 @@ func (s *testPessimisticSuite) TestGenerateColPointGet(c *C) { tk2.MustExec("begin pessimistic") err = tk2.ExecToErr("select x from tu where z in (3, 7, 9) for update nowait") c.Assert(err, NotNil) - c.Assert(terror.ErrorEqual(err, tikverr.ErrLockAcquireFailAndNoWaitSet), IsTrue) + c.Assert(terror.ErrorEqual(err, txndriver.ErrLockAcquireFailAndNoWaitSet), IsTrue) tk.MustExec("begin pessimistic") tk.MustExec("insert into tu(x, y) values(5, 6);") err = tk2.ExecToErr("select * from tu where z = 11 for update nowait") c.Assert(err, NotNil) - c.Assert(terror.ErrorEqual(err, tikverr.ErrLockAcquireFailAndNoWaitSet), IsTrue) + c.Assert(terror.ErrorEqual(err, txndriver.ErrLockAcquireFailAndNoWaitSet), IsTrue) tk.MustExec("commit") tk2.MustExec("commit") diff --git a/store/driver/txn/error.go b/store/driver/txn/error.go index 456a9f118c9cd..4c8e770c44ff7 100644 --- a/store/driver/txn/error.go +++ b/store/driver/txn/error.go @@ -51,7 +51,9 @@ var ( ErrQueryInterrupted = dbterror.ClassTiKV.NewStd(errno.ErrQueryInterrupted) // ErrTiKVMaxTimestampNotSynced is the error that tikv's max timestamp is not synced. ErrTiKVMaxTimestampNotSynced = dbterror.ClassTiKV.NewStd(errno.ErrTiKVMaxTimestampNotSynced) - ErrResolveLockTimeout = dbterror.ClassTiKV.NewStd(errno.ErrResolveLockTimeout) + // ErrLockAcquireFailAndNoWaitSet is the error that acquire the lock failed while no wait is setted. + ErrLockAcquireFailAndNoWaitSet = dbterror.ClassTiKV.NewStd(errno.ErrLockAcquireFailAndNoWaitSet) + ErrResolveLockTimeout = dbterror.ClassTiKV.NewStd(errno.ErrResolveLockTimeout) // ErrLockWaitTimeout is the error that wait for the lock is timeout. ErrLockWaitTimeout = dbterror.ClassTiKV.NewStd(errno.ErrLockWaitTimeout) // ErrTiKVServerBusy is the error when tikv server is busy. @@ -257,6 +259,10 @@ func ToTiDBErr(err error) error { return ErrTiKVMaxTimestampNotSynced } + if errors.ErrorEqual(err, tikverr.ErrLockAcquireFailAndNoWaitSet) { + return ErrLockAcquireFailAndNoWaitSet + } + if errors.ErrorEqual(err, tikverr.ErrResolveLockTimeout) { return ErrResolveLockTimeout } diff --git a/store/tikv/error/errcode.go b/store/tikv/error/errcode.go deleted file mode 100644 index 5455c75da2ed4..0000000000000 --- a/store/tikv/error/errcode.go +++ /dev/null @@ -1,20 +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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package error - -// MySQL error code. -// This value is numeric. It is not portable to other database systems. -const ( - CodeLockAcquireFailAndNoWaitSet = 3572 -) diff --git a/store/tikv/error/error.go b/store/tikv/error/error.go index d8d5122fad774..898354cc11a2d 100644 --- a/store/tikv/error/error.go +++ b/store/tikv/error/error.go @@ -20,7 +20,6 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/pingcap/tidb/util/dbterror" ) var ( @@ -44,6 +43,8 @@ var ( ErrTiKVStaleCommand = errors.New("tikv stale command") // ErrTiKVMaxTimestampNotSynced is the error that tikv's max timestamp is not synced. ErrTiKVMaxTimestampNotSynced = errors.New("tikv max timestamp not synced") + // ErrLockAcquireFailAndNoWaitSet is the error that acquire the lock failed while no wait is setted. + ErrLockAcquireFailAndNoWaitSet = errors.New("lock acquired failed and no wait is setted") // ErrResolveLockTimeout is the error that resolve lock timeout. ErrResolveLockTimeout = errors.New("resolve lock timeout") // ErrLockWaitTimeout is the error that wait for the lock is timeout. @@ -61,11 +62,6 @@ var ( // MismatchClusterID represents the message that the cluster ID of the PD client does not match the PD. const MismatchClusterID = "mismatch cluster id" -// error instances. -var ( - ErrLockAcquireFailAndNoWaitSet = dbterror.ClassTiKV.NewStd(CodeLockAcquireFailAndNoWaitSet) -) - // IsErrNotFound checks if err is a kind of NotFound error. func IsErrNotFound(err error) bool { return errors.ErrorEqual(err, ErrNotExist) From c5e20dd1a2b5bf662cb42e240dac97da050c9729 Mon Sep 17 00:00:00 2001 From: Ryan Leung Date: Tue, 11 May 2021 19:21:40 +0800 Subject: [PATCH 66/67] variable: fix the TiDBDistSQLScanConcurrency comment (#24451) --- sessionctx/variable/tidb_vars.go | 1 - 1 file changed, 1 deletion(-) diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 30d52ac54f386..54616f902db5e 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -216,7 +216,6 @@ const ( // A distsql scan task can be a table scan or a index scan, which may be distributed to many TiKV nodes. // Higher concurrency may reduce latency, but with the cost of higher memory usage and system performance impact. // If the query has a LIMIT clause, high concurrency makes the system do much more work than needed. - // tidb_distsql_scan_concurrency is deprecated, use tidb_executor_concurrency instead. TiDBDistSQLScanConcurrency = "tidb_distsql_scan_concurrency" // tidb_opt_insubquery_to_join_and_agg is used to enable/disable the optimizer rule of rewriting IN subquery. From 3eedd404b852287a899e3952ed776a2db03d80af Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Tue, 11 May 2021 19:59:40 +0800 Subject: [PATCH 67/67] executor: add some test cases about partition table dynamic-mode with clustered-index (#24523) --- session/clustered_index_test.go | 49 +++++++++++++++++++++++++++++++++ 1 file changed, 49 insertions(+) diff --git a/session/clustered_index_test.go b/session/clustered_index_test.go index fd40cfd567f11..0f79b1b13fc2e 100644 --- a/session/clustered_index_test.go +++ b/session/clustered_index_test.go @@ -14,11 +14,16 @@ package session_test import ( + "fmt" + "math/rand" + "strings" + . "github.com/pingcap/check" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/util/collate" + "github.com/pingcap/tidb/util/israce" "github.com/pingcap/tidb/util/testkit" "github.com/pingcap/tidb/util/testutil" ) @@ -578,6 +583,50 @@ func (s *testClusteredSerialSuite) TestPrefixClusteredIndexAddIndexAndRecover(c tk1.MustExec("admin check table t") } +func (s *testClusteredSerialSuite) TestPartitionTable(c *C) { + if israce.RaceEnabled { + c.Skip("exhaustive types test, skip race test") + } + + tk := testkit.NewTestKitWithInit(c, s.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 (s *testClusteredSerialSuite) TestClusteredIndexDecodeRestoredDataV5(c *C) { tk := testkit.NewTestKitWithInit(c, s.store)