From 2abf83ddd50755dc7c3af6d2f2cf669fab29051d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E4=BA=8C=E6=89=8B=E6=8E=89=E5=8C=85=E5=B7=A5=E7=A8=8B?= =?UTF-8?q?=E5=B8=88?= Date: Mon, 21 Aug 2023 19:03:34 +0800 Subject: [PATCH] *: remove feedback (#45969) ref pingcap/tidb#46056 --- br/pkg/backup/client.go | 4 +- br/pkg/checksum/executor.go | 2 +- br/pkg/lightning/backend/local/duplicate.go | 4 +- ddl/index_cop.go | 3 +- ddl/reorg.go | 4 +- distsql/BUILD.bazel | 2 - distsql/distsql.go | 11 +- distsql/distsql_test.go | 7 +- distsql/request_builder.go | 112 +- distsql/request_builder_test.go | 30 +- distsql/select_result.go | 10 +- domain/domain.go | 19 - executor/admin.go | 9 +- executor/analyze_col.go | 2 +- executor/analyze_idx.go | 2 +- executor/builder.go | 84 +- executor/checksum.go | 2 +- executor/distsql.go | 44 +- executor/index_merge_reader.go | 29 +- executor/inspection_summary.go | 3 - executor/table_reader.go | 32 +- executor/table_readers_required_rows_test.go | 3 +- executor/test/analyzetest/analyze_test.go | 20 +- executor/test/executor/BUILD.bazel | 1 - executor/test/executor/executor_test.go | 18 - executor/test/issuetest/BUILD.bazel | 1 - .../test/issuetest/executor_issue_test.go | 18 - infoschema/metric_table_def.go | 25 - metrics/grafana/tidb.json | 423 ++---- metrics/metrics.go | 4 - metrics/stats.go | 60 +- planner/core/BUILD.bazel | 1 - planner/core/find_best_task.go | 25 - planner/core/fragment.go | 4 +- planner/core/physical_plans.go | 15 +- planner/core/planbuilder.go | 12 - planner/core/planbuilder_test.go | 6 - session/BUILD.bazel | 1 - session/bootstrap.go | 3 + session/session.go | 25 - sessionctx/context.go | 4 - sessionctx/variable/sysvar.go | 23 +- statistics/BUILD.bazel | 6 - statistics/feedback.go | 1076 --------------- statistics/feedback_test.go | 310 ----- statistics/handle/BUILD.bazel | 4 - statistics/handle/gc.go | 3 - statistics/handle/handle.go | 37 - .../handle/handletest/globalstats/BUILD.bazel | 4 +- .../globalstats/globalstats_test.go | 96 -- statistics/handle/historical_stats_handler.go | 2 - statistics/handle/update.go | 735 +--------- statistics/handle/update_list_test.go | 2 - statistics/handle/updatetest/BUILD.bazel | 6 +- statistics/handle/updatetest/update_test.go | 1204 ----------------- statistics/histogram.go | 7 +- types/datum.go | 1 - util/mock/context.go | 3 - 58 files changed, 226 insertions(+), 4377 deletions(-) delete mode 100644 statistics/feedback.go delete mode 100644 statistics/feedback_test.go diff --git a/br/pkg/backup/client.go b/br/pkg/backup/client.go index edbe3efd0ae98..ab52aac9ab692 100644 --- a/br/pkg/backup/client.go +++ b/br/pkg/backup/client.go @@ -471,7 +471,7 @@ func appendRanges(tbl *model.TableInfo, tblID int64) ([]kv.KeyRange, error) { } retRanges := make([]kv.KeyRange, 0, 1+len(tbl.Indices)) - kvRanges, err := distsql.TableHandleRangesToKVRanges(nil, []int64{tblID}, tbl.IsCommonHandle, ranges, nil) + kvRanges, err := distsql.TableHandleRangesToKVRanges(nil, []int64{tblID}, tbl.IsCommonHandle, ranges) if err != nil { return nil, errors.Trace(err) } @@ -482,7 +482,7 @@ func appendRanges(tbl *model.TableInfo, tblID int64) ([]kv.KeyRange, error) { continue } ranges = ranger.FullRange() - idxRanges, err := distsql.IndexRangesToKVRanges(nil, tblID, index.ID, ranges, nil) + idxRanges, err := distsql.IndexRangesToKVRanges(nil, tblID, index.ID, ranges) if err != nil { return nil, errors.Trace(err) } diff --git a/br/pkg/checksum/executor.go b/br/pkg/checksum/executor.go index a77b0df85fa93..896a7297bab83 100644 --- a/br/pkg/checksum/executor.go +++ b/br/pkg/checksum/executor.go @@ -238,7 +238,7 @@ func buildTableRequest( var builder distsql.RequestBuilder // Use low priority to reducing impact to other requests. builder.Request.Priority = kv.PriorityLow - return builder.SetHandleRanges(nil, tableID, tableInfo.IsCommonHandle, ranges, nil). + return builder.SetHandleRanges(nil, tableID, tableInfo.IsCommonHandle, ranges). SetStartTS(startTS). SetChecksumRequest(checksum). SetConcurrency(int(concurrency)). diff --git a/br/pkg/lightning/backend/local/duplicate.go b/br/pkg/lightning/backend/local/duplicate.go index 69c1a1feec8c0..721b685e12504 100644 --- a/br/pkg/lightning/backend/local/duplicate.go +++ b/br/pkg/lightning/backend/local/duplicate.go @@ -227,13 +227,13 @@ func tableHandleKeyRanges(tableInfo *model.TableInfo) (*tidbkv.KeyRanges, error) ranges = ranger.FullRange() } tableIDs := physicalTableIDs(tableInfo) - return distsql.TableHandleRangesToKVRanges(nil, tableIDs, tableInfo.IsCommonHandle, ranges, nil) + return distsql.TableHandleRangesToKVRanges(nil, tableIDs, tableInfo.IsCommonHandle, ranges) } // tableIndexKeyRanges returns all key ranges associated with the tableInfo and indexInfo. func tableIndexKeyRanges(tableInfo *model.TableInfo, indexInfo *model.IndexInfo) (*tidbkv.KeyRanges, error) { tableIDs := physicalTableIDs(tableInfo) - return distsql.IndexRangesToKVRangesForTables(nil, tableIDs, indexInfo.ID, ranger.FullRange(), nil) + return distsql.IndexRangesToKVRangesForTables(nil, tableIDs, indexInfo.ID, ranger.FullRange()) } // DupKVStream is a streaming interface for collecting duplicate key-value pairs. diff --git a/ddl/index_cop.go b/ddl/index_cop.go index 909981f70f015..d3fdd59fa5ad1 100644 --- a/ddl/index_cop.go +++ b/ddl/index_cop.go @@ -32,7 +32,6 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" - "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/tablecodec" @@ -429,7 +428,7 @@ func (c *copContext) buildTableScan(ctx context.Context, startTS uint64, start, if err != nil { return nil, err } - return distsql.Select(ctx, c.sessCtx, kvReq, c.fieldTps, statistics.NewQueryFeedback(0, nil, 0, false)) + return distsql.Select(ctx, c.sessCtx, kvReq, c.fieldTps) } func (c *copContext) fetchTableScanResult(ctx context.Context, result distsql.SelectResult, diff --git a/ddl/reorg.go b/ddl/reorg.go index 1abda7c828e47..4d82071f4ad3a 100644 --- a/ddl/reorg.go +++ b/ddl/reorg.go @@ -561,7 +561,7 @@ func (dc *ddlCtx) buildDescTableScan(ctx *JobContext, startTS uint64, tbl table. } else { ranges = ranger.FullIntRange(false) } - builder = b.SetHandleRanges(sctx.GetSessionVars().StmtCtx, tbl.GetPhysicalID(), tbl.Meta().IsCommonHandle, ranges, nil) + builder = b.SetHandleRanges(sctx.GetSessionVars().StmtCtx, tbl.GetPhysicalID(), tbl.Meta().IsCommonHandle, ranges) builder.SetDAGRequest(dagPB). SetStartTS(startTS). SetKeepOrder(true). @@ -578,7 +578,7 @@ func (dc *ddlCtx) buildDescTableScan(ctx *JobContext, startTS uint64, tbl table. return nil, errors.Trace(err) } - result, err := distsql.Select(ctx.ddlJobCtx, sctx, kvReq, getColumnsTypes(handleCols), statistics.NewQueryFeedback(0, nil, 0, false)) + result, err := distsql.Select(ctx.ddlJobCtx, sctx, kvReq, getColumnsTypes(handleCols)) if err != nil { return nil, errors.Trace(err) } diff --git a/distsql/BUILD.bazel b/distsql/BUILD.bazel index 3a4044139c8eb..2601df8d0d0ce 100644 --- a/distsql/BUILD.bazel +++ b/distsql/BUILD.bazel @@ -23,7 +23,6 @@ go_library( "//sessionctx", "//sessionctx/stmtctx", "//sessionctx/variable", - "//statistics", "//store/copr", "//tablecodec", "//telemetry", @@ -74,7 +73,6 @@ go_test( "//sessionctx", "//sessionctx/stmtctx", "//sessionctx/variable", - "//statistics", "//store/copr", "//tablecodec", "//testkit/testsetup", diff --git a/distsql/distsql.go b/distsql/distsql.go index a648732676d62..053f9ee9f8fe3 100644 --- a/distsql/distsql.go +++ b/distsql/distsql.go @@ -26,7 +26,6 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" - "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/tracing" @@ -46,7 +45,6 @@ func GenSelectResultFromResponse(sctx sessionctx.Context, fieldTypes []*types.Fi rowLen: len(fieldTypes), fieldTypes: fieldTypes, ctx: sctx, - feedback: statistics.NewQueryFeedback(0, nil, 0, false), copPlanIDs: planIDs, rootPlanID: rootID, storeType: kv.TiFlash, @@ -55,7 +53,7 @@ func GenSelectResultFromResponse(sctx sessionctx.Context, fieldTypes []*types.Fi // Select sends a DAG request, returns SelectResult. // In kvReq, KeyRanges is required, Concurrency/KeepOrder/Desc/IsolationLevel/Priority are optional. -func Select(ctx context.Context, sctx sessionctx.Context, kvReq *kv.Request, fieldTypes []*types.FieldType, fb *statistics.QueryFeedback) (SelectResult, error) { +func Select(ctx context.Context, sctx sessionctx.Context, kvReq *kv.Request, fieldTypes []*types.FieldType) (SelectResult, error) { r, ctx := tracing.StartRegionEx(ctx, "distsql.Select") defer r.End() @@ -109,7 +107,6 @@ func Select(ctx context.Context, sctx sessionctx.Context, kvReq *kv.Request, fie rowLen: len(fieldTypes), fieldTypes: fieldTypes, ctx: sctx, - feedback: fb, sqlType: label, memTracker: kvReq.MemTracker, storeType: kvReq.StoreType, @@ -144,8 +141,8 @@ func SetTiFlashConfVarsInContext(ctx context.Context, sctx sessionctx.Context) c // The difference from Select is that SelectWithRuntimeStats will set copPlanIDs into selectResult, // which can help selectResult to collect runtime stats. func SelectWithRuntimeStats(ctx context.Context, sctx sessionctx.Context, kvReq *kv.Request, - fieldTypes []*types.FieldType, fb *statistics.QueryFeedback, copPlanIDs []int, rootPlanID int) (SelectResult, error) { - sr, err := Select(ctx, sctx, kvReq, fieldTypes, fb) + fieldTypes []*types.FieldType, copPlanIDs []int, rootPlanID int) (SelectResult, error) { + sr, err := Select(ctx, sctx, kvReq, fieldTypes) if err != nil { return nil, err } @@ -173,7 +170,6 @@ func Analyze(ctx context.Context, client kv.Client, kvReq *kv.Request, vars inte result := &selectResult{ label: "analyze", resp: resp, - feedback: statistics.NewQueryFeedback(0, nil, 0, false), sqlType: label, storeType: kvReq.StoreType, } @@ -191,7 +187,6 @@ func Checksum(ctx context.Context, client kv.Client, kvReq *kv.Request, vars int result := &selectResult{ label: "checksum", resp: resp, - feedback: statistics.NewQueryFeedback(0, nil, 0, false), sqlType: metrics.LblGeneral, storeType: kvReq.StoreType, } diff --git a/distsql/distsql_test.go b/distsql/distsql_test.go index 9fbee11a07a74..49ed70c2b672d 100644 --- a/distsql/distsql_test.go +++ b/distsql/distsql_test.go @@ -26,7 +26,6 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" - "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/codec" @@ -338,7 +337,7 @@ func createSelectNormalByBenchmarkTest(batch, totalRows int, ctx sessionctx.Cont // Test Next. var response SelectResult - response, _ = Select(context.TODO(), ctx, request, colTypes, statistics.NewQueryFeedback(0, nil, 0, false)) + response, _ = Select(context.TODO(), ctx, request, colTypes) result, _ := response.(*selectResult) resp, _ := result.resp.(*mockResponse) @@ -413,9 +412,9 @@ func createSelectNormal(t *testing.T, batch, totalRows int, planIDs []int, sctx // Test Next. var response SelectResult if planIDs == nil { - response, err = Select(context.TODO(), sctx, request, colTypes, statistics.NewQueryFeedback(0, nil, 0, false)) + response, err = Select(context.TODO(), sctx, request, colTypes) } else { - response, err = SelectWithRuntimeStats(context.TODO(), sctx, request, colTypes, statistics.NewQueryFeedback(0, nil, 0, false), planIDs, 1) + response, err = SelectWithRuntimeStats(context.TODO(), sctx, request, colTypes, planIDs, 1) } require.NoError(t, err) diff --git a/distsql/request_builder.go b/distsql/request_builder.go index 0d7a5912c1c0d..d165edc8edc74 100644 --- a/distsql/request_builder.go +++ b/distsql/request_builder.go @@ -28,7 +28,6 @@ import ( "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" - "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/codec" @@ -86,9 +85,9 @@ func (builder *RequestBuilder) SetMemTracker(tracker *memory.Tracker) *RequestBu // to "KeyRanges" firstly. // Note this function should be deleted or at least not exported, but currently // br refers it, so have to keep it. -func (builder *RequestBuilder) SetTableRanges(tid int64, tableRanges []*ranger.Range, fb *statistics.QueryFeedback) *RequestBuilder { +func (builder *RequestBuilder) SetTableRanges(tid int64, tableRanges []*ranger.Range) *RequestBuilder { if builder.err == nil { - builder.Request.KeyRanges = kv.NewNonParitionedKeyRanges(TableRangesToKVRanges(tid, tableRanges, fb)) + builder.Request.KeyRanges = kv.NewNonParitionedKeyRanges(TableRangesToKVRanges(tid, tableRanges)) } return builder } @@ -97,7 +96,7 @@ func (builder *RequestBuilder) SetTableRanges(tid int64, tableRanges []*ranger.R // "ranges" to "KeyRanges" firstly. func (builder *RequestBuilder) SetIndexRanges(sc *stmtctx.StatementContext, tid, idxID int64, ranges []*ranger.Range) *RequestBuilder { if builder.err == nil { - builder.Request.KeyRanges, builder.err = IndexRangesToKVRanges(sc, tid, idxID, ranges, nil) + builder.Request.KeyRanges, builder.err = IndexRangesToKVRanges(sc, tid, idxID, ranges) } return builder } @@ -106,24 +105,24 @@ func (builder *RequestBuilder) SetIndexRanges(sc *stmtctx.StatementContext, tid, // "ranges" to "KeyRanges" firstly. func (builder *RequestBuilder) SetIndexRangesForTables(sc *stmtctx.StatementContext, tids []int64, idxID int64, ranges []*ranger.Range) *RequestBuilder { if builder.err == nil { - builder.Request.KeyRanges, builder.err = IndexRangesToKVRangesForTables(sc, tids, idxID, ranges, nil) + builder.Request.KeyRanges, builder.err = IndexRangesToKVRangesForTables(sc, tids, idxID, ranges) } return builder } // SetHandleRanges sets "KeyRanges" for "kv.Request" by converting table handle range // "ranges" to "KeyRanges" firstly. -func (builder *RequestBuilder) SetHandleRanges(sc *stmtctx.StatementContext, tid int64, isCommonHandle bool, ranges []*ranger.Range, fb *statistics.QueryFeedback) *RequestBuilder { - builder = builder.SetHandleRangesForTables(sc, []int64{tid}, isCommonHandle, ranges, fb) +func (builder *RequestBuilder) SetHandleRanges(sc *stmtctx.StatementContext, tid int64, isCommonHandle bool, ranges []*ranger.Range) *RequestBuilder { + builder = builder.SetHandleRangesForTables(sc, []int64{tid}, isCommonHandle, ranges) builder.err = builder.Request.KeyRanges.SetToNonPartitioned() return builder } // SetHandleRangesForTables sets "KeyRanges" for "kv.Request" by converting table handle range // "ranges" to "KeyRanges" firstly for multiple tables. -func (builder *RequestBuilder) SetHandleRangesForTables(sc *stmtctx.StatementContext, tid []int64, isCommonHandle bool, ranges []*ranger.Range, fb *statistics.QueryFeedback) *RequestBuilder { +func (builder *RequestBuilder) SetHandleRangesForTables(sc *stmtctx.StatementContext, tid []int64, isCommonHandle bool, ranges []*ranger.Range) *RequestBuilder { if builder.err == nil { - builder.Request.KeyRanges, builder.err = TableHandleRangesToKVRanges(sc, tid, isCommonHandle, ranges, fb) + builder.Request.KeyRanges, builder.err = TableHandleRangesToKVRanges(sc, tid, isCommonHandle, ranges) } return builder } @@ -435,9 +434,9 @@ func (builder *RequestBuilder) SetConnID(connID uint64) *RequestBuilder { } // TableHandleRangesToKVRanges convert table handle ranges to "KeyRanges" for multiple tables. -func TableHandleRangesToKVRanges(sc *stmtctx.StatementContext, tid []int64, isCommonHandle bool, ranges []*ranger.Range, fb *statistics.QueryFeedback) (*kv.KeyRanges, error) { +func TableHandleRangesToKVRanges(sc *stmtctx.StatementContext, tid []int64, isCommonHandle bool, ranges []*ranger.Range) (*kv.KeyRanges, error) { if !isCommonHandle { - return tablesRangesToKVRanges(tid, ranges, fb), nil + return tablesRangesToKVRanges(tid, ranges), nil } return CommonHandleRangesToKVRanges(sc, tid, ranges) } @@ -445,45 +444,16 @@ func TableHandleRangesToKVRanges(sc *stmtctx.StatementContext, tid []int64, isCo // TableRangesToKVRanges converts table ranges to "KeyRange". // Note this function should not be exported, but currently // br refers to it, so have to keep it. -func TableRangesToKVRanges(tid int64, ranges []*ranger.Range, fb *statistics.QueryFeedback) []kv.KeyRange { +func TableRangesToKVRanges(tid int64, ranges []*ranger.Range) []kv.KeyRange { if len(ranges) == 0 { return []kv.KeyRange{} } - return tablesRangesToKVRanges([]int64{tid}, ranges, fb).FirstPartitionRange() + return tablesRangesToKVRanges([]int64{tid}, ranges).FirstPartitionRange() } // tablesRangesToKVRanges converts table ranges to "KeyRange". -func tablesRangesToKVRanges(tids []int64, ranges []*ranger.Range, fb *statistics.QueryFeedback) *kv.KeyRanges { - if fb == nil || fb.Hist == nil { - return tableRangesToKVRangesWithoutSplit(tids, ranges) - } - // The following codes are deprecated since the feedback is deprecated. - krs := make([]kv.KeyRange, 0, len(ranges)) - feedbackRanges := make([]*ranger.Range, 0, len(ranges)) - for _, ran := range ranges { - low := codec.EncodeInt(nil, ran.LowVal[0].GetInt64()) - high := codec.EncodeInt(nil, ran.HighVal[0].GetInt64()) - if ran.LowExclude { - low = kv.Key(low).PrefixNext() - } - // If this range is split by histogram, then the high val will equal to one bucket's upper bound, - // since we need to guarantee each range falls inside the exactly one bucket, `PrefixNext` will make the - // high value greater than upper bound, so we store the range here. - r := &ranger.Range{LowVal: []types.Datum{types.NewBytesDatum(low)}, - HighVal: []types.Datum{types.NewBytesDatum(high)}, Collators: collate.GetBinaryCollatorSlice(1)} - feedbackRanges = append(feedbackRanges, r) - - if !ran.HighExclude { - high = kv.Key(high).PrefixNext() - } - for _, tid := range tids { - startKey := tablecodec.EncodeRowKey(tid, low) - endKey := tablecodec.EncodeRowKey(tid, high) - krs = append(krs, kv.KeyRange{StartKey: startKey, EndKey: endKey}) - } - } - fb.StoreRanges(feedbackRanges) - return kv.NewNonParitionedKeyRanges(krs) +func tablesRangesToKVRanges(tids []int64, ranges []*ranger.Range) *kv.KeyRanges { + return tableRangesToKVRangesWithoutSplit(tids, ranges) } func tableRangesToKVRangesWithoutSplit(tids []int64, ranges []*ranger.Range) *kv.KeyRanges { @@ -656,14 +626,14 @@ func PartitionHandlesToKVRanges(handles []kv.Handle) ([]kv.KeyRange, []int) { } // IndexRangesToKVRanges converts index ranges to "KeyRange". -func IndexRangesToKVRanges(sc *stmtctx.StatementContext, tid, idxID int64, ranges []*ranger.Range, fb *statistics.QueryFeedback) (*kv.KeyRanges, error) { - return IndexRangesToKVRangesWithInterruptSignal(sc, tid, idxID, ranges, fb, nil, nil) +func IndexRangesToKVRanges(sc *stmtctx.StatementContext, tid, idxID int64, ranges []*ranger.Range) (*kv.KeyRanges, error) { + return IndexRangesToKVRangesWithInterruptSignal(sc, tid, idxID, ranges, nil, nil) } // IndexRangesToKVRangesWithInterruptSignal converts index ranges to "KeyRange". // The process can be interrupted by set `interruptSignal` to true. -func IndexRangesToKVRangesWithInterruptSignal(sc *stmtctx.StatementContext, tid, idxID int64, ranges []*ranger.Range, fb *statistics.QueryFeedback, memTracker *memory.Tracker, interruptSignal *atomic.Value) (*kv.KeyRanges, error) { - keyRanges, err := indexRangesToKVRangesForTablesWithInterruptSignal(sc, []int64{tid}, idxID, ranges, fb, memTracker, interruptSignal) +func IndexRangesToKVRangesWithInterruptSignal(sc *stmtctx.StatementContext, tid, idxID int64, ranges []*ranger.Range, memTracker *memory.Tracker, interruptSignal *atomic.Value) (*kv.KeyRanges, error) { + keyRanges, err := indexRangesToKVRangesForTablesWithInterruptSignal(sc, []int64{tid}, idxID, ranges, memTracker, interruptSignal) if err != nil { return nil, err } @@ -672,52 +642,14 @@ func IndexRangesToKVRangesWithInterruptSignal(sc *stmtctx.StatementContext, tid, } // IndexRangesToKVRangesForTables converts indexes ranges to "KeyRange". -func IndexRangesToKVRangesForTables(sc *stmtctx.StatementContext, tids []int64, idxID int64, ranges []*ranger.Range, fb *statistics.QueryFeedback) (*kv.KeyRanges, error) { - return indexRangesToKVRangesForTablesWithInterruptSignal(sc, tids, idxID, ranges, fb, nil, nil) +func IndexRangesToKVRangesForTables(sc *stmtctx.StatementContext, tids []int64, idxID int64, ranges []*ranger.Range) (*kv.KeyRanges, error) { + return indexRangesToKVRangesForTablesWithInterruptSignal(sc, tids, idxID, ranges, nil, nil) } // IndexRangesToKVRangesForTablesWithInterruptSignal converts indexes ranges to "KeyRange". // The process can be interrupted by set `interruptSignal` to true. -func indexRangesToKVRangesForTablesWithInterruptSignal(sc *stmtctx.StatementContext, tids []int64, idxID int64, ranges []*ranger.Range, fb *statistics.QueryFeedback, memTracker *memory.Tracker, interruptSignal *atomic.Value) (*kv.KeyRanges, error) { - if fb == nil || fb.Hist == nil { - return indexRangesToKVWithoutSplit(sc, tids, idxID, ranges, memTracker, interruptSignal) - } - // The following code is non maintained since the feedback deprecated. - feedbackRanges := make([]*ranger.Range, 0, len(ranges)) - for _, ran := range ranges { - low, high, err := EncodeIndexKey(sc, ran) - if err != nil { - return nil, err - } - feedbackRanges = append(feedbackRanges, &ranger.Range{LowVal: []types.Datum{types.NewBytesDatum(low)}, - HighVal: []types.Datum{types.NewBytesDatum(high)}, LowExclude: false, HighExclude: true, Collators: collate.GetBinaryCollatorSlice(1)}) - } - feedbackRanges, ok := fb.Hist.SplitRange(sc, feedbackRanges, true) - if !ok { - fb.Invalidate() - } - krs := make([]kv.KeyRange, 0, len(feedbackRanges)) - for _, ran := range feedbackRanges { - low, high := ran.LowVal[0].GetBytes(), ran.HighVal[0].GetBytes() - if ran.LowExclude { - low = kv.Key(low).PrefixNext() - } - ran.LowVal[0].SetBytes(low) - // If this range is split by histogram, then the high val will equal to one bucket's upper bound, - // since we need to guarantee each range falls inside the exactly one bucket, `PrefixNext` will make the - // high value greater than upper bound, so we store the high value here. - ran.HighVal[0].SetBytes(high) - if !ran.HighExclude { - high = kv.Key(high).PrefixNext() - } - for _, tid := range tids { - startKey := tablecodec.EncodeIndexSeekKey(tid, idxID, low) - endKey := tablecodec.EncodeIndexSeekKey(tid, idxID, high) - krs = append(krs, kv.KeyRange{StartKey: startKey, EndKey: endKey}) - } - } - fb.StoreRanges(feedbackRanges) - return kv.NewNonParitionedKeyRanges(krs), nil +func indexRangesToKVRangesForTablesWithInterruptSignal(sc *stmtctx.StatementContext, tids []int64, idxID int64, ranges []*ranger.Range, memTracker *memory.Tracker, interruptSignal *atomic.Value) (*kv.KeyRanges, error) { + return indexRangesToKVWithoutSplit(sc, tids, idxID, ranges, memTracker, interruptSignal) } // CommonHandleRangesToKVRanges converts common handle ranges to "KeyRange". diff --git a/distsql/request_builder_test.go b/distsql/request_builder_test.go index 4db0435da9495..da10f5827156a 100644 --- a/distsql/request_builder_test.go +++ b/distsql/request_builder_test.go @@ -19,13 +19,10 @@ import ( "github.com/pingcap/tidb/domain/resourcegroup" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" - "github.com/pingcap/tidb/statistics" "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/memory" @@ -107,7 +104,7 @@ func TestTableRangesToKVRanges(t *testing.T) { }, } - actual := TableRangesToKVRanges(13, ranges, nil) + actual := TableRangesToKVRanges(13, ranges) expect := []kv.KeyRange{ { StartKey: kv.Key{0x74, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xd, 0x5f, 0x72, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x1}, @@ -192,7 +189,7 @@ func TestIndexRangesToKVRanges(t *testing.T) { }, } - actual, err := IndexRangesToKVRanges(new(stmtctx.StatementContext), 12, 15, ranges, nil) + actual, err := IndexRangesToKVRanges(new(stmtctx.StatementContext), 12, 15, ranges) require.NoError(t, err) for i := range actual.FirstPartitionRange() { require.Equal(t, expect[i], actual.FirstPartitionRange()[i]) @@ -233,7 +230,7 @@ func TestRequestBuilder1(t *testing.T) { }, } - actual, err := (&RequestBuilder{}).SetHandleRanges(nil, 12, false, ranges, nil). + actual, err := (&RequestBuilder{}).SetHandleRanges(nil, 12, false, ranges). SetDAGRequest(&tipb.DAGRequest{}). SetDesc(false). SetKeepOrder(false). @@ -641,8 +638,7 @@ func TestTableRangesToKVRangesWithFbs(t *testing.T) { Collators: collate.GetBinaryCollatorSlice(1), }, } - fb := newTestFb() - actual := TableRangesToKVRanges(0, ranges, fb) + actual := TableRangesToKVRanges(0, ranges) expect := []kv.KeyRange{ { StartKey: kv.Key{0x74, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x5f, 0x72, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x1}, @@ -663,8 +659,7 @@ func TestIndexRangesToKVRangesWithFbs(t *testing.T) { Collators: collate.GetBinaryCollatorSlice(1), }, } - fb := newTestFb() - actual, err := IndexRangesToKVRanges(new(stmtctx.StatementContext), 0, 0, ranges, fb) + actual, err := IndexRangesToKVRanges(new(stmtctx.StatementContext), 0, 0, ranges) require.NoError(t, err) expect := []kv.KeyRange{ { @@ -726,18 +721,3 @@ func getExpectedRanges(tid int64, hrs []*handleRange) []kv.KeyRange { } return krs } - -func newTestFb() *statistics.QueryFeedback { - hist := statistics.NewHistogram(1, 30, 30, 0, types.NewFieldType(mysql.TypeLonglong), chunk.InitialCapacity, 0) - for i := 0; i < 10; i++ { - hist.Bounds.AppendInt64(0, int64(i)) - hist.Bounds.AppendInt64(0, int64(i+2)) - hist.Buckets = append(hist.Buckets, statistics.Bucket{Repeat: 10, Count: int64(i + 30)}) - } - fb := statistics.NewQueryFeedback(0, hist, 0, false) - lower, upper := types.NewIntDatum(2), types.NewIntDatum(3) - fb.Feedback = []statistics.Feedback{ - {Lower: &lower, Upper: &upper, Count: 1, Repeat: 1}, - } - return fb -} diff --git a/distsql/select_result.go b/distsql/select_result.go index c0ad0f6cdf308..2287cc439e21c 100644 --- a/distsql/select_result.go +++ b/distsql/select_result.go @@ -34,7 +34,6 @@ import ( "github.com/pingcap/tidb/planner/util" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" - "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/store/copr" "github.com/pingcap/tidb/telemetry" "github.com/pingcap/tidb/types" @@ -300,7 +299,6 @@ type selectResult struct { respChkIdx int respChunkDecoder *chunk.Decoder - feedback *statistics.QueryFeedback partialCount int64 // number of partial results. sqlType string @@ -398,9 +396,7 @@ func (r *selectResult) fetchResp(ctx context.Context) error { for _, warning := range r.selectResp.Warnings { sc.AppendWarning(dbterror.ClassTiKV.Synthesize(terror.ErrCode(warning.Code), warning.Msg)) } - if r.feedback != nil { - r.feedback.Update(resultSubset.GetStartKey(), r.selectResp.OutputCounts, r.selectResp.Ndvs) - } + r.partialCount++ hasStats, ok := resultSubset.(CopRuntimeStats) @@ -451,7 +447,6 @@ func (r *selectResult) NextRaw(ctx context.Context) (data []byte, err error) { resultSubset, err := r.resp.Next(ctx) r.partialCount++ - r.feedback.Invalidate() if resultSubset != nil && err == nil { data = resultSubset.GetData() } @@ -629,9 +624,6 @@ func (r *selectResult) memConsume(bytes int64) { // Close closes selectResult. func (r *selectResult) Close() error { - if r.feedback.Actual() >= 0 { - metrics.DistSQLScanKeysHistogram.Observe(float64(r.feedback.Actual())) - } metrics.DistSQLPartialCountHistogram.Observe(float64(r.partialCount)) respSize := atomic.SwapInt64(&r.selectRespSize, 0) if respSize > 0 { diff --git a/domain/domain.go b/domain/domain.go index 115774c153531..fa161f85aef07 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -2383,16 +2383,12 @@ func (do *Domain) updateStatsWorker(ctx sessionctx.Context, owner owner.Manager) lease := do.statsLease deltaUpdateTicker := time.NewTicker(20 * lease) gcStatsTicker := time.NewTicker(100 * lease) - dumpFeedbackTicker := time.NewTicker(200 * lease) - loadFeedbackTicker := time.NewTicker(5 * lease) loadLockedTablesTicker := time.NewTicker(5 * lease) dumpColStatsUsageTicker := time.NewTicker(100 * lease) readMemTricker := time.NewTicker(memory.ReadMemInterval) statsHandle := do.StatsHandle() defer func() { dumpColStatsUsageTicker.Stop() - loadFeedbackTicker.Stop() - dumpFeedbackTicker.Stop() gcStatsTicker.Stop() deltaUpdateTicker.Stop() readMemTricker.Stop() @@ -2417,26 +2413,11 @@ func (do *Domain) updateStatsWorker(ctx sessionctx.Context, owner owner.Manager) if err != nil { logutil.BgLogger().Debug("dump stats delta failed", zap.Error(err)) } - statsHandle.UpdateErrorRate(do.InfoSchema()) - case <-loadFeedbackTicker.C: - statsHandle.UpdateStatsByLocalFeedback(do.InfoSchema()) - if !owner.IsOwner() { - continue - } - err := statsHandle.HandleUpdateStats(do.InfoSchema()) - if err != nil { - logutil.BgLogger().Debug("update stats using feedback failed", zap.Error(err)) - } case <-loadLockedTablesTicker.C: err := statsHandle.LoadLockedTables() if err != nil { logutil.BgLogger().Debug("load locked table failed", zap.Error(err)) } - case <-dumpFeedbackTicker.C: - err := statsHandle.DumpStatsFeedbackToKV() - if err != nil { - logutil.BgLogger().Debug("dump stats feedback failed", zap.Error(err)) - } case <-gcStatsTicker.C: if !owner.IsOwner() { continue diff --git a/executor/admin.go b/executor/admin.go index 0a5b087c94f21..da0233acc2659 100644 --- a/executor/admin.go +++ b/executor/admin.go @@ -29,7 +29,6 @@ import ( "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/terror" plannercore "github.com/pingcap/tidb/planner/core" - "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/tablecodec" @@ -135,7 +134,7 @@ func (e *CheckIndexRangeExec) Open(ctx context.Context) error { return err } - e.result, err = distsql.Select(ctx, e.Ctx(), kvReq, e.RetFieldTypes(), statistics.NewQueryFeedback(0, nil, 0, false)) + e.result, err = distsql.Select(ctx, e.Ctx(), kvReq, e.RetFieldTypes()) if err != nil { return err } @@ -290,7 +289,7 @@ func (e *RecoverIndexExec) buildTableScan(ctx context.Context, txn kv.Transactio // Actually, with limitCnt, the match datas maybe only in one region, so let the concurrency to be 1, // avoid unnecessary region scan. kvReq.Concurrency = 1 - result, err := distsql.Select(ctx, e.Ctx(), kvReq, e.columnsTypes(), statistics.NewQueryFeedback(0, nil, 0, false)) + result, err := distsql.Select(ctx, e.Ctx(), kvReq, e.columnsTypes()) if err != nil { return nil, err } @@ -795,7 +794,7 @@ func (e *CleanupIndexExec) buildIndexScan(ctx context.Context, txn kv.Transactio sc := e.Ctx().GetSessionVars().StmtCtx var builder distsql.RequestBuilder ranges := ranger.FullRange() - keyRanges, err := distsql.IndexRangesToKVRanges(sc, e.physicalID, e.index.Meta().ID, ranges, nil) + keyRanges, err := distsql.IndexRangesToKVRanges(sc, e.physicalID, e.index.Meta().ID, ranges) if err != nil { return nil, err } @@ -817,7 +816,7 @@ func (e *CleanupIndexExec) buildIndexScan(ctx context.Context, txn kv.Transactio } kvReq.Concurrency = 1 - result, err := distsql.Select(ctx, e.Ctx(), kvReq, e.getIdxColTypes(), statistics.NewQueryFeedback(0, nil, 0, false)) + result, err := distsql.Select(ctx, e.Ctx(), kvReq, e.getIdxColTypes()) if err != nil { return nil, err } diff --git a/executor/analyze_col.go b/executor/analyze_col.go index c7c7c0e0029e8..a47ebecce5651 100644 --- a/executor/analyze_col.go +++ b/executor/analyze_col.go @@ -108,7 +108,7 @@ func (e *AnalyzeColumnsExec) open(ranges []*ranger.Range) error { func (e *AnalyzeColumnsExec) buildResp(ranges []*ranger.Range) (distsql.SelectResult, error) { var builder distsql.RequestBuilder - reqBuilder := builder.SetHandleRangesForTables(e.ctx.GetSessionVars().StmtCtx, []int64{e.TableID.GetStatisticsID()}, e.handleCols != nil && !e.handleCols.IsInt(), ranges, nil) + reqBuilder := builder.SetHandleRangesForTables(e.ctx.GetSessionVars().StmtCtx, []int64{e.TableID.GetStatisticsID()}, e.handleCols != nil && !e.handleCols.IsInt(), ranges) builder.SetResourceGroupTagger(e.ctx.GetSessionVars().StmtCtx.GetResourceGroupTagger()) startTS := uint64(math.MaxUint64) isoLevel := kv.RC diff --git a/executor/analyze_idx.go b/executor/analyze_idx.go index bf30d4fd9749d..257127ef09c19 100644 --- a/executor/analyze_idx.go +++ b/executor/analyze_idx.go @@ -139,7 +139,7 @@ func (e *AnalyzeIndexExec) fetchAnalyzeResult(ranges []*ranger.Range, isNullRang var builder distsql.RequestBuilder var kvReqBuilder *distsql.RequestBuilder if e.isCommonHandle && e.idxInfo.Primary { - kvReqBuilder = builder.SetHandleRangesForTables(e.ctx.GetSessionVars().StmtCtx, []int64{e.tableID.GetStatisticsID()}, true, ranges, nil) + kvReqBuilder = builder.SetHandleRangesForTables(e.ctx.GetSessionVars().StmtCtx, []int64{e.tableID.GetStatisticsID()}, true, ranges) } else { kvReqBuilder = builder.SetIndexRangesForTables(e.ctx.GetSessionVars().StmtCtx, []int64{e.tableID.GetStatisticsID()}, e.idxInfo.ID, ranges) } diff --git a/executor/builder.go b/executor/builder.go index f43d8331784b9..d4315002463a5 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -3438,17 +3438,6 @@ func (b *executorBuilder) buildIndexNestedLoopHashJoin(v *plannercore.PhysicalIn return idxHash } -// containsLimit tests if the execs contains Limit because we do not know whether `Limit` has consumed all of its' source, -// so the feedback may not be accurate. -func containsLimit(execs []*tipb.Executor) bool { - for _, exec := range execs { - if exec.Limit != nil { - return true - } - } - return false -} - func buildNoRangeTableReader(b *executorBuilder, v *plannercore.PhysicalTableReader) (*TableReaderExecutor, error) { tablePlans := v.TablePlans if v.StoreType == kv.TiFlash { @@ -3499,20 +3488,7 @@ func buildNoRangeTableReader(b *executorBuilder, v *plannercore.PhysicalTableRea batchCop: v.ReadReqType == plannercore.BatchCop, } e.buildVirtualColumnInfo() - if containsLimit(dagReq.Executors) { - e.feedback = statistics.NewQueryFeedback(0, nil, 0, ts.Desc) - } else { - e.feedback = statistics.NewQueryFeedback(getFeedbackStatsTableID(e.Ctx(), tbl), ts.Hist, int64(ts.StatsCount()), ts.Desc) - } - collect := statistics.CollectFeedback(b.ctx.GetSessionVars().StmtCtx, e.feedback, len(ts.Ranges)) - // Do not collect the feedback when the table is the partition table. - if collect && tbl.Meta().Partition != nil { - collect = false - } - if !collect { - e.feedback.Invalidate() - } - e.dagPB.CollectRangeCounts = &collect + if v.StoreType == kv.TiDB && b.ctx.GetSessionVars().User != nil { // User info is used to do privilege check. It is only used in TiDB cluster memory table. e.dagPB.User = &tipb.UserIdentity{ @@ -3845,24 +3821,6 @@ func buildNoRangeIndexReader(b *executorBuilder, v *plannercore.PhysicalIndexRea plans: v.IndexPlans, outputColumns: v.OutputColumns, } - if containsLimit(dagReq.Executors) { - e.feedback = statistics.NewQueryFeedback(0, nil, 0, is.Desc) - } else { - tblID := e.physicalTableID - if b.ctx.GetSessionVars().StmtCtx.UseDynamicPartitionPrune() { - tblID = e.table.Meta().ID - } - e.feedback = statistics.NewQueryFeedback(tblID, is.Hist, int64(is.StatsCount()), is.Desc) - } - collect := statistics.CollectFeedback(b.ctx.GetSessionVars().StmtCtx, e.feedback, len(is.Ranges)) - // Do not collect the feedback when the table is the partition table. - if collect && tbl.Meta().Partition != nil { - collect = false - } - if !collect { - e.feedback.Invalidate() - } - e.dagPB.CollectRangeCounts = &collect for _, col := range v.OutputColumns { dagReq.OutputOffsets = append(dagReq.OutputOffsets, uint32(col.Index)) @@ -4052,23 +4010,6 @@ func buildNoRangeIndexLookUpReader(b *executorBuilder, v *plannercore.PhysicalIn avgRowSize: v.GetAvgTableRowSize(), } - if containsLimit(indexReq.Executors) { - e.feedback = statistics.NewQueryFeedback(0, nil, 0, is.Desc) - } else { - e.feedback = statistics.NewQueryFeedback(getFeedbackStatsTableID(e.Ctx(), tbl), is.Hist, int64(is.StatsCount()), is.Desc) - } - // Do not collect the feedback for table request. - collectTable := false - e.tableRequest.CollectRangeCounts = &collectTable - collectIndex := statistics.CollectFeedback(b.ctx.GetSessionVars().StmtCtx, e.feedback, len(is.Ranges)) - // Do not collect the feedback when the table is the partition table. - if collectIndex && tbl.Meta().GetPartitionInfo() != nil { - collectIndex = false - } - if !collectIndex { - e.feedback.Invalidate() - } - e.dagPB.CollectRangeCounts = &collectIndex if v.ExtraHandleCol != nil { e.handleIdx = append(e.handleIdx, v.ExtraHandleCol.Index) e.handleCols = []*expression.Column{v.ExtraHandleCol} @@ -4163,7 +4104,6 @@ func buildNoRangeIndexMergeReader(b *executorBuilder, v *plannercore.PhysicalInd partialDataSizes := make([]float64, 0, partialPlanCount) indexes := make([]*model.IndexInfo, 0, partialPlanCount) descs := make([]bool, 0, partialPlanCount) - feedbacks := make([]*statistics.QueryFeedback, 0, partialPlanCount) ts := v.TablePlans[0].(*plannercore.PhysicalTableScan) isCorColInPartialFilters := make([]bool, 0, partialPlanCount) isCorColInPartialAccess := make([]bool, 0, partialPlanCount) @@ -4171,10 +4111,6 @@ func buildNoRangeIndexMergeReader(b *executorBuilder, v *plannercore.PhysicalInd var tempReq *tipb.DAGRequest var err error - feedback := statistics.NewQueryFeedback(0, nil, 0, ts.Desc) - feedback.Invalidate() - feedbacks = append(feedbacks, feedback) - if is, ok := v.PartialPlans[i][0].(*plannercore.PhysicalIndexScan); ok { tempReq, err = buildIndexReq(b.ctx, is.Index.Columns, ts.HandleCols.NumCols(), v.PartialPlans[i]) descs = append(descs, is.Desc) @@ -4225,7 +4161,6 @@ func buildNoRangeIndexMergeReader(b *executorBuilder, v *plannercore.PhysicalInd partialNetDataSizes: partialDataSizes, dataAvgRowSize: v.GetAvgTableRowSize(), dataReaderBuilder: readerBuilder, - feedbacks: feedbacks, paging: paging, handleCols: v.HandleCols, isCorColInPartialFilters: isCorColInPartialFilters, @@ -4545,7 +4480,7 @@ func (h kvRangeBuilderFromRangeAndPartition) buildKeyRangeSeparately(ranges []*r if len(ranges) == 0 { continue } - kvRange, err := distsql.TableHandleRangesToKVRanges(h.sctx.GetSessionVars().StmtCtx, []int64{pid}, meta != nil && meta.IsCommonHandle, ranges, nil) + kvRange, err := distsql.TableHandleRangesToKVRanges(h.sctx.GetSessionVars().StmtCtx, []int64{pid}, meta != nil && meta.IsCommonHandle, ranges) if err != nil { return nil, nil, err } @@ -4562,7 +4497,7 @@ func (h kvRangeBuilderFromRangeAndPartition) buildKeyRange(ranges []*ranger.Rang for i, p := range h.partitions { pid := p.GetPhysicalID() meta := p.Meta() - kvRange, err := distsql.TableHandleRangesToKVRanges(h.sctx.GetSessionVars().StmtCtx, []int64{pid}, meta != nil && meta.IsCommonHandle, ranges, nil) + kvRange, err := distsql.TableHandleRangesToKVRanges(h.sctx.GetSessionVars().StmtCtx, []int64{pid}, meta != nil && meta.IsCommonHandle, ranges) if err != nil { return nil, err } @@ -4616,7 +4551,7 @@ func (builder *dataReaderBuilder) buildTableReaderBase(ctx context.Context, e *T } e.kvRanges = kvReq.KeyRanges.AppendSelfTo(e.kvRanges) e.resultHandler = &tableResultHandler{} - result, err := builder.SelectResult(ctx, builder.ctx, kvReq, retTypes(e), e.feedback, getPhysicalPlanIDs(e.plans), e.ID()) + result, err := builder.SelectResult(ctx, builder.ctx, kvReq, retTypes(e), getPhysicalPlanIDs(e.plans), e.ID()) if err != nil { return nil, err } @@ -4900,7 +4835,7 @@ func buildKvRangesForIndexJoin(ctx sessionctx.Context, tableID, indexID int64, l if indexID == -1 { tmpKvRanges, err = distsql.CommonHandleRangesToKVRanges(sc, []int64{tableID}, ranges) } else { - tmpKvRanges, err = distsql.IndexRangesToKVRangesWithInterruptSignal(sc, tableID, indexID, ranges, nil, memTracker, interruptSignal) + tmpKvRanges, err = distsql.IndexRangesToKVRangesWithInterruptSignal(sc, tableID, indexID, ranges, memTracker, interruptSignal) } if err != nil { return nil, err @@ -4945,7 +4880,7 @@ func buildKvRangesForIndexJoin(ctx sessionctx.Context, tableID, indexID int64, l tmpKeyRanges, err := distsql.CommonHandleRangesToKVRanges(ctx.GetSessionVars().StmtCtx, []int64{tableID}, tmpDatumRanges) return tmpKeyRanges.FirstPartitionRange(), err } - tmpKeyRanges, err := distsql.IndexRangesToKVRangesWithInterruptSignal(ctx.GetSessionVars().StmtCtx, tableID, indexID, tmpDatumRanges, nil, memTracker, interruptSignal) + tmpKeyRanges, err := distsql.IndexRangesToKVRangesWithInterruptSignal(ctx.GetSessionVars().StmtCtx, tableID, indexID, tmpDatumRanges, memTracker, interruptSignal) return tmpKeyRanges.FirstPartitionRange(), err } @@ -5356,13 +5291,6 @@ func getPhysicalTableID(t table.Table) int64 { return t.Meta().ID } -func getFeedbackStatsTableID(ctx sessionctx.Context, t table.Table) int64 { - if p, ok := t.(table.PhysicalTable); ok && !ctx.GetSessionVars().StmtCtx.UseDynamicPartitionPrune() { - return p.GetPhysicalID() - } - return t.Meta().ID -} - func (b *executorBuilder) buildAdminShowTelemetry(v *plannercore.AdminShowTelemetry) exec.Executor { return &AdminShowTelemetryExec{BaseExecutor: exec.NewBaseExecutor(b.ctx, v.Schema(), v.ID())} } diff --git a/executor/checksum.go b/executor/checksum.go index decc77943c8fa..e367ab7862776 100644 --- a/executor/checksum.go +++ b/executor/checksum.go @@ -243,7 +243,7 @@ func (c *checksumContext) buildTableRequest(ctx sessionctx.Context, tableID int6 var builder distsql.RequestBuilder builder.SetResourceGroupTagger(ctx.GetSessionVars().StmtCtx.GetResourceGroupTagger()) - return builder.SetHandleRanges(ctx.GetSessionVars().StmtCtx, tableID, c.TableInfo.IsCommonHandle, ranges, nil). + return builder.SetHandleRanges(ctx.GetSessionVars().StmtCtx, tableID, c.TableInfo.IsCommonHandle, ranges). SetChecksumRequest(checksum). SetStartTS(c.StartTs). SetConcurrency(ctx.GetSessionVars().DistSQLScanConcurrency()). diff --git a/executor/distsql.go b/executor/distsql.go index 06075bb60ed48..71fc5744e889b 100644 --- a/executor/distsql.go +++ b/executor/distsql.go @@ -42,7 +42,6 @@ import ( plannerutil "github.com/pingcap/tidb/planner/util" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" - "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/tablecodec" @@ -193,8 +192,7 @@ type IndexReaderExecutor struct { // outputColumns are only required by union scan. outputColumns []*expression.Column - feedback *statistics.QueryFeedback - paging bool + paging bool keepOrder bool desc bool @@ -235,7 +233,6 @@ func (e *IndexReaderExecutor) Close() (err error) { if e.dummy { return nil } - e.Ctx().StoreQueryFeedback(e.feedback) return err } @@ -246,11 +243,7 @@ func (e *IndexReaderExecutor) Next(ctx context.Context, req *chunk.Chunk) error return nil } - err := e.result.Next(ctx, req) - if err != nil { - e.feedback.Invalidate() - } - return err + return e.result.Next(ctx, req) } // TODO: cleanup this method. @@ -262,7 +255,7 @@ func (e *IndexReaderExecutor) buildKeyRanges(sc *stmtctx.StatementContext, range if e.index.ID == -1 { rRanges, err = distsql.CommonHandleRangesToKVRanges(sc, []int64{physicalID}, ranges) } else { - rRanges, err = distsql.IndexRangesToKVRanges(sc, physicalID, e.index.ID, ranges, e.feedback) + rRanges, err = distsql.IndexRangesToKVRanges(sc, physicalID, e.index.ID, ranges) } return rRanges.FirstPartitionRange(), err } @@ -390,12 +383,10 @@ func (e *IndexReaderExecutor) open(ctx context.Context, kvRanges []kv.KeyRange) if e.byItems == nil || len(e.partitions) <= 1 { kvReq, err := e.buildKVReq(kvRanges) if err != nil { - e.feedback.Invalidate() return err } - e.result, err = e.SelectResult(ctx, e.Ctx(), kvReq, retTypes(e), e.feedback, getPhysicalPlanIDs(e.plans), e.ID()) + e.result, err = e.SelectResult(ctx, e.Ctx(), kvReq, retTypes(e), getPhysicalPlanIDs(e.plans), e.ID()) if err != nil { - e.feedback.Invalidate() return err } } else { @@ -403,16 +394,14 @@ func (e *IndexReaderExecutor) open(ctx context.Context, kvRanges []kv.KeyRange) for _, kvRange := range kvRanges { kvReq, err := e.buildKVReq([]kv.KeyRange{kvRange}) if err != nil { - e.feedback.Invalidate() return err } kvReqs = append(kvReqs, kvReq) } var results []distsql.SelectResult for _, kvReq := range kvReqs { - result, err := e.SelectResult(ctx, e.Ctx(), kvReq, retTypes(e), e.feedback, getPhysicalPlanIDs(e.plans), e.ID()) + result, err := e.SelectResult(ctx, e.Ctx(), kvReq, retTypes(e), getPhysicalPlanIDs(e.plans), e.ID()) if err != nil { - e.feedback.Invalidate() return err } results = append(results, result) @@ -457,7 +446,6 @@ type IndexLookUpExecutor struct { resultCh chan *lookupTableTask resultCurr *lookupTableTask - feedback *statistics.QueryFeedback // memTracker is used to track the memory usage of this executor. memTracker *memory.Tracker @@ -527,7 +515,6 @@ func (e *IndexLookUpExecutor) Open(ctx context.Context) error { } err = e.buildTableKeyRanges() if err != nil { - e.feedback.Invalidate() return err } @@ -536,17 +523,12 @@ func (e *IndexLookUpExecutor) Open(ctx context.Context) error { return nil } - err = e.open(ctx) - if err != nil { - e.feedback.Invalidate() - } - return err + return e.open(ctx) } func (e *IndexLookUpExecutor) buildTableKeyRanges() (err error) { sc := e.Ctx().GetSessionVars().StmtCtx if e.partitionTableMode { - e.feedback.Invalidate() // feedback for partition tables is not ready e.partitionKVRanges = make([][]kv.KeyRange, 0, len(e.prunedPartitions)) for _, p := range e.prunedPartitions { // TODO: prune and adjust e.ranges for each partition again, since not all e.ranges are suitable for all e.prunedPartitions. @@ -561,7 +543,7 @@ func (e *IndexLookUpExecutor) buildTableKeyRanges() (err error) { if e.index.ID == -1 { kvRange, err = distsql.CommonHandleRangesToKVRanges(sc, []int64{physicalID}, ranges) } else { - kvRange, err = distsql.IndexRangesToKVRanges(sc, physicalID, e.index.ID, ranges, e.feedback) + kvRange, err = distsql.IndexRangesToKVRanges(sc, physicalID, e.index.ID, ranges) } if err != nil { return err @@ -574,7 +556,7 @@ func (e *IndexLookUpExecutor) buildTableKeyRanges() (err error) { if e.index.ID == -1 { kvRanges, err = distsql.CommonHandleRangesToKVRanges(sc, []int64{physicalID}, e.ranges) } else { - kvRanges, err = distsql.IndexRangesToKVRanges(sc, physicalID, e.index.ID, e.ranges, e.feedback) + kvRanges, err = distsql.IndexRangesToKVRanges(sc, physicalID, e.index.ID, e.ranges) } e.kvRanges = kvRanges.FirstPartitionRange() } @@ -755,7 +737,7 @@ func (e *IndexLookUpExecutor) startIndexWorker(ctx context.Context, workCh chan< worker.syncErr(err) break } - result, err := distsql.SelectWithRuntimeStats(ctx, e.Ctx(), kvReq, tps, e.feedback, getPhysicalPlanIDs(e.idxPlans), idxID) + result, err := distsql.SelectWithRuntimeStats(ctx, e.Ctx(), kvReq, tps, getPhysicalPlanIDs(e.idxPlans), idxID) if err != nil { worker.syncErr(err) break @@ -769,17 +751,14 @@ func (e *IndexLookUpExecutor) startIndexWorker(ctx context.Context, workCh chan< results = []distsql.SelectResult{ssr} } ctx1, cancel := context.WithCancel(ctx) - fetchErr := worker.fetchHandles(ctx1, results) - if fetchErr != nil { // this error is synced in fetchHandles(), don't sync it again - e.feedback.Invalidate() - } + // this error is synced in fetchHandles(), don't sync it again + _ = worker.fetchHandles(ctx1, results) cancel() for _, result := range results { if err := result.Close(); err != nil { logutil.Logger(ctx).Error("close Select result failed", zap.Error(err)) } } - e.Ctx().StoreQueryFeedback(e.feedback) close(workCh) close(e.resultCh) e.idxWorkerWg.Done() @@ -827,7 +806,6 @@ func (e *IndexLookUpExecutor) buildTableReader(ctx context.Context, task *lookup readReplicaScope: e.readReplicaScope, isStaleness: e.isStaleness, columns: e.columns, - feedback: statistics.NewQueryFeedback(0, nil, 0, false), corColInFilter: e.corColInTblSide, plans: e.tblPlans, netDataSize: e.avgRowSize * float64(len(task.handles)), diff --git a/executor/index_merge_reader.go b/executor/index_merge_reader.go index faa49266d8628..4111b96db34f5 100644 --- a/executor/index_merge_reader.go +++ b/executor/index_merge_reader.go @@ -41,7 +41,6 @@ import ( plannercore "github.com/pingcap/tidb/planner/core" plannerutil "github.com/pingcap/tidb/planner/util" "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" @@ -122,7 +121,6 @@ type IndexMergeReaderExecutor struct { resultCh chan *indexMergeTableTask resultCurr *indexMergeTableTask - feedbacks []*statistics.QueryFeedback // memTracker is used to track the memory usage of this executor. memTracker *memory.Tracker @@ -183,9 +181,6 @@ func (e *IndexMergeReaderExecutor) Open(_ context.Context) (err error) { return err } } else { - for _, feedback := range e.feedbacks { - feedback.Invalidate() // feedback is not ready for partition tables - } e.partitionKeyRanges = make([][][]kv.KeyRange, len(e.prunedPartitions)) for i, p := range e.prunedPartitions { if e.partitionKeyRanges[i], err = e.buildKeyRangesForTable(p); err != nil { @@ -234,11 +229,11 @@ func (e *IndexMergeReaderExecutor) buildKeyRangesForTable(tbl table.Table) (rang _, ok := plan[0].(*plannercore.PhysicalIndexScan) if !ok { firstPartRanges, secondPartRanges := distsql.SplitRangesAcrossInt64Boundary(e.ranges[i], false, e.descs[i], tbl.Meta().IsCommonHandle) - firstKeyRanges, err := distsql.TableHandleRangesToKVRanges(sc, []int64{getPhysicalTableID(tbl)}, tbl.Meta().IsCommonHandle, firstPartRanges, nil) + firstKeyRanges, err := distsql.TableHandleRangesToKVRanges(sc, []int64{getPhysicalTableID(tbl)}, tbl.Meta().IsCommonHandle, firstPartRanges) if err != nil { return nil, err } - secondKeyRanges, err := distsql.TableHandleRangesToKVRanges(sc, []int64{getPhysicalTableID(tbl)}, tbl.Meta().IsCommonHandle, secondPartRanges, nil) + secondKeyRanges, err := distsql.TableHandleRangesToKVRanges(sc, []int64{getPhysicalTableID(tbl)}, tbl.Meta().IsCommonHandle, secondPartRanges) if err != nil { return nil, err } @@ -246,7 +241,7 @@ func (e *IndexMergeReaderExecutor) buildKeyRangesForTable(tbl table.Table) (rang ranges = append(ranges, keyRanges) continue } - keyRange, err := distsql.IndexRangesToKVRanges(sc, getPhysicalTableID(tbl), e.indexes[i].ID, e.ranges[i], e.feedbacks[i]) + keyRange, err := distsql.IndexRangesToKVRanges(sc, getPhysicalTableID(tbl), e.indexes[i].ID, e.ranges[i]) if err != nil { return nil, err } @@ -418,7 +413,7 @@ func (e *IndexMergeReaderExecutor) startPartialIndexWorker(ctx context.Context, syncErr(ctx, e.finished, fetchCh, err) return } - result, err := distsql.SelectWithRuntimeStats(ctx, e.Ctx(), kvReq, tps, e.feedbacks[workID], getPhysicalPlanIDs(e.partialPlans[workID]), e.getPartitalPlanID(workID)) + result, err := distsql.SelectWithRuntimeStats(ctx, e.Ctx(), kvReq, tps, getPhysicalPlanIDs(e.partialPlans[workID]), e.getPartitalPlanID(workID)) if err != nil { syncErr(ctx, e.finished, fetchCh, err) return @@ -433,12 +428,9 @@ func (e *IndexMergeReaderExecutor) startPartialIndexWorker(ctx context.Context, results = []distsql.SelectResult{ssr} } ctx1, cancel := context.WithCancel(ctx) - _, fetchErr := worker.fetchHandles(ctx1, results, exitCh, fetchCh, e.finished, e.handleCols, workID) - if fetchErr != nil { // this error is synced in fetchHandles(), don't sync it again - e.feedbacks[workID].Invalidate() - } + // this error is reported in fetchHandles(), so ignore it here. + _, _ = worker.fetchHandles(ctx1, results, exitCh, fetchCh, e.finished, e.handleCols, workID) cancel() - e.Ctx().StoreQueryFeedback(e.feedbacks[workID]) }, handleWorkerPanic(ctx, e.finished, fetchCh, nil, partialIndexWorkerType), ) @@ -473,7 +465,6 @@ func (e *IndexMergeReaderExecutor) startPartialTableWorker(ctx context.Context, txnScope: e.txnScope, readReplicaScope: e.readReplicaScope, isStaleness: e.isStaleness, - feedback: statistics.NewQueryFeedback(0, nil, 0, false), plans: e.partialPlans[workID], ranges: e.ranges[workID], netDataSize: e.partialNetDataSizes[workID], @@ -547,17 +538,13 @@ func (e *IndexMergeReaderExecutor) startPartialTableWorker(ctx context.Context, // fetch all handles from this table ctx1, cancel := context.WithCancel(ctx) _, fetchErr := worker.fetchHandles(ctx1, exitCh, fetchCh, e.finished, e.handleCols, parTblIdx, workID) - if fetchErr != nil { // this error is synced in fetchHandles, so don't sync it again - e.feedbacks[workID].Invalidate() - } - // release related resources cancel() tableReaderClosed = true if err = worker.tableReader.Close(); err != nil { logutil.Logger(ctx).Error("close Select result failed:", zap.Error(err)) } - e.Ctx().StoreQueryFeedback(e.feedbacks[workID]) + // this error is reported in fetchHandles(), so ignore it here. if fetchErr != nil { break } @@ -791,7 +778,6 @@ func (e *IndexMergeReaderExecutor) buildFinalTableReader(ctx context.Context, tb readReplicaScope: e.readReplicaScope, isStaleness: e.isStaleness, columns: e.columns, - feedback: statistics.NewQueryFeedback(0, nil, 0, false), plans: e.tblPlans, netDataSize: e.dataAvgRowSize * float64(len(handles)), } @@ -916,7 +902,6 @@ func (e *IndexMergeReaderExecutor) Close() error { e.processWorkerWg.Wait() e.finished = nil e.workerStarted = false - // TODO: how to store e.feedbacks return nil } diff --git a/executor/inspection_summary.go b/executor/inspection_summary.go index 30fc542a9898b..d105f890b76a0 100644 --- a/executor/inspection_summary.go +++ b/executor/inspection_summary.go @@ -272,12 +272,9 @@ var inspectionSummaryRules = map[string][]string{ "stats": { "tidb_statistics_auto_analyze_duration", "tidb_statistics_auto_analyze_ops", - "tidb_statistics_dump_feedback_ops", "tidb_statistics_fast_analyze_status", "tidb_statistics_pseudo_estimation_ops", - "tidb_statistics_significant_feedback", "tidb_statistics_stats_inaccuracy_rate", - "tidb_statistics_store_query_feedback_qps", "tidb_statistics_update_stats_ops", }, "gc": { diff --git a/executor/table_reader.go b/executor/table_reader.go index a4c015a174754..4130956ff7f85 100644 --- a/executor/table_reader.go +++ b/executor/table_reader.go @@ -36,7 +36,6 @@ import ( "github.com/pingcap/tidb/planner/util" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessiontxn" - "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" @@ -54,15 +53,15 @@ var _ exec.Executor = &TableReaderExecutor{} // selectResultHook is used to hack distsql.SelectWithRuntimeStats safely for testing. type selectResultHook struct { selectResultFunc func(ctx context.Context, sctx sessionctx.Context, kvReq *kv.Request, - fieldTypes []*types.FieldType, fb *statistics.QueryFeedback, copPlanIDs []int) (distsql.SelectResult, error) + fieldTypes []*types.FieldType, copPlanIDs []int) (distsql.SelectResult, error) } func (sr selectResultHook) SelectResult(ctx context.Context, sctx sessionctx.Context, kvReq *kv.Request, - fieldTypes []*types.FieldType, fb *statistics.QueryFeedback, copPlanIDs []int, rootPlanID int) (distsql.SelectResult, error) { + fieldTypes []*types.FieldType, copPlanIDs []int, rootPlanID int) (distsql.SelectResult, error) { if sr.selectResultFunc == nil { - return distsql.SelectWithRuntimeStats(ctx, sctx, kvReq, fieldTypes, fb, copPlanIDs, rootPlanID) + return distsql.SelectWithRuntimeStats(ctx, sctx, kvReq, fieldTypes, copPlanIDs, rootPlanID) } - return sr.selectResultFunc(ctx, sctx, kvReq, fieldTypes, fb, copPlanIDs) + return sr.selectResultFunc(ctx, sctx, kvReq, fieldTypes, copPlanIDs) } type kvRangeBuilder interface { @@ -101,7 +100,6 @@ type TableReaderExecutor struct { // resultHandler handles the order of the result. Since (MAXInt64, MAXUint64] stores before [0, MaxInt64] physically // for unsigned int. resultHandler *tableResultHandler - feedback *statistics.QueryFeedback plans []plannercore.PhysicalPlan tablePlan plannercore.PhysicalPlan @@ -184,14 +182,6 @@ func (e *TableReaderExecutor) Open(ctx context.Context) error { } e.resultHandler = &tableResultHandler{} - if e.feedback != nil && e.feedback.Hist != nil { - // EncodeInt don't need *statement.Context. - var ok bool - e.ranges, ok = e.feedback.Hist.SplitRange(nil, e.ranges, false) - if !ok { - e.feedback.Invalidate() - } - } firstPartRanges, secondPartRanges := distsql.SplitRangesAcrossInt64Boundary(e.ranges, e.keepOrder, e.desc, e.table.Meta() != nil && e.table.Meta().IsCommonHandle) @@ -227,7 +217,6 @@ func (e *TableReaderExecutor) Open(ctx context.Context) error { firstResult, err := e.buildResp(ctx, firstPartRanges) if err != nil { - e.feedback.Invalidate() return err } if len(secondPartRanges) == 0 { @@ -237,7 +226,6 @@ func (e *TableReaderExecutor) Open(ctx context.Context) error { var secondResult distsql.SelectResult secondResult, err = e.buildResp(ctx, secondPartRanges) if err != nil { - e.feedback.Invalidate() return err } e.resultHandler.open(firstResult, secondResult) @@ -261,7 +249,6 @@ func (e *TableReaderExecutor) Next(ctx context.Context, req *chunk.Chunk) error return tableName }), e.ranges) if err := e.resultHandler.nextChunk(ctx, req); err != nil { - e.feedback.Invalidate() return err } @@ -283,7 +270,6 @@ func (e *TableReaderExecutor) Close() error { if e.dummy { return nil } - e.Ctx().StoreQueryFeedback(e.feedback) return err } @@ -299,7 +285,7 @@ func (e *TableReaderExecutor) buildResp(ctx context.Context, ranges []*ranger.Ra } var results []distsql.SelectResult for _, kvReq := range kvReqs { - result, err := e.SelectResult(ctx, e.Ctx(), kvReq, retTypes(e), e.feedback, getPhysicalPlanIDs(e.plans), e.ID()) + result, err := e.SelectResult(ctx, e.Ctx(), kvReq, retTypes(e), getPhysicalPlanIDs(e.plans), e.ID()) if err != nil { return nil, err } @@ -312,7 +298,7 @@ func (e *TableReaderExecutor) buildResp(ctx context.Context, ranges []*ranger.Ra if err != nil { return nil, err } - result, err := e.SelectResult(ctx, e.Ctx(), kvReq, retTypes(e), e.feedback, getPhysicalPlanIDs(e.plans), e.ID()) + result, err := e.SelectResult(ctx, e.Ctx(), kvReq, retTypes(e), getPhysicalPlanIDs(e.plans), e.ID()) if err != nil { return nil, err } @@ -327,7 +313,7 @@ func (e *TableReaderExecutor) buildResp(ctx context.Context, ranges []*ranger.Ra } var results []distsql.SelectResult for _, kvReq := range kvReqs { - result, err := e.SelectResult(ctx, e.Ctx(), kvReq, retTypes(e), e.feedback, getPhysicalPlanIDs(e.plans), e.ID()) + result, err := e.SelectResult(ctx, e.Ctx(), kvReq, retTypes(e), getPhysicalPlanIDs(e.plans), e.ID()) if err != nil { return nil, err } @@ -348,7 +334,7 @@ func (e *TableReaderExecutor) buildResp(ctx context.Context, ranges []*ranger.Ra }) e.kvRanges = kvReq.KeyRanges.AppendSelfTo(e.kvRanges) - result, err := e.SelectResult(ctx, e.Ctx(), kvReq, retTypes(e), e.feedback, getPhysicalPlanIDs(e.plans), e.ID()) + result, err := e.SelectResult(ctx, e.Ctx(), kvReq, retTypes(e), getPhysicalPlanIDs(e.plans), e.ID()) if err != nil { return nil, err } @@ -442,7 +428,7 @@ func (e *TableReaderExecutor) buildKVReq(ctx context.Context, ranges []*ranger.R } reqBuilder = builder.SetPartitionKeyRanges(kvRange) } else { - reqBuilder = builder.SetHandleRanges(e.Ctx().GetSessionVars().StmtCtx, getPhysicalTableID(e.table), e.table.Meta() != nil && e.table.Meta().IsCommonHandle, ranges, e.feedback) + reqBuilder = builder.SetHandleRanges(e.Ctx().GetSessionVars().StmtCtx, getPhysicalTableID(e.table), e.table.Meta() != nil && e.table.Meta().IsCommonHandle, ranges) } if e.table != nil && e.table.Type().IsClusterTable() { copDestination := infoschema.GetClusterTableCopDestination(e.table.Meta().Name.L) diff --git a/executor/table_readers_required_rows_test.go b/executor/table_readers_required_rows_test.go index 2a18ae7a48cab..8a3367fd5a414 100644 --- a/executor/table_readers_required_rows_test.go +++ b/executor/table_readers_required_rows_test.go @@ -29,7 +29,6 @@ import ( "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" @@ -114,7 +113,7 @@ func mockDistsqlSelectCtxGet(ctx context.Context) (totalRows int, expectedRowsRe } func mockSelectResult(ctx context.Context, sctx sessionctx.Context, kvReq *kv.Request, - fieldTypes []*types.FieldType, fb *statistics.QueryFeedback, copPlanIDs []int) (distsql.SelectResult, error) { + fieldTypes []*types.FieldType, copPlanIDs []int) (distsql.SelectResult, error) { totalRows, expectedRowsRet := mockDistsqlSelectCtxGet(ctx) return &requiredRowsSelectResult{ retTypes: fieldTypes, diff --git a/executor/test/analyzetest/analyze_test.go b/executor/test/analyzetest/analyze_test.go index 379f437bec51c..ff13f753f912c 100644 --- a/executor/test/analyzetest/analyze_test.go +++ b/executor/test/analyzetest/analyze_test.go @@ -746,19 +746,9 @@ func testAnalyzeIncremental(tk *testkit.TestKit, t *testing.T, dom *domain.Domai // Result should not change. tk.MustQuery("show stats_buckets").Check(testkit.Rows("test t a 0 0 1 1 1 1 0", "test t a 0 1 2 1 2 2 0", "test t idx 1 0 1 1 1 1 0", "test t idx 1 1 2 1 2 2 0")) - // Test analyze incremental with feedback. - // paging is not compatible with feedback. tk.MustExec("set @@tidb_enable_paging = off") tk.MustExec("insert into t values (3,3)") - oriProbability := statistics.FeedbackProbability.Load() - oriMinLogCount := handle.MinLogScanCount.Load() - defer func() { - statistics.FeedbackProbability.Store(oriProbability) - handle.MinLogScanCount.Store(oriMinLogCount) - }() - statistics.FeedbackProbability.Store(1) - handle.MinLogScanCount.Store(0) is := dom.InfoSchema() table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) require.NoError(t, err) @@ -767,17 +757,15 @@ func testAnalyzeIncremental(tk *testkit.TestKit, t *testing.T, dom *domain.Domai tk.MustQuery("select * from t where a > 1") h := dom.StatsHandle() require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - require.NoError(t, h.DumpStatsFeedbackToKV()) - require.NoError(t, h.HandleUpdateStats(is)) require.NoError(t, h.Update(is)) require.NoError(t, h.LoadNeededHistograms()) - tk.MustQuery("show stats_buckets").Check(testkit.Rows("test t a 0 0 1 1 1 1 0", "test t a 0 1 3 0 2 2147483647 0", "test t idx 1 0 1 1 1 1 0", "test t idx 1 1 2 1 2 2 0")) + tk.MustQuery("show stats_buckets").Check(testkit.Rows("test t a 0 0 1 1 1 1 0", "test t a 0 1 2 1 2 2 0", "test t idx 1 0 1 1 1 1 0", "test t idx 1 1 2 1 2 2 0")) tblStats := h.GetTableStats(tblInfo) val, err := codec.EncodeKey(tk.Session().GetSessionVars().StmtCtx, nil, types.NewIntDatum(3)) require.NoError(t, err) - require.Equal(t, uint64(1), tblStats.Indices[tblInfo.Indices[0].ID].QueryBytes(nil, val)) - require.False(t, statistics.IsAnalyzed(tblStats.Indices[tblInfo.Indices[0].ID].Flag)) - require.False(t, statistics.IsAnalyzed(tblStats.Columns[tblInfo.Columns[0].ID].Flag)) + require.Equal(t, uint64(0), tblStats.Indices[tblInfo.Indices[0].ID].QueryBytes(nil, val)) + require.True(t, statistics.IsAnalyzed(tblStats.Indices[tblInfo.Indices[0].ID].Flag)) + require.True(t, statistics.IsAnalyzed(tblStats.Columns[tblInfo.Columns[0].ID].Flag)) tk.MustExec("analyze incremental table t index") require.NoError(t, h.LoadNeededHistograms()) diff --git a/executor/test/executor/BUILD.bazel b/executor/test/executor/BUILD.bazel index bee916f714c86..7671d43abe2fa 100644 --- a/executor/test/executor/BUILD.bazel +++ b/executor/test/executor/BUILD.bazel @@ -33,7 +33,6 @@ go_test( "//sessionctx/stmtctx", "//sessionctx/variable", "//sessiontxn", - "//statistics", "//store/driver/error", "//store/mockstore", "//table/tables", diff --git a/executor/test/executor/executor_test.go b/executor/test/executor/executor_test.go index ea80d99a36b56..2e86189d50f3a 100644 --- a/executor/test/executor/executor_test.go +++ b/executor/test/executor/executor_test.go @@ -54,7 +54,6 @@ import ( "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/sessiontxn" - "github.com/pingcap/tidb/statistics" error2 "github.com/pingcap/tidb/store/driver/error" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/table/tables" @@ -1559,23 +1558,6 @@ func TestPlanReplayerDumpSingle(t *testing.T) { } } -func TestUnsignedFeedback(t *testing.T) { - store := testkit.CreateMockStore(t) - - tk := testkit.NewTestKit(t, store) - oriProbability := statistics.FeedbackProbability.Load() - statistics.FeedbackProbability.Store(1.0) - defer func() { statistics.FeedbackProbability.Store(oriProbability) }() - tk.MustExec("use test") - tk.MustExec("create table t(a bigint unsigned, b int, primary key(a))") - tk.MustExec("insert into t values (1,1),(2,2)") - tk.MustExec("analyze table t") - tk.MustQuery("select count(distinct b) from t").Check(testkit.Rows("2")) - result := tk.MustQuery("explain analyze select count(distinct b) from t") - require.Equal(t, "table:t", result.Rows()[2][4]) - require.Equal(t, "keep order:false", result.Rows()[2][6]) -} - func TestAlterTableComment(t *testing.T) { store := testkit.CreateMockStore(t) diff --git a/executor/test/issuetest/BUILD.bazel b/executor/test/issuetest/BUILD.bazel index 2680bd5126f9c..32ce75a0136f4 100644 --- a/executor/test/issuetest/BUILD.bazel +++ b/executor/test/issuetest/BUILD.bazel @@ -19,7 +19,6 @@ go_test( "//parser/mysql", "//session", "//sessionctx/variable", - "//statistics", "//testkit", "//util", "//util/memory", diff --git a/executor/test/issuetest/executor_issue_test.go b/executor/test/issuetest/executor_issue_test.go index d72903bd77b76..e7e997d106b79 100644 --- a/executor/test/issuetest/executor_issue_test.go +++ b/executor/test/issuetest/executor_issue_test.go @@ -31,7 +31,6 @@ import ( "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx/variable" - "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/memory" @@ -1180,23 +1179,6 @@ func Test15492(t *testing.T) { tk.MustQuery("select a + 1 as field1, a as field2 from t order by field1, field2 limit 2").Check(testkit.Rows("2 1", "3 2")) } -func TestIssue23567(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - oriProbability := statistics.FeedbackProbability.Load() - statistics.FeedbackProbability.Store(1.0) - defer func() { statistics.FeedbackProbability.Store(oriProbability) }() - failpoint.Enable("github.com/pingcap/tidb/statistics/feedbackNoNDVCollect", `return("")`) - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(a bigint unsigned, b int, primary key(a))") - tk.MustExec("insert into t values (1, 1), (2, 2)") - tk.MustExec("analyze table t") - // The SQL should not panic. - tk.MustQuery("select count(distinct b) from t") - failpoint.Disable("github.com/pingcap/tidb/statistics/feedbackNoNDVCollect") -} - func TestIssue33038(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) diff --git a/infoschema/metric_table_def.go b/infoschema/metric_table_def.go index 0bc722ad6ffb8..08ebcb501721c 100644 --- a/infoschema/metric_table_def.go +++ b/infoschema/metric_table_def.go @@ -541,31 +541,6 @@ var MetricTableMap = map[string]MetricTableDef{ PromQL: "sum(increase(tidb_statistics_pseudo_estimation_total{$LABEL_CONDITIONS}[$RANGE_DURATION]))", Labels: []string{"instance"}, }, - "tidb_statistics_dump_feedback_ops": { - Comment: "TiDB dumping statistics back to kv storage times", - PromQL: "sum(rate(tidb_statistics_dump_feedback_total{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (type,instance)", - Labels: []string{"instance", "type"}, - }, - "tidb_statistics_dump_feedback_total_count": { - Comment: "The total count of operations that TiDB dumping statistics back to kv storage", - PromQL: "sum(increase(tidb_statistics_dump_feedback_total{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (type,instance)", - Labels: []string{"instance", "type"}, - }, - "tidb_statistics_store_query_feedback_qps": { - Comment: "TiDB store quering feedback counts", - PromQL: "sum(rate(tidb_statistics_store_query_feedback_total{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (type,instance) ", - Labels: []string{"instance", "type"}, - }, - "tidb_statistics_store_query_feedback_total_count": { - Comment: "The total count of TiDB store quering feedback", - PromQL: "sum(increase(tidb_statistics_store_query_feedback_total{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (type,instance) ", - Labels: []string{"instance", "type"}, - }, - "tidb_statistics_significant_feedback": { - Comment: "Counter of query feedback whose actual count is much different than calculated by current statistics", - PromQL: "sum(rate(tidb_statistics_high_error_rate_feedback_total{$LABEL_CONDITIONS}[$RANGE_DURATION]))", - Labels: []string{"instance"}, - }, "tidb_statistics_update_stats_ops": { Comment: "TiDB updating statistics using feed back counts", PromQL: "sum(rate(tidb_statistics_update_stats_total{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (type,instance)", diff --git a/metrics/grafana/tidb.json b/metrics/grafana/tidb.json index 847c3e3fbdfb0..dd8cd7bb0e965 100644 --- a/metrics/grafana/tidb.json +++ b/metrics/grafana/tidb.json @@ -51,8 +51,8 @@ "editable": true, "gnetId": null, "graphTooltip": 1, - "id": null, - "iteration": 1655990780337, + "id": 32, + "iteration": 1692177797027, "links": [], "panels": [ { @@ -1598,9 +1598,7 @@ "mode": "time", "name": null, "show": true, - "values": [ - "max" - ] + "values": ["max"] }, "yaxes": [ { @@ -8014,14 +8012,6 @@ } }, { - "type": "graph", - "title": "Mpp Coordinator Counter", - "gridPos": { - "x": 12, - "y": 37, - "w": 12, - "h": 8 - }, "aliasColors": {}, "bars": false, "dashLength": 10, @@ -8034,6 +8024,12 @@ }, "fill": 0, "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 37 + }, "hiddenSeries": false, "id": 311, "legend": { @@ -8078,11 +8074,13 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, + "title": "Mpp Coordinator Counter", "tooltip": { "shared": true, "sort": 0, "value_type": "individual" }, + "type": "graph", "xaxis": { "buckets": null, "mode": "time", @@ -8114,14 +8112,6 @@ } }, { - "type": "graph", - "title": "Mpp Coordinator Latency", - "gridPos": { - "x": 0, - "y": 45, - "w": 12, - "h": 8 - }, "aliasColors": {}, "bars": false, "dashLength": 10, @@ -8134,6 +8124,12 @@ }, "fill": 0, "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 45 + }, "hiddenSeries": false, "id": 312, "legend": { @@ -8199,11 +8195,13 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, + "title": "Mpp Coordinator Latency", "tooltip": { "shared": true, "sort": 0, "value_type": "individual" }, + "type": "graph", "xaxis": { "buckets": null, "mode": "time", @@ -11601,7 +11599,7 @@ "alertThreshold": true }, "percentage": false, - "pluginVersion": "7.5.10", + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -11757,7 +11755,7 @@ "alertThreshold": true }, "percentage": false, - "pluginVersion": "7.5.10", + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -11873,7 +11871,7 @@ "alertThreshold": true }, "percentage": false, - "pluginVersion": "7.5.10", + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -13565,7 +13563,7 @@ "alertThreshold": true }, "percentage": false, - "pluginVersion": "7.5.10", + "pluginVersion": "7.5.11", "pointradius": 2, "points": false, "renderer": "flot", @@ -13648,13 +13646,19 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "description": "TiDB auto analyze time durations within 95 percent histogram buckets", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 7, "w": 8, "x": 0, - "y": 149 + "y": 12 }, + "hiddenSeries": false, "id": 46, "legend": { "avg": false, @@ -13669,7 +13673,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -13735,13 +13743,19 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "description": "TiDB auto analyze query per second", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 7, "w": 8, "x": 8, - "y": 149 + "y": 12 }, + "hiddenSeries": false, "id": 47, "legend": { "avg": false, @@ -13756,7 +13770,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -13822,13 +13840,19 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "description": "TiDB statistics inaccurate rate", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 7, "w": 8, "x": 16, - "y": 149 + "y": 12 }, + "hiddenSeries": false, "id": 70, "legend": { "avg": false, @@ -13843,7 +13867,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -13923,13 +13951,19 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "description": "TiDB optimizer using pseudo estimation counts", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 7, "w": 8, "x": 0, - "y": 156 + "y": 19 }, + "hiddenSeries": false, "id": 71, "legend": { "avg": false, @@ -13944,7 +13978,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -14003,267 +14041,6 @@ "alignLevel": null } }, - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "TiDB dumping statistics back to kv storage times", - "fill": 1, - "gridPos": { - "h": 7, - "w": 8, - "x": 8, - "y": 156 - }, - "id": 92, - "legend": { - "avg": false, - "current": false, - "max": false, - "min": false, - "show": true, - "total": false, - "values": false - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "percentage": false, - "pointradius": 5, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "sum(rate(tidb_statistics_dump_feedback_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": "{{type}}", - "refId": "A", - "step": 30 - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Dump Feedback OPS", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } - }, - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "TiDB store quering feedback counts", - "fill": 1, - "gridPos": { - "h": 7, - "w": 8, - "x": 16, - "y": 156 - }, - "id": 170, - "legend": { - "avg": false, - "current": false, - "max": false, - "min": false, - "show": true, - "total": false, - "values": false - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "percentage": false, - "pointradius": 5, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "sum(rate(tidb_statistics_store_query_feedback_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type) ", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": "{{type}}", - "refId": "A", - "step": 30 - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Store Query Feedback QPS", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } - }, - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "Counter of query feedback whose actual count is much different than calculated by current statistics", - "fill": 1, - "gridPos": { - "h": 7, - "w": 8, - "x": 0, - "y": 163 - }, - "id": 113, - "legend": { - "avg": false, - "current": false, - "max": false, - "min": false, - "show": true, - "total": false, - "values": false - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "percentage": false, - "pointradius": 5, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "sum(rate(tidb_statistics_high_error_rate_feedback_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))", - "format": "time_series", - "hide": false, - "intervalFactor": 2, - "legendFormat": "Significant Feedback", - "refId": "A" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Significant Feedback", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } - }, { "aliasColors": {}, "bars": false, @@ -14271,13 +14048,19 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "description": "TiDB updating statistics using feed back counts", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 7, "w": 8, "x": 8, - "y": 163 + "y": 19 }, + "hiddenSeries": false, "id": 93, "legend": { "avg": false, @@ -14292,7 +14075,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -14360,14 +14147,20 @@ "description": "TiDB fast analyze statistics ", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 7, "w": 8, "x": 16, - "y": 163 + "y": 19 }, + "hiddenSeries": false, "id": 173, "legend": { "alignAsTable": true, @@ -14386,7 +14179,11 @@ "linewidth": 2, "links": [], "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -14463,7 +14260,7 @@ "h": 7, "w": 8, "x": 0, - "y": 170 + "y": 26 }, "hiddenSeries": false, "id": 229, @@ -14484,6 +14281,7 @@ "alertThreshold": true }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -14572,7 +14370,7 @@ "h": 7, "w": 8, "x": 8, - "y": 170 + "y": 26 }, "hiddenSeries": false, "id": 230, @@ -14593,6 +14391,7 @@ "alertThreshold": true }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -14681,7 +14480,7 @@ "h": 7, "w": 8, "x": 16, - "y": 170 + "y": 26 }, "hiddenSeries": false, "id": 233, @@ -14782,7 +14581,7 @@ "h": 7, "w": 12, "x": 0, - "y": 177 + "y": 33 }, "hiddenSeries": false, "id": 234, @@ -14894,7 +14693,7 @@ "h": 7, "w": 12, "x": 12, - "y": 177 + "y": 33 }, "hiddenSeries": false, "id": 235, @@ -14997,7 +14796,7 @@ "h": 7, "w": 8, "x": 0, - "y": 184 + "y": 40 }, "hiddenSeries": false, "id": 236, @@ -15116,7 +14915,7 @@ "h": 7, "w": 8, "x": 8, - "y": 184 + "y": 40 }, "hiddenSeries": false, "id": 237, @@ -19377,7 +19176,7 @@ "alertThreshold": true }, "percentage": false, - "pluginVersion": "7.5.10", + "pluginVersion": "7.5.11", "pointradius": 2, "points": false, "renderer": "flot", @@ -19534,16 +19333,14 @@ "displayMode": "lcd", "orientation": "horizontal", "reduceOptions": { - "calcs": [ - "lastNotNull" - ], + "calcs": ["lastNotNull"], "fields": "", "values": false }, "showUnfilled": true, "text": {} }, - "pluginVersion": "7.5.10", + "pluginVersion": "7.5.11", "targets": [ { "exemplar": true, @@ -19639,7 +19436,7 @@ "alertThreshold": true }, "percentage": false, - "pluginVersion": "7.5.10", + "pluginVersion": "7.5.11", "pointradius": 2, "points": false, "renderer": "flot", @@ -19725,7 +19522,7 @@ "h": 1, "w": 24, "x": 0, - "y": 19 + "y": 18 }, "id": 291, "panels": [ @@ -19931,7 +19728,7 @@ "h": 1, "w": 24, "x": 0, - "y": 104 + "y": 19 }, "id": 309, "panels": [ @@ -20387,17 +20184,7 @@ "2h", "1d" ], - "time_options": [ - "5m", - "15m", - "1h", - "6h", - "12h", - "24h", - "2d", - "7d", - "30d" - ] + "time_options": ["5m", "15m", "1h", "6h", "12h", "24h", "2d", "7d", "30d"] }, "timezone": "browser", "title": "Test-Cluster-TiDB", diff --git a/metrics/metrics.go b/metrics/metrics.go index 1120978b0efb5..5957b13f358e5 100644 --- a/metrics/metrics.go +++ b/metrics/metrics.go @@ -143,14 +143,12 @@ func RegisterMetrics() { prometheus.MustRegister(DistSQLQueryHistogram) prometheus.MustRegister(DistSQLScanKeysHistogram) prometheus.MustRegister(DistSQLScanKeysPartialHistogram) - prometheus.MustRegister(DumpFeedbackCounter) prometheus.MustRegister(ExecuteErrorCounter) prometheus.MustRegister(ExecutorCounter) prometheus.MustRegister(GetTokenDurationHistogram) prometheus.MustRegister(NumOfMultiQueryHistogram) prometheus.MustRegister(HandShakeErrorCounter) prometheus.MustRegister(HandleJobHistogram) - prometheus.MustRegister(SignificantFeedbackCounter) prometheus.MustRegister(FastAnalyzeHistogram) prometheus.MustRegister(SyncLoadCounter) prometheus.MustRegister(SyncLoadTimeoutCounter) @@ -190,7 +188,6 @@ func RegisterMetrics() { prometheus.MustRegister(OngoingTxnDurationHistogram) prometheus.MustRegister(MppCoordinatorStats) prometheus.MustRegister(MppCoordinatorLatency) - prometheus.MustRegister(StoreQueryFeedbackCounter) prometheus.MustRegister(TimeJumpBackCounter) prometheus.MustRegister(TransactionDuration) prometheus.MustRegister(StatementDeadlockDetectDuration) @@ -198,7 +195,6 @@ func RegisterMetrics() { prometheus.MustRegister(StatementLockKeysCount) prometheus.MustRegister(ValidateReadTSFromPDCount) prometheus.MustRegister(UpdateSelfVersionHistogram) - prometheus.MustRegister(UpdateStatsCounter) prometheus.MustRegister(WatchOwnerCounter) prometheus.MustRegister(GCActionRegionResultCounter) prometheus.MustRegister(GCConfigGauge) diff --git a/metrics/stats.go b/metrics/stats.go index eb146bd5a5fcc..4258cc5b30b32 100644 --- a/metrics/stats.go +++ b/metrics/stats.go @@ -20,22 +20,18 @@ import ( // Stats metrics. var ( - AutoAnalyzeHistogram prometheus.Histogram - AutoAnalyzeCounter *prometheus.CounterVec - StatsInaccuracyRate prometheus.Histogram - PseudoEstimation *prometheus.CounterVec - DumpFeedbackCounter *prometheus.CounterVec - UpdateStatsCounter *prometheus.CounterVec - StoreQueryFeedbackCounter *prometheus.CounterVec - SignificantFeedbackCounter prometheus.Counter - FastAnalyzeHistogram *prometheus.HistogramVec - SyncLoadCounter prometheus.Counter - SyncLoadTimeoutCounter prometheus.Counter - SyncLoadHistogram prometheus.Histogram - ReadStatsHistogram prometheus.Histogram - StatsCacheCounter *prometheus.CounterVec - StatsCacheGauge *prometheus.GaugeVec - StatsHealthyGauge *prometheus.GaugeVec + AutoAnalyzeHistogram prometheus.Histogram + AutoAnalyzeCounter *prometheus.CounterVec + StatsInaccuracyRate prometheus.Histogram + PseudoEstimation *prometheus.CounterVec + FastAnalyzeHistogram *prometheus.HistogramVec + SyncLoadCounter prometheus.Counter + SyncLoadTimeoutCounter prometheus.Counter + SyncLoadHistogram prometheus.Histogram + ReadStatsHistogram prometheus.Histogram + StatsCacheCounter *prometheus.CounterVec + StatsCacheGauge *prometheus.GaugeVec + StatsHealthyGauge *prometheus.GaugeVec HistoricalStatsCounter *prometheus.CounterVec PlanReplayerTaskCounter *prometheus.CounterVec @@ -78,38 +74,6 @@ func InitStatsMetrics() { Help: "Counter of pseudo estimation caused by outdated stats.", }, []string{LblType}) - DumpFeedbackCounter = NewCounterVec( - prometheus.CounterOpts{ - Namespace: "tidb", - Subsystem: "statistics", - Name: "dump_feedback_total", - Help: "Counter of dumping feedback.", - }, []string{LblType}) - - UpdateStatsCounter = NewCounterVec( - prometheus.CounterOpts{ - Namespace: "tidb", - Subsystem: "statistics", - Name: "update_stats_total", - Help: "Counter of updating stats using feedback.", - }, []string{LblType}) - - StoreQueryFeedbackCounter = NewCounterVec( - prometheus.CounterOpts{ - Namespace: "tidb", - Subsystem: "statistics", - Name: "store_query_feedback_total", - Help: "Counter of storing query feedback.", - }, []string{LblType}) - - SignificantFeedbackCounter = NewCounter( - prometheus.CounterOpts{ - Namespace: "tidb", - Subsystem: "statistics", - Name: "high_error_rate_feedback_total", - Help: "Counter of query feedback whose actual count is much different than calculated by current statistics", - }) - FastAnalyzeHistogram = NewHistogramVec( prometheus.HistogramOpts{ Namespace: "tidb", diff --git a/planner/core/BUILD.bazel b/planner/core/BUILD.bazel index 7833bead1485a..d18fb0424f71a 100644 --- a/planner/core/BUILD.bazel +++ b/planner/core/BUILD.bazel @@ -272,7 +272,6 @@ go_test( "//types/parser_driver", "//util", "//util/benchdaily", - "//util/chunk", "//util/collate", "//util/dbterror", "//util/hack", diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index 57e3871361c35..b4525849f047a 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -1477,13 +1477,6 @@ func (ds *DataSource) buildIndexMergeTableScan(tableFilters []expression.Express if err != nil { return nil, nil, false, err } - if ts.Table.PKIsHandle { - if pkColInfo := ts.Table.GetPkColInfo(); pkColInfo != nil { - if ds.statisticTable.Columns[pkColInfo.ID] != nil { - ts.Hist = &ds.statisticTable.Columns[pkColInfo.ID].Histogram - } - } - } ts.SetStats(ds.tableStats.ScaleByExpectCnt(totalRowCount)) usedStats := ds.SCtx().GetSessionVars().StmtCtx.GetUsedStatsInfo(false) if usedStats != nil && usedStats[ts.physicalTableID] != nil { @@ -2208,13 +2201,6 @@ func (s *LogicalTableScan) GetPhysicalScan(schema *expression.Schema, stats *pro }.Init(s.SCtx(), s.SelectBlockOffset()) ts.SetStats(stats) ts.SetSchema(schema.Clone()) - if ts.Table.PKIsHandle { - if pkColInfo := ts.Table.GetPkColInfo(); pkColInfo != nil { - if ds.statisticTable.Columns[pkColInfo.ID] != nil { - ts.Hist = &ds.statisticTable.Columns[pkColInfo.ID].Histogram - } - } - } return ts } @@ -2672,13 +2658,6 @@ func (ds *DataSource) getOriginalPhysicalTableScan(prop *property.PhysicalProper filterCondition: slices.Clone(path.TableFilters), }.Init(ds.SCtx(), ds.SelectBlockOffset()) ts.SetSchema(ds.schema.Clone()) - if ts.Table.PKIsHandle { - if pkColInfo := ts.Table.GetPkColInfo(); pkColInfo != nil { - if ds.statisticTable.Columns[pkColInfo.ID] != nil { - ts.Hist = &ds.statisticTable.Columns[pkColInfo.ID].Histogram - } - } - } rowCount := path.CountAfterAccess if prop.ExpectedCnt < ds.StatsInfo().RowCount { selectivity := ds.StatsInfo().RowCount / path.CountAfterAccess @@ -2738,10 +2717,6 @@ func (ds *DataSource) getOriginalPhysicalIndexScan(prop *property.PhysicalProper constColsByCond: path.ConstCols, prop: prop, }.Init(ds.SCtx(), ds.SelectBlockOffset()) - statsTbl := ds.statisticTable - if statsTbl.Indices[idx.ID] != nil { - is.Hist = &statsTbl.Indices[idx.ID].Histogram - } rowCount := path.CountAfterAccess is.initSchema(append(path.FullIdxCols, ds.commonHandleCols...), !isSingleScan) diff --git a/planner/core/fragment.go b/planner/core/fragment.go index 5ee4acdcc728a..b361f13afe123 100644 --- a/planner/core/fragment.go +++ b/planner/core/fragment.go @@ -608,7 +608,7 @@ func (e *mppTaskGenerator) constructMPPBuildTaskReqForPartitionedTable(ts *Physi for i, p := range partitions { pid := p.GetPhysicalID() meta := p.Meta() - kvRanges, err := distsql.TableHandleRangesToKVRanges(e.ctx.GetSessionVars().StmtCtx, []int64{pid}, meta != nil && ts.Table.IsCommonHandle, splitedRanges, nil) + kvRanges, err := distsql.TableHandleRangesToKVRanges(e.ctx.GetSessionVars().StmtCtx, []int64{pid}, meta != nil && ts.Table.IsCommonHandle, splitedRanges) if err != nil { return nil, nil, errors.Trace(err) } @@ -620,7 +620,7 @@ func (e *mppTaskGenerator) constructMPPBuildTaskReqForPartitionedTable(ts *Physi } func (e *mppTaskGenerator) constructMPPBuildTaskForNonPartitionTable(tid int64, isCommonHandle bool, splitedRanges []*ranger.Range) (*kv.MPPBuildTasksRequest, error) { - kvRanges, err := distsql.TableHandleRangesToKVRanges(e.ctx.GetSessionVars().StmtCtx, []int64{tid}, isCommonHandle, splitedRanges, nil) + kvRanges, err := distsql.TableHandleRangesToKVRanges(e.ctx.GetSessionVars().StmtCtx, []int64{tid}, isCommonHandle, splitedRanges) if err != nil { return nil, errors.Trace(err) } diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index eb716eba5d5f2..2e564f83f9a4e 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -671,10 +671,6 @@ type PhysicalIndexScan struct { // will be different. The schema of index scan will decode all columns of index but the TiDB only need some of them. dataSourceSchema *expression.Schema - // Hist is the histogram when the query was issued. - // It is used for query feedback. - Hist *statistics.Histogram - rangeInfo string // The index scan may be on a partition. @@ -734,9 +730,7 @@ func (p *PhysicalIndexScan) Clone() (PhysicalPlan, error) { if p.dataSourceSchema != nil { cloned.dataSourceSchema = p.dataSourceSchema.Clone() } - if p.Hist != nil { - cloned.Hist = p.Hist.Copy() - } + return cloned, nil } @@ -846,10 +840,6 @@ type PhysicalTableScan struct { TableAsName *model.CIStr - // Hist is the histogram when the query was issued. - // It is used for query feedback. - Hist *statistics.Histogram - physicalTableID int64 rangeInfo string @@ -915,9 +905,6 @@ func (ts *PhysicalTableScan) Clone() (PhysicalPlan, error) { clonedScan.Columns = util.CloneColInfos(ts.Columns) clonedScan.Ranges = util.CloneRanges(ts.Ranges) clonedScan.TableAsName = ts.TableAsName - if ts.Hist != nil { - clonedScan.Hist = ts.Hist.Copy() - } clonedScan.rangeInfo = ts.rangeInfo clonedScan.runtimeFilterList = make([]*RuntimeFilter, len(ts.runtimeFilterList)) for i, rf := range ts.runtimeFilterList { diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 366f2dded0f93..ef58e47b40147 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -2790,18 +2790,6 @@ func (b *PlanBuilder) buildAnalyzeTable(as *ast.AnalyzeTableStmt, opts map[ast.A return nil, err } var commonHandleInfo *model.IndexInfo - // If we want to analyze this table with analyze version 2 but the existing stats is version 1 and stats feedback is enabled, - // we will switch back to analyze version 1. - if statistics.FeedbackProbability.Load() > 0 && version == statistics.Version2 { - statsHandle := domain.GetDomain(b.ctx).StatsHandle() - versionIsSame := statsHandle.CheckAnalyzeVersion(tbl.TableInfo, physicalIDs, &version) - if !versionIsSame { - b.ctx.GetSessionVars().StmtCtx.AppendWarning(fmt.Errorf("Use analyze version 1 on table `%s` "+ - "because this table already has version 1 statistics and query feedback is also enabled. "+ - "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.", tbl.Name)) - } - } - if version == statistics.Version2 { p.ColTasks, err = b.buildAnalyzeFullSamplingTask(as, p.ColTasks, physicalIDs, partitionNames, tbl, version, usePersistedOptions, p.OptionsMap) if err != nil { diff --git a/planner/core/planbuilder_test.go b/planner/core/planbuilder_test.go index 6a61bcdd2d84d..5b95414a05943 100644 --- a/planner/core/planbuilder_test.go +++ b/planner/core/planbuilder_test.go @@ -34,7 +34,6 @@ import ( "github.com/pingcap/tidb/planner/util" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/hint" "github.com/pingcap/tidb/util/mock" "github.com/stretchr/testify/require" @@ -231,13 +230,11 @@ func TestTablePlansAndTablePlanInPhysicalTableReaderClone(t *testing.T) { col, cst := &expression.Column{RetType: types.NewFieldType(mysql.TypeString)}, &expression.Constant{RetType: types.NewFieldType(mysql.TypeLonglong)} schema := expression.NewSchema(col) tblInfo := &model.TableInfo{} - hist := &statistics.Histogram{Bounds: chunk.New(nil, 0, 0)} // table scan tableScan := &PhysicalTableScan{ AccessCondition: []expression.Expression{col, cst}, Table: tblInfo, - Hist: hist, } tableScan = tableScan.Init(ctx, 0) tableScan.SetSchema(schema) @@ -263,7 +260,6 @@ func TestPhysicalPlanClone(t *testing.T) { schema := expression.NewSchema(col) tblInfo := &model.TableInfo{} idxInfo := &model.IndexInfo{} - hist := &statistics.Histogram{Bounds: chunk.New(nil, 0, 0)} aggDesc1, err := aggregation.NewAggFuncDesc(ctx, ast.AggFuncAvg, []expression.Expression{col}, false) require.NoError(t, err) aggDesc2, err := aggregation.NewAggFuncDesc(ctx, ast.AggFuncCount, []expression.Expression{cst}, true) @@ -274,7 +270,6 @@ func TestPhysicalPlanClone(t *testing.T) { tableScan := &PhysicalTableScan{ AccessCondition: []expression.Expression{col, cst}, Table: tblInfo, - Hist: hist, } tableScan = tableScan.Init(ctx, 0) tableScan.SetSchema(schema) @@ -294,7 +289,6 @@ func TestPhysicalPlanClone(t *testing.T) { AccessCondition: []expression.Expression{col, cst}, Table: tblInfo, Index: idxInfo, - Hist: hist, dataSourceSchema: schema, } indexScan = indexScan.Init(ctx, 0) diff --git a/session/BUILD.bazel b/session/BUILD.bazel index 88c863f13765c..35a5943935d84 100644 --- a/session/BUILD.bazel +++ b/session/BUILD.bazel @@ -59,7 +59,6 @@ go_library( "//sessiontxn", "//sessiontxn/isolation", "//sessiontxn/staleread", - "//statistics", "//statistics/handle", "//store/driver/error", "//store/driver/txn", diff --git a/session/bootstrap.go b/session/bootstrap.go index 62d9cd263ce8b..0730e12e746a1 100644 --- a/session/bootstrap.go +++ b/session/bootstrap.go @@ -272,6 +272,7 @@ const ( );` // CreateStatsFeedbackTable stores the feedback info which is used to update stats. + // NOTE: Feedback is deprecated, but we still need to create this table for compatibility. CreateStatsFeedbackTable = `CREATE TABLE IF NOT EXISTS mysql.stats_feedback ( table_id BIGINT(64) NOT NULL, is_index TINYINT(2) NOT NULL, @@ -1597,6 +1598,7 @@ func upgradeToVer20(s Session, ver int64) { if ver >= version20 { return } + // NOTE: Feedback is deprecated, but we still need to create this table for compatibility. doReentrantDDL(s, CreateStatsFeedbackTable) } @@ -2844,6 +2846,7 @@ func doDDLWorks(s Session) { // Create gc_delete_range_done table. mustExecute(s, CreateGCDeleteRangeDoneTable) // Create stats_feedback table. + // NOTE: Feedback is deprecated, but we still need to create this table for compatibility. mustExecute(s, CreateStatsFeedbackTable) // Create role_edges table. mustExecute(s, CreateRoleEdgesTable) diff --git a/session/session.go b/session/session.go index 81b864f038fa7..5d74b720b1cd9 100644 --- a/session/session.go +++ b/session/session.go @@ -76,7 +76,6 @@ import ( "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/sessiontxn" - "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/statistics/handle" storeerr "github.com/pingcap/tidb/store/driver/error" "github.com/pingcap/tidb/store/driver/txn" @@ -446,30 +445,6 @@ func (s *session) GetSessionManager() util.SessionManager { return s.sessionManager } -func (s *session) StoreQueryFeedback(feedback interface{}) { - if variable.FeedbackProbability.Load() <= 0 { - return - } - if fb, ok := feedback.(*statistics.QueryFeedback); !ok || fb == nil || !fb.Valid.Load() { - return - } - if s.statsCollector != nil { - do, err := GetDomain(s.store) - if err != nil { - logutil.BgLogger().Debug("domain not found", zap.Error(err)) - metrics.StoreQueryFeedbackCounter.WithLabelValues(metrics.LblError).Inc() - return - } - err = s.statsCollector.StoreQueryFeedback(feedback, do.StatsHandle(), s.GetSessionVars().GetEnablePseudoForOutdatedStats()) - if err != nil { - logutil.BgLogger().Debug("store query feedback", zap.Error(err)) - metrics.StoreQueryFeedbackCounter.WithLabelValues(metrics.LblError).Inc() - return - } - metrics.StoreQueryFeedbackCounter.WithLabelValues(metrics.LblOK).Inc() - } -} - func (s *session) UpdateColStatsUsage(predicateColumns []model.TableItemID) { if s.statsCollector == nil { return diff --git a/sessionctx/context.go b/sessionctx/context.go index 29ba2b0dc29d8..05f1e34bfd18f 100644 --- a/sessionctx/context.go +++ b/sessionctx/context.go @@ -122,11 +122,7 @@ type Context interface { // GetSessionPlanCache returns the session-level cache of the physical plan. GetSessionPlanCache() PlanCache - // StoreQueryFeedback stores the query feedback. - StoreQueryFeedback(feedback interface{}) - // UpdateColStatsUsage updates the column stats usage. - // TODO: maybe we can use a method called GetSessionStatsCollector to replace both StoreQueryFeedback and UpdateColStatsUsage but we need to deal with import circle if we do so. UpdateColStatsUsage(predicateColumns []model.TableItemID) // HasDirtyContent checks whether there's dirty update on the given table. diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index eeb51efbe9a57..9a31b9eb15181 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -54,7 +54,6 @@ import ( tikvcfg "github.com/tikv/client-go/v2/config" tikvstore "github.com/tikv/client-go/v2/kv" tikvcliutil "github.com/tikv/client-go/v2/util" - atomic2 "go.uber.org/atomic" "go.uber.org/zap" ) @@ -2018,23 +2017,7 @@ var defaultSysVars = []*SysVar{ s.GuaranteeLinearizability = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBAnalyzeVersion, Value: strconv.Itoa(DefTiDBAnalyzeVersion), Type: TypeInt, MinValue: 1, MaxValue: 2, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { - if normalizedValue == "2" && FeedbackProbability != nil && FeedbackProbability.Load() > 0 { - var original string - var err error - if scope == ScopeGlobal { - original, err = vars.GlobalVarsAccessor.GetGlobalSysVar(TiDBAnalyzeVersion) - if err != nil { - return normalizedValue, nil - } - } else { - original = strconv.Itoa(vars.AnalyzeVersion) - } - vars.StmtCtx.AppendError(errors.New("variable tidb_analyze_version not updated because analyze version 2 is incompatible with query feedback. Please consider setting feedback-probability to 0.0 in config file to disable query feedback")) - return original, nil - } - return normalizedValue, nil - }, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeGlobal | ScopeSession, Name: TiDBAnalyzeVersion, Value: strconv.Itoa(DefTiDBAnalyzeVersion), Type: TypeInt, MinValue: 1, MaxValue: 2, SetSession: func(s *SessionVars, val string) error { s.AnalyzeVersion = tidbOptPositiveInt32(val, DefTiDBAnalyzeVersion) return nil }}, @@ -2823,10 +2806,6 @@ func setTiFlashComputeDispatchPolicy(s *SessionVars, val string) error { return nil } -// FeedbackProbability points to the FeedbackProbability in statistics package. -// It's initialized in init() in feedback.go to solve import cycle. -var FeedbackProbability *atomic2.Float64 - // SetNamesVariables is the system variable names related to set names statements. var SetNamesVariables = []string{ CharacterSetClient, diff --git a/statistics/BUILD.bazel b/statistics/BUILD.bazel index 9fdfe41144e49..ff7c8f7cae493 100644 --- a/statistics/BUILD.bazel +++ b/statistics/BUILD.bazel @@ -10,7 +10,6 @@ go_library( "column.go", "debugtrace.go", "estimate.go", - "feedback.go", "fmsketch.go", "histogram.go", "index.go", @@ -28,7 +27,6 @@ go_library( "//config", "//expression", "//kv", - "//metrics", "//parser/ast", "//parser/charset", "//parser/format", @@ -57,7 +55,6 @@ go_library( "//util/tracing", "@com_github_pingcap_errors//:errors", "@com_github_pingcap_failpoint//:failpoint", - "@com_github_pingcap_log//:log", "@com_github_pingcap_tipb//go-tipb", "@com_github_twmb_murmur3//:murmur3", "@org_golang_x_exp//maps", @@ -73,7 +70,6 @@ go_test( srcs = [ "cmsketch_bench_test.go", "cmsketch_test.go", - "feedback_test.go", "fmsketch_test.go", "histogram_test.go", "integration_test.go", @@ -119,10 +115,8 @@ go_test( "//util/tracing", "@com_github_pingcap_errors//:errors", "@com_github_pingcap_failpoint//:failpoint", - "@com_github_pingcap_log//:log", "@com_github_stretchr_testify//require", "@com_github_tiancaiamao_gp//:gp", "@org_uber_go_goleak//:goleak", - "@org_uber_go_zap//:zap", ], ) diff --git a/statistics/feedback.go b/statistics/feedback.go deleted file mode 100644 index 585ba3cb4e0ab..0000000000000 --- a/statistics/feedback.go +++ /dev/null @@ -1,1076 +0,0 @@ -// Copyright 2018 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package statistics - -import ( - "bytes" - "encoding/gob" - "math" - "math/rand" - "slices" - "sort" - goatomic "sync/atomic" - "time" - - "github.com/pingcap/errors" - "github.com/pingcap/failpoint" - "github.com/pingcap/log" - "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/metrics" - "github.com/pingcap/tidb/parser/mysql" - "github.com/pingcap/tidb/sessionctx/stmtctx" - "github.com/pingcap/tidb/sessionctx/variable" - "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/logutil" - "github.com/pingcap/tidb/util/mathutil" - "github.com/pingcap/tidb/util/ranger" - "go.uber.org/atomic" - "go.uber.org/zap" -) - -// Feedback represents the total scan count in range [lower, upper). -type Feedback struct { - Lower *types.Datum - Upper *types.Datum - Count int64 - Repeat int64 - Ndv int64 -} - -// QueryFeedback is used to represent the query feedback info. It contains the query's scan ranges and number of rows -// in each range. -type QueryFeedback struct { - Hist *Histogram - Feedback []Feedback - PhysicalID int64 - Tp int - Expected int64 // Expected is the Expected scan count of corresponding query. - actual int64 // actual is the actual scan count of corresponding query. - Valid goatomic.Bool // Valid represents the whether this query feedback is still Valid. - desc bool // desc represents the corresponding query is desc scan. -} - -// NewQueryFeedback returns a new query feedback. -func NewQueryFeedback(physicalID int64, hist *Histogram, expected int64, desc bool) *QueryFeedback { - if hist != nil && hist.Len() == 0 { - hist = nil - } - tp := PkType - if hist != nil && hist.IsIndexHist() { - tp = IndexType - } - rs := &QueryFeedback{ - PhysicalID: physicalID, - Tp: tp, - Hist: hist, - Expected: expected, - desc: desc, - } - rs.Valid.Store(FeedbackProbability.Load() > 0) - return rs -} - -// QueryFeedbackKey is the key for a group of feedbacks on the same index/column. -type QueryFeedbackKey struct { - PhysicalID int64 - HistID int64 - Tp int -} - -// QueryFeedbackMap is the collection of feedbacks. -type QueryFeedbackMap struct { - Feedbacks map[QueryFeedbackKey][]*QueryFeedback - Size int -} - -// NewQueryFeedbackMap builds a feedback collection. -func NewQueryFeedbackMap() *QueryFeedbackMap { - return &QueryFeedbackMap{Feedbacks: make(map[QueryFeedbackKey][]*QueryFeedback)} -} - -// Append adds a feedback into map. -func (m *QueryFeedbackMap) Append(q *QueryFeedback) { - k := QueryFeedbackKey{ - PhysicalID: q.PhysicalID, - HistID: q.Hist.ID, - Tp: q.Tp, - } - m.append(k, []*QueryFeedback{q}) -} - -// MaxQueryFeedbackCount is the max number of feedbacks that are cached in memory. -var MaxQueryFeedbackCount = atomic.NewInt64(1 << 9) - -func (m *QueryFeedbackMap) append(k QueryFeedbackKey, qs []*QueryFeedback) bool { - remained := MaxQueryFeedbackCount.Load() - int64(m.Size) - if remained <= 0 { - return false - } - s, ok := m.Feedbacks[k] - if !ok || s == nil { - s = make([]*QueryFeedback, 0, 8) - } - l := mathutil.Min(int64(len(qs)), remained) - s = append(s, qs[:l]...) - m.Feedbacks[k] = s - m.Size = m.Size + int(l) - return true -} - -// SiftFeedbacks eliminates feedbacks which are overlapped with others. It is a tradeoff between -// feedback accuracy and its overhead. -func (m *QueryFeedbackMap) SiftFeedbacks() { - sc := &stmtctx.StatementContext{TimeZone: time.UTC} - for k, qs := range m.Feedbacks { - fbs := make([]Feedback, 0, len(qs)*2) - for _, q := range qs { - fbs = append(fbs, q.Feedback...) - } - if len(fbs) == 0 { - delete(m.Feedbacks, k) - continue - } - m.Feedbacks[k] = m.Feedbacks[k][:1] - m.Feedbacks[k][0].Feedback, _ = NonOverlappedFeedbacks(sc, fbs) - } - m.Size = len(m.Feedbacks) -} - -// Merge combines 2 collections of feedbacks. -func (m *QueryFeedbackMap) Merge(r *QueryFeedbackMap) { - for k, qs := range r.Feedbacks { - if !m.append(k, qs) { - break - } - } -} - -var ( - // MaxNumberOfRanges is the max number of ranges before split to collect feedback. - MaxNumberOfRanges = 20 - // FeedbackProbability is the probability to collect the feedback. - FeedbackProbability = atomic.NewFloat64(0) -) - -func init() { - // This is for solving import cycle. - // We need to read the value of FeedbackProbability when setting the variable tidb_analyze_version in sessionctx/variable package - // but we have imported sessionctx/variable in statistics package here. - variable.FeedbackProbability = FeedbackProbability -} - -// CalcErrorRate calculates the error rate the current QueryFeedback. -func (q *QueryFeedback) CalcErrorRate() float64 { - expected := float64(q.Expected) - if q.actual == 0 { - if expected == 0 { - return 0 - } - return 1 - } - return math.Abs(expected-float64(q.actual)) / float64(q.actual) -} - -// CollectFeedback decides whether to collect the feedback. It returns false when: -// 1: the feedback is not generated by select query; -// 2: the histogram is nil or has no buckets; -// 3: the number of scan ranges exceeds the limit because it may affect the performance; -// 4: it does not pass the probabilistic sampler. -func CollectFeedback(sc *stmtctx.StatementContext, q *QueryFeedback, numOfRanges int) bool { - if !sc.InSelectStmt { - return false - } - if q.Hist == nil || q.Hist.Len() == 0 { - return false - } - // #nosec G404 - if numOfRanges > MaxNumberOfRanges || rand.Float64() > FeedbackProbability.Load() { - return false - } - return true -} - -// DecodeToRanges decode the feedback to ranges. -func (q *QueryFeedback) DecodeToRanges(isIndex bool) ([]*ranger.Range, error) { - ranges := make([]*ranger.Range, 0, len(q.Feedback)) - for _, val := range q.Feedback { - low, high := *val.Lower, *val.Upper - var lowVal, highVal []types.Datum - if isIndex { - var err error - // As we do not know the origin length, just use a custom value here. - lowVal, _, err = codec.DecodeRange(low.GetBytes(), 4, nil, nil) - if err != nil { - return nil, errors.Trace(err) - } - highVal, _, err = codec.DecodeRange(high.GetBytes(), 4, nil, nil) - if err != nil { - return nil, errors.Trace(err) - } - } else { - _, lowInt, err := codec.DecodeInt(val.Lower.GetBytes()) - if err != nil { - return nil, errors.Trace(err) - } - _, highInt, err := codec.DecodeInt(val.Upper.GetBytes()) - if err != nil { - return nil, errors.Trace(err) - } - lowVal = []types.Datum{types.NewIntDatum(lowInt)} - highVal = []types.Datum{types.NewIntDatum(highInt)} - } - ranges = append(ranges, &(ranger.Range{ - LowVal: lowVal, - HighVal: highVal, - HighExclude: true, - Collators: collate.GetBinaryCollatorSlice(len(lowVal)), - })) - } - return ranges, nil -} - -// DecodeIntValues is called when the current Feedback stores encoded int values. -func (q *QueryFeedback) DecodeIntValues() *QueryFeedback { - nq := &QueryFeedback{} - nq.Feedback = make([]Feedback, 0, len(q.Feedback)) - for _, fb := range q.Feedback { - _, lowInt, err := codec.DecodeInt(fb.Lower.GetBytes()) - if err != nil { - logutil.BgLogger().Debug("decode feedback lower bound value to integer failed", zap.Binary("value", fb.Lower.GetBytes()), zap.Error(err)) - continue - } - _, highInt, err := codec.DecodeInt(fb.Upper.GetBytes()) - if err != nil { - logutil.BgLogger().Debug("decode feedback upper bound value to integer failed", zap.Binary("value", fb.Upper.GetBytes()), zap.Error(err)) - continue - } - low, high := types.NewIntDatum(lowInt), types.NewIntDatum(highInt) - nq.Feedback = append(nq.Feedback, Feedback{Lower: &low, Upper: &high, Count: fb.Count}) - } - return nq -} - -// StoreRanges stores the ranges for update. -func (q *QueryFeedback) StoreRanges(ranges []*ranger.Range) { - q.Feedback = make([]Feedback, 0, len(ranges)) - for _, ran := range ranges { - q.Feedback = append(q.Feedback, Feedback{&ran.LowVal[0], &ran.HighVal[0], 0, 0, 0}) - } -} - -// Invalidate is used to invalidate the query feedback. -func (q *QueryFeedback) Invalidate() { - q.Feedback = nil - q.Hist = nil - q.Valid.Store(false) - q.actual = -1 -} - -// Actual gets the actual row count. -func (q *QueryFeedback) Actual() int64 { - if !q.Valid.Load() { - return -1 - } - return q.actual -} - -// Update updates the query feedback. `startKey` is the start scan key of the partial result, used to find -// the range for update. `counts` is the scan counts of each range, used to update the feedback count info. -func (q *QueryFeedback) Update(startKey kv.Key, counts, ndvs []int64) { - failpoint.Inject("feedbackNoNDVCollect", func() { - ndvs = nil - }) - // Older versions do not have the counts info. - if len(counts) == 0 { - q.Invalidate() - return - } - sum := int64(0) - for _, count := range counts { - sum += count - } - metrics.DistSQLScanKeysPartialHistogram.Observe(float64(sum)) - q.actual += sum - if !q.Valid.Load() || q.Hist == nil { - return - } - - if q.Tp == IndexType { - startKey = tablecodec.CutIndexPrefix(startKey) - } else { - startKey = tablecodec.CutRowKeyPrefix(startKey) - } - // Find the range that startKey falls in. - idx := sort.Search(len(q.Feedback), func(i int) bool { - return bytes.Compare(q.Feedback[i].Lower.GetBytes(), startKey) > 0 - }) - idx-- - if idx < 0 { - return - } - // If the desc is true, the counts is reversed, so here we need to reverse it back. - if q.desc { - for i := 0; i < len(counts)/2; i++ { - j := len(counts) - i - 1 - counts[i], counts[j] = counts[j], counts[i] - } - for i := 0; i < len(ndvs)/2; i++ { - j := len(ndvs) - i - 1 - ndvs[i], ndvs[j] = ndvs[j], ndvs[i] - } - } - // Update the feedback count info. - for i, count := range counts { - if i+idx >= len(q.Feedback) { - q.Invalidate() - break - } - q.Feedback[i+idx].Count += count - } - for i, ndv := range ndvs { - q.Feedback[i+idx].Ndv += ndv - } -} - -// NonOverlappedFeedbacks extracts a set of feedbacks which are not overlapped with each other. -func NonOverlappedFeedbacks(sc *stmtctx.StatementContext, fbs []Feedback) ([]Feedback, bool) { - // Sort feedbacks by end point and start point incrementally, then pick every feedback that is not overlapped - // with the previous chosen feedbacks. - var existsErr bool - slices.SortFunc(fbs, func(i, j Feedback) int { - res, err := i.Upper.Compare(sc, j.Upper, collate.GetBinaryCollator()) - if err != nil { - existsErr = true - } - if existsErr || res != 0 { - return res - } - res, err = i.Lower.Compare(sc, j.Lower, collate.GetBinaryCollator()) - if err != nil { - existsErr = true - } - return res - }) - if existsErr { - return fbs, false - } - resFBs := make([]Feedback, 0, len(fbs)) - previousEnd := &types.Datum{} - for _, fb := range fbs { - res, err := previousEnd.Compare(sc, fb.Lower, collate.GetBinaryCollator()) - if err != nil { - return fbs, false - } - if res <= 0 { - resFBs = append(resFBs, fb) - previousEnd = fb.Upper - } - } - return resFBs, true -} - -// BucketFeedback stands for all the feedback for a bucket. -type BucketFeedback struct { - lower *types.Datum // The lower bound of the new bucket. - upper *types.Datum // The upper bound of the new bucket. - feedback []Feedback // All the feedback info in the same bucket. -} - -// outOfRange checks if the `val` is between `min` and `max`. -func outOfRange(sc *stmtctx.StatementContext, min, max, val *types.Datum) (int, error) { - result, err := val.Compare(sc, min, collate.GetBinaryCollator()) - if err != nil { - return 0, err - } - if result < 0 { - return result, nil - } - result, err = val.Compare(sc, max, collate.GetBinaryCollator()) - if err != nil { - return 0, err - } - if result > 0 { - return result, nil - } - return 0, nil -} - -// adjustFeedbackBoundaries adjust the feedback boundaries according to the `min` and `max`. -// If the feedback has no intersection with `min` and `max`, we could just skip this feedback. -func (f *Feedback) adjustFeedbackBoundaries(sc *stmtctx.StatementContext, min, max *types.Datum) (bool, error) { - result, err := outOfRange(sc, min, max, f.Lower) - if err != nil { - return false, err - } - if result > 0 { - return true, nil - } - if result < 0 { - f.Lower = min - } - result, err = outOfRange(sc, min, max, f.Upper) - if err != nil { - return false, err - } - if result < 0 { - return true, nil - } - if result > 0 { - f.Upper = max - } - return false, nil -} - -// buildBucketFeedback build the feedback for each bucket from the histogram feedback. -func buildBucketFeedback(h *Histogram, feedback *QueryFeedback) (map[int]*BucketFeedback, int) { - bktID2FB := make(map[int]*BucketFeedback) - if len(feedback.Feedback) == 0 { - return bktID2FB, 0 - } - total := 0 - sc := &stmtctx.StatementContext{TimeZone: time.UTC} - min, max := types.GetMinValue(h.Tp), types.GetMaxValue(h.Tp) - for _, fb := range feedback.Feedback { - skip, err := fb.adjustFeedbackBoundaries(sc, &min, &max) - if err != nil { - logutil.BgLogger().Debug("adjust feedback boundaries failed", zap.Error(err)) - continue - } - if skip { - continue - } - idx := h.Bounds.UpperBound(0, fb.Lower) - bktIdx := 0 - // The last bucket also stores the feedback that falls outside the upper bound. - if idx >= h.Bounds.NumRows()-1 { - bktIdx = h.Len() - 1 - } else if h.Len() == 1 { - bktIdx = 0 - } else { - if idx == 0 { - bktIdx = 0 - } else { - bktIdx = (idx - 1) / 2 - } - // Make sure that this feedback lies within the bucket. - if chunk.Compare(h.Bounds.GetRow(2*(bktIdx+1)), 0, fb.Upper) < 0 { - continue - } - } - total++ - bkt := bktID2FB[bktIdx] - if bkt == nil { - bkt = &BucketFeedback{lower: h.GetLower(bktIdx), upper: h.GetUpper(bktIdx)} - bktID2FB[bktIdx] = bkt - } - bkt.feedback = append(bkt.feedback, fb) - // Update the bound if necessary. - res, err := bkt.lower.Compare(nil, fb.Lower, collate.GetBinaryCollator()) - if err != nil { - logutil.BgLogger().Debug("compare datum failed", zap.Any("value1", bkt.lower), zap.Any("value2", fb.Lower), zap.Error(err)) - continue - } - if res > 0 { - bkt.lower = fb.Lower - } - res, err = bkt.upper.Compare(nil, fb.Upper, collate.GetBinaryCollator()) - if err != nil { - logutil.BgLogger().Debug("compare datum failed", zap.Any("value1", bkt.upper), zap.Any("value2", fb.Upper), zap.Error(err)) - continue - } - if res < 0 { - bkt.upper = fb.Upper - } - } - return bktID2FB, total -} - -// getBoundaries gets the new boundaries after split. -func (b *BucketFeedback) getBoundaries(num int) []types.Datum { - // Get all the possible new boundaries. - vals := make([]types.Datum, 0, len(b.feedback)*2+2) - for _, fb := range b.feedback { - vals = append(vals, *fb.Lower, *fb.Upper) - } - vals = append(vals, *b.lower) - err := types.SortDatums(nil, vals) - if err != nil { - logutil.BgLogger().Debug("sort datums failed", zap.Error(err)) - return []types.Datum{*b.lower, *b.upper} - } - total, interval := 0, len(vals)/num - // Pick values per `interval`. - for i := 0; i < len(vals); i, total = i+interval, total+1 { - vals[total] = vals[i] - } - // Append the upper bound. - vals[total] = *b.upper - vals = vals[:total+1] - total = 1 - // Erase the repeat values. - for i := 1; i < len(vals); i++ { - cmp, err := vals[total-1].Compare(nil, &vals[i], collate.GetBinaryCollator()) - if err != nil { - logutil.BgLogger().Debug("compare datum failed", zap.Any("value1", vals[total-1]), zap.Any("value2", vals[i]), zap.Error(err)) - continue - } - if cmp == 0 { - continue - } - vals[total] = vals[i] - total++ - } - return vals[:total] -} - -// There are only two types of datum in bucket: one is `Blob`, which is for index; the other one -// is `Int`, which is for primary key. -type bucket = Feedback - -// splitBucket firstly splits this "BucketFeedback" to "newNumBkts" new buckets, -// calculates the count for each new bucket, merge the new bucket whose count -// is smaller than "minBucketFraction*totalCount" with the next new bucket -// until the last new bucket. -func (b *BucketFeedback) splitBucket(newNumBkts int, totalCount float64, originBucketCount float64, originalNdv int64) []bucket { - // Split the bucket. - bounds := b.getBoundaries(newNumBkts + 1) - bkts := make([]bucket, 0, len(bounds)-1) - sc := &stmtctx.StatementContext{TimeZone: time.UTC} - for i := 1; i < len(bounds); i++ { - newBkt := bucket{&bounds[i-1], bounds[i].Clone(), 0, 0, 0} - // get bucket count - _, ratio := getOverlapFraction(Feedback{b.lower, b.upper, int64(originBucketCount), 0, 0}, newBkt) - countInNewBkt := originBucketCount * ratio - ndvInNewBkt := int64(float64(originalNdv) * ratio) - countInNewBkt, ndvInNewBkt = b.refineBucketCount(sc, newBkt, countInNewBkt, ndvInNewBkt) - // do not split if the count of result bucket is too small. - if countInNewBkt < minBucketFraction*totalCount { - bounds[i] = bounds[i-1] - continue - } - newBkt.Count = int64(countInNewBkt) - newBkt.Ndv = ndvInNewBkt - bkts = append(bkts, newBkt) - // To guarantee that each bucket's range will not overlap. - setNextValue(&bounds[i]) - } - return bkts -} - -// getOverlapFraction gets the overlap fraction of feedback and bucket range. In order to get the bucket count, it also -// returns the ratio between bucket fraction and feedback fraction. -func getOverlapFraction(fb Feedback, bkt bucket) (overlap, ratio float64) { - datums := make([]types.Datum, 0, 4) - datums = append(datums, *fb.Lower, *fb.Upper) - datums = append(datums, *bkt.Lower, *bkt.Upper) - err := types.SortDatums(nil, datums) - if err != nil { - return 0, 0 - } - minValue, maxValue := &datums[0], &datums[3] - fbLower := calcFraction4Datums(minValue, maxValue, fb.Lower) - fbUpper := calcFraction4Datums(minValue, maxValue, fb.Upper) - bktLower := calcFraction4Datums(minValue, maxValue, bkt.Lower) - bktUpper := calcFraction4Datums(minValue, maxValue, bkt.Upper) - ratio = (bktUpper - bktLower) / (fbUpper - fbLower) - // full overlap - if fbLower <= bktLower && bktUpper <= fbUpper { - return bktUpper - bktLower, ratio - } - if bktLower <= fbLower && fbUpper <= bktUpper { - return fbUpper - fbLower, ratio - } - // partial overlap - overlap = math.Min(bktUpper-fbLower, fbUpper-bktLower) - return overlap, ratio -} - -// mergeFullyContainedFeedback merges the max fraction of non-overlapped feedbacks that are fully contained in the bucket. -func (b *BucketFeedback) mergeFullyContainedFeedback(sc *stmtctx.StatementContext, bkt bucket) ( - sumFraction, sumCount float64, ndv int64, ok bool) { - feedbacks := make([]Feedback, 0, len(b.feedback)) - // Get all the fully contained feedbacks. - for _, fb := range b.feedback { - res, err := outOfRange(sc, bkt.Lower, bkt.Upper, fb.Lower) - if res != 0 || err != nil { - return 0, 0, 0, false - } - res, err = outOfRange(sc, bkt.Lower, bkt.Upper, fb.Upper) - if res != 0 || err != nil { - return 0, 0, 0, false - } - feedbacks = append(feedbacks, fb) - } - if len(feedbacks) == 0 { - return 0, 0, 0, false - } - sortedFBs, ok := NonOverlappedFeedbacks(sc, feedbacks) - if !ok { - return 0, 0, 0, false - } - for _, fb := range sortedFBs { - fraction, _ := getOverlapFraction(fb, bkt) - sumFraction += fraction - sumCount += float64(fb.Count) - ndv += fb.Ndv - } - return sumFraction, sumCount, ndv, true -} - -// refineBucketCount refine the newly split bucket count. It uses the feedback that overlaps most -// with the bucket to get the bucket count. -func (b *BucketFeedback) refineBucketCount(sc *stmtctx.StatementContext, bkt bucket, defaultCount float64, defaultNdv int64) (float64, int64) { - bestFraction := minBucketFraction - count := defaultCount - ndv := defaultNdv - sumFraction, sumCount, sumNdv, ok := b.mergeFullyContainedFeedback(sc, bkt) - if ok && sumFraction > bestFraction { - bestFraction = sumFraction - count = sumCount / sumFraction - ndv = int64(float64(sumNdv) / sumFraction) - } - for _, fb := range b.feedback { - fraction, ratio := getOverlapFraction(fb, bkt) - // choose the max overlap fraction - if fraction > bestFraction { - bestFraction = fraction - count = float64(fb.Count) * ratio - ndv = int64(float64(fb.Ndv) * ratio) - } - } - return count, ndv -} - -const ( - defaultSplitCount = 10 - splitPerFeedback = 10 - // defaultBucketCount is the number of buckets a column histogram has. - defaultBucketCount = 256 -) - -// getSplitCount gets the split count for the histogram. It is based on the intuition that: -// 1: If we have more remaining unused buckets, we can split more. -// 2: We cannot split too aggressive, thus we make it split every `splitPerFeedback`. -func getSplitCount(numFeedbacks, remainBuckets int) int { - // Split more if have more buckets available. - splitCount := mathutil.Max(remainBuckets, defaultSplitCount) - return mathutil.Min(splitCount, numFeedbacks/splitPerFeedback) -} - -type bucketScore struct { - id int - score float64 -} - -type bucketScores []bucketScore - -func (bs bucketScores) Len() int { return len(bs) } -func (bs bucketScores) Swap(i, j int) { bs[i], bs[j] = bs[j], bs[i] } -func (bs bucketScores) Less(i, j int) bool { return bs[i].score < bs[j].score } - -const ( - // To avoid the histogram been too imbalanced, we constrain the count of a bucket in range - // [minBucketFraction * totalCount, maxBucketFraction * totalCount]. - minBucketFraction = 1 / 10000.0 - maxBucketFraction = 1 / 10.0 -) - -// getBucketScore gets the score for merge this bucket with previous one. -// TODO: We also need to consider the bucket hit count. -func getBucketScore(bkts []bucket, totalCount float64, id int) bucketScore { - preCount, count := float64(bkts[id-1].Count), float64(bkts[id].Count) - // do not merge if the result bucket is too large - if (preCount + count) > maxBucketFraction*totalCount { - return bucketScore{id, math.MaxFloat64} - } - // Merge them if the result bucket is already too small. - if (preCount + count) < minBucketFraction*totalCount { - return bucketScore{id, 0} - } - low, mid, high := bkts[id-1].Lower, bkts[id-1].Upper, bkts[id].Upper - // If we choose to merge, err is the absolute estimate error for the previous bucket. - err := calcFraction4Datums(low, high, mid)*(preCount+count) - preCount - return bucketScore{id, math.Abs(err / (preCount + count))} -} - -func mergeBuckets(bkts []bucket, isNewBuckets []bool, bucketCount int, totalCount float64) []bucket { - mergeCount := len(bkts) - bucketCount - if mergeCount <= 0 { - return bkts - } - bs := make(bucketScores, 0, len(bkts)) - for i := 1; i < len(bkts); i++ { - // Do not merge the newly created buckets. - if !isNewBuckets[i] && !isNewBuckets[i-1] { - bs = append(bs, getBucketScore(bkts, totalCount, i)) - } - } - sort.Sort(bs) - ids := make([]int, 0, mergeCount) - for i := 0; i < mergeCount; i++ { - ids = append(ids, bs[i].id) - } - slices.Sort(ids) - idCursor, bktCursor := 0, 0 - for i := range bkts { - // Merge this bucket with last one. - if idCursor < mergeCount && ids[idCursor] == i { - bkts[bktCursor-1].Upper = bkts[i].Upper - bkts[bktCursor-1].Count += bkts[i].Count - bkts[bktCursor-1].Repeat = bkts[i].Repeat - bkts[bktCursor-1].Ndv += bkts[i].Ndv - idCursor++ - } else { - bkts[bktCursor] = bkts[i] - bktCursor++ - } - } - bkts = bkts[:bktCursor] - return bkts -} - -// splitBuckets split the histogram buckets according to the feedback. -func splitBuckets(h *Histogram, feedback *QueryFeedback, bucketCount int) ([]bucket, []bool, int64) { - bktID2FB, numTotalFBs := buildBucketFeedback(h, feedback) - buckets := make([]bucket, 0, h.Len()) - isNewBuckets := make([]bool, 0, h.Len()) - splitCount := getSplitCount(numTotalFBs, bucketCount-h.Len()) - for i := 0; i < h.Len(); i++ { - bktFB, ok := bktID2FB[i] - // No feedback, just use the original one. - if !ok { - buckets = append(buckets, bucket{h.GetLower(i), h.GetUpper(i), h.bucketCount(i), h.Buckets[i].Repeat, h.Buckets[i].NDV}) - isNewBuckets = append(isNewBuckets, false) - continue - } - // Distribute the total split count to bucket based on number of bucket feedback. - newBktNums := splitCount * len(bktFB.feedback) / numTotalFBs - bkts := bktFB.splitBucket(newBktNums, h.TotalRowCount(), float64(h.bucketCount(i)), h.Buckets[i].NDV) - buckets = append(buckets, bkts...) - if len(bkts) == 1 { - isNewBuckets = append(isNewBuckets, false) - } else { - for i := 0; i < len(bkts); i++ { - isNewBuckets = append(isNewBuckets, true) - } - } - } - totCount := int64(0) - for _, bkt := range buckets { - totCount += bkt.Count - } - return buckets, isNewBuckets, totCount -} - -// UpdateHistogram updates the histogram according buckets. -func UpdateHistogram(h *Histogram, feedback *QueryFeedback, statsVer int) *Histogram { - return UpdateHistogramWithBucketCount(h, feedback, statsVer, defaultBucketCount) -} - -// UpdateHistogramWithBucketCount updates the histogram according buckets with customized -// bucketCount for testing. -func UpdateHistogramWithBucketCount(h *Histogram, feedback *QueryFeedback, statsVer int, bucketCount int) *Histogram { - if statsVer < Version2 { - // If it's the stats we haven't maintained the bucket NDV yet. Reset the ndv. - for i := range feedback.Feedback { - feedback.Feedback[i].Ndv = 0 - } - } - buckets, isNewBuckets, totalCount := splitBuckets(h, feedback, bucketCount) - buckets = mergeBuckets(buckets, isNewBuckets, bucketCount, float64(totalCount)) - hist := buildNewHistogram(h, buckets) - // Update the NDV of primary key column. - if feedback.Tp == PkType { - hist.NDV = int64(hist.TotalRowCount()) - } else if feedback.Tp == IndexType && statsVer == 2 { - // If we maintained the NDV of bucket. We can also update the total ndv. - totNdv := int64(0) - for _, bkt := range buckets { - totNdv += bkt.Ndv - } - hist.NDV = totNdv - } - return hist -} - -// UpdateCMSketchAndTopN updates the CMSketch and TopN by feedback. -func UpdateCMSketchAndTopN(c *CMSketch, t *TopN, eqFeedbacks []Feedback) (*CMSketch, *TopN) { - if c == nil || len(eqFeedbacks) == 0 { - return c, t - } - newCMSketch := c.Copy() - newTopN := t.Copy() - for _, fb := range eqFeedbacks { - updateValueBytes(newCMSketch, newTopN, fb.Lower.GetBytes(), uint64(fb.Count)) - } - return newCMSketch, newTopN -} - -func buildNewHistogram(h *Histogram, buckets []bucket) *Histogram { - hist := NewHistogram(h.ID, h.NDV, h.NullCount, h.LastUpdateVersion, h.Tp, len(buckets), h.TotColSize) - preCount := int64(0) - for _, bkt := range buckets { - hist.AppendBucketWithNDV(bkt.Lower, bkt.Upper, bkt.Count+preCount, bkt.Repeat, bkt.Ndv) - preCount += bkt.Count - } - return hist -} - -// queryFeedback is used to serialize the QueryFeedback. -type queryFeedback struct { - IntRanges []int64 - // HashValues is the murmur hash values for each index point. - // Note that index points will be stored in `IndexPoints`, we keep it here only for compatibility. - HashValues []uint64 - IndexRanges [][]byte - // IndexPoints stores the value of each equal condition. - IndexPoints [][]byte - // Counts is the number of scan keys in each range. It first stores the count for `IntRanges`, `IndexRanges` or `ColumnRanges`. - // After that, it stores the Ranges for `HashValues`. - Counts []int64 - ColumnRanges [][]byte - - Ndvs []int64 -} - -func encodePKFeedback(q *QueryFeedback) (*queryFeedback, error) { - pb := &queryFeedback{} - for _, fb := range q.Feedback { - // There is no need to update the point queries. - if bytes.Compare(kv.Key(fb.Lower.GetBytes()).PrefixNext(), fb.Upper.GetBytes()) >= 0 { - continue - } - _, low, err := codec.DecodeInt(fb.Lower.GetBytes()) - if err != nil { - return nil, errors.Trace(err) - } - _, high, err := codec.DecodeInt(fb.Upper.GetBytes()) - if err != nil { - return nil, errors.Trace(err) - } - pb.IntRanges = append(pb.IntRanges, low, high) - pb.Counts = append(pb.Counts, fb.Count) - pb.Ndvs = append(pb.Ndvs, fb.Ndv) - } - return pb, nil -} - -func encodeIndexFeedback(q *QueryFeedback) *queryFeedback { - pb := &queryFeedback{} - var pointCounts []int64 - for _, fb := range q.Feedback { - if bytes.Compare(kv.Key(fb.Lower.GetBytes()).PrefixNext(), fb.Upper.GetBytes()) >= 0 { - pb.IndexPoints = append(pb.IndexPoints, fb.Lower.GetBytes()) - pointCounts = append(pointCounts, fb.Count) - pb.Ndvs = append(pb.Ndvs, fb.Ndv) - } else { - pb.IndexRanges = append(pb.IndexRanges, fb.Lower.GetBytes(), fb.Upper.GetBytes()) - pb.Counts = append(pb.Counts, fb.Count) - pb.Ndvs = append(pb.Ndvs, fb.Ndv) - } - } - pb.Counts = append(pb.Counts, pointCounts...) - return pb -} - -func encodeColumnFeedback(q *QueryFeedback) (*queryFeedback, error) { - pb := &queryFeedback{} - sc := stmtctx.StatementContext{TimeZone: time.UTC} - for _, fb := range q.Feedback { - lowerBytes, err := codec.EncodeKey(&sc, nil, *fb.Lower) - if err != nil { - return nil, errors.Trace(err) - } - upperBytes, err := codec.EncodeKey(&sc, nil, *fb.Upper) - if err != nil { - return nil, errors.Trace(err) - } - pb.ColumnRanges = append(pb.ColumnRanges, lowerBytes, upperBytes) - pb.Counts = append(pb.Counts, fb.Count) - } - return pb, nil -} - -// EncodeFeedback encodes the given feedback to byte slice. -func EncodeFeedback(q *QueryFeedback) ([]byte, error) { - var pb *queryFeedback - var err error - switch q.Tp { - case PkType: - pb, err = encodePKFeedback(q) - case IndexType: - pb = encodeIndexFeedback(q) - case ColType: - pb, err = encodeColumnFeedback(q) - } - if err != nil { - return nil, errors.Trace(err) - } - var buf bytes.Buffer - enc := gob.NewEncoder(&buf) - err = enc.Encode(pb) - return buf.Bytes(), errors.Trace(err) -} - -func decodeFeedbackForIndex(q *QueryFeedback, pb *queryFeedback, c *CMSketch, t *TopN) { - q.Tp = IndexType - // decode the index range feedback - for i := 0; i < len(pb.IndexRanges); i += 2 { - lower, upper := types.NewBytesDatum(pb.IndexRanges[i]), types.NewBytesDatum(pb.IndexRanges[i+1]) - q.Feedback = append(q.Feedback, Feedback{&lower, &upper, pb.Counts[i/2], 0, pb.Ndvs[i/2]}) - } - if c != nil { - // decode the index point feedback, just set value count in CM Sketch - start := len(pb.IndexRanges) / 2 - if len(pb.HashValues) > 0 { - for i := 0; i < len(pb.HashValues); i += 2 { - c.setValue(pb.HashValues[i], pb.HashValues[i+1], uint64(pb.Counts[start+i/2])) - } - return - } - for i := 0; i < len(pb.IndexPoints); i++ { - updateValueBytes(c, t, pb.IndexPoints[i], uint64(pb.Counts[start+i])) - } - } -} - -func decodeFeedbackForPK(q *QueryFeedback, pb *queryFeedback, isUnsigned bool) { - q.Tp = PkType - // decode feedback for primary key - for i := 0; i < len(pb.IntRanges); i += 2 { - var lower, upper types.Datum - if isUnsigned { - lower.SetUint64(uint64(pb.IntRanges[i])) - upper.SetUint64(uint64(pb.IntRanges[i+1])) - } else { - lower.SetInt64(pb.IntRanges[i]) - upper.SetInt64(pb.IntRanges[i+1]) - } - q.Feedback = append(q.Feedback, Feedback{&lower, &upper, pb.Counts[i/2], 0, pb.Ndvs[i/2]}) - } -} - -// ConvertDatumsType converts the datums type to `ft`. -func ConvertDatumsType(vals []types.Datum, ft *types.FieldType, loc *time.Location) error { - for i, val := range vals { - if val.Kind() == types.KindMinNotNull || val.Kind() == types.KindMaxValue { - continue - } - newVal, err := tablecodec.UnflattenDatums([]types.Datum{val}, []*types.FieldType{ft}, loc) - if err != nil { - return err - } - vals[i] = newVal[0] - } - return nil -} - -func decodeColumnBounds(data []byte, ft *types.FieldType) ([]types.Datum, error) { - vals, _, err := codec.DecodeRange(data, 1, nil, nil) - if err != nil { - return nil, err - } - err = ConvertDatumsType(vals, ft, time.UTC) - return vals, err -} - -func decodeFeedbackForColumn(q *QueryFeedback, pb *queryFeedback, ft *types.FieldType) error { - q.Tp = ColType - for i := 0; i < len(pb.ColumnRanges); i += 2 { - low, err := decodeColumnBounds(pb.ColumnRanges[i], ft) - if err != nil { - return err - } - high, err := decodeColumnBounds(pb.ColumnRanges[i+1], ft) - if err != nil { - return err - } - q.Feedback = append(q.Feedback, Feedback{&low[0], &high[0], pb.Counts[i/2], 0, 0}) - } - return nil -} - -// DecodeFeedback decodes a byte slice to feedback. -func DecodeFeedback(val []byte, q *QueryFeedback, c *CMSketch, t *TopN, ft *types.FieldType) error { - buf := bytes.NewBuffer(val) - dec := gob.NewDecoder(buf) - pb := &queryFeedback{} - err := dec.Decode(pb) - if err != nil { - return errors.Trace(err) - } - if len(pb.IndexRanges) > 0 || len(pb.HashValues) > 0 || len(pb.IndexPoints) > 0 { - decodeFeedbackForIndex(q, pb, c, t) - } else if len(pb.IntRanges) > 0 { - decodeFeedbackForPK(q, pb, mysql.HasUnsignedFlag(ft.GetFlag())) - } else { - err = decodeFeedbackForColumn(q, pb, ft) - } - return err -} - -// SplitFeedbackByQueryType splits the feedbacks into equality feedbacks and range feedbacks. -func SplitFeedbackByQueryType(feedbacks []Feedback) ([]Feedback, []Feedback) { - var eqFB, ranFB []Feedback - for _, fb := range feedbacks { - // Use `>=` here because sometimes the lower is equal to upper. - if bytes.Compare(kv.Key(fb.Lower.GetBytes()).PrefixNext(), fb.Upper.GetBytes()) >= 0 { - eqFB = append(eqFB, fb) - } else { - ranFB = append(ranFB, fb) - } - } - return eqFB, ranFB -} - -// setNextValue sets the next value for the given datum. For types like float, -// we do not set because it is not discrete and does not matter too much when estimating the scalar info. -func setNextValue(d *types.Datum) { - switch d.Kind() { - case types.KindBytes, types.KindString: - // Here is the encoded value instead of string value, so SetBytes is enough. - d.SetBytes(kv.Key(d.GetBytes()).PrefixNext()) - case types.KindInt64: - d.SetInt64(d.GetInt64() + 1) - case types.KindUint64: - d.SetUint64(d.GetUint64() + 1) - case types.KindMysqlDuration: - duration := d.GetMysqlDuration() - duration.Duration = duration.Duration + 1 - d.SetMysqlDuration(duration) - case types.KindMysqlTime: - t := d.GetMysqlTime() - sc := &stmtctx.StatementContext{TimeZone: types.BoundTimezone} - if _, err := t.Add(sc, types.Duration{Duration: 1, Fsp: 0}); err != nil { - log.Error(errors.ErrorStack(err)) - } - d.SetMysqlTime(t) - } -} - -// SupportColumnType checks if the type of the column can be updated by feedback. -func SupportColumnType(ft *types.FieldType) bool { - switch ft.GetType() { - case mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24, mysql.TypeLong, mysql.TypeLonglong, mysql.TypeFloat, - mysql.TypeDouble, mysql.TypeString, mysql.TypeVarString, mysql.TypeVarchar, mysql.TypeBlob, mysql.TypeTinyBlob, mysql.TypeMediumBlob, mysql.TypeLongBlob, - mysql.TypeNewDecimal, mysql.TypeDuration, mysql.TypeDate, mysql.TypeDatetime, mysql.TypeTimestamp: - return true - } - return false -} diff --git a/statistics/feedback_test.go b/statistics/feedback_test.go deleted file mode 100644 index 33366f2805e7e..0000000000000 --- a/statistics/feedback_test.go +++ /dev/null @@ -1,310 +0,0 @@ -// Copyright 2018 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package statistics - -import ( - "bytes" - "testing" - - "github.com/pingcap/log" - "github.com/pingcap/tidb/parser/mysql" - "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/codec" - "github.com/stretchr/testify/require" - "go.uber.org/zap" -) - -func newFeedback(lower, upper, count, ndv int64) Feedback { - low, upp := types.NewIntDatum(lower), types.NewIntDatum(upper) - return Feedback{&low, &upp, count, 0, ndv} -} - -func genFeedbacks(lower, upper int64) []Feedback { - var feedbacks []Feedback - for i := lower; i < upper; i++ { - feedbacks = append(feedbacks, newFeedback(i, upper, upper-i+1, upper-i+1)) - } - return feedbacks -} - -func appendBucket(h *Histogram, l, r int64) { - lower, upper := types.NewIntDatum(l), types.NewIntDatum(r) - h.AppendBucketWithNDV(&lower, &upper, 0, 0, 0) -} - -func genHistogram() *Histogram { - h := NewHistogram(0, 0, 0, 0, types.NewFieldType(mysql.TypeLong), 5, 0) - appendBucket(h, 1, 1) - appendBucket(h, 2, 3) - appendBucket(h, 5, 7) - appendBucket(h, 10, 20) - appendBucket(h, 30, 50) - return h -} - -func TestUpdateHistogram(t *testing.T) { - feedbacks := []Feedback{ - newFeedback(0, 1, 10000, 1), - newFeedback(1, 2, 1, 1), - newFeedback(2, 3, 3, 1), - newFeedback(4, 5, 2, 1), - newFeedback(5, 7, 4, 1), - } - feedbacks = append(feedbacks, genFeedbacks(8, 20)...) - feedbacks = append(feedbacks, genFeedbacks(21, 60)...) - - q := NewQueryFeedback(0, genHistogram(), 0, false) - q.Feedback = feedbacks - require.Equal(t, - "column:0 ndv:10053 totColSize:0\n"+ - "num: 10001 lower_bound: 0 upper_bound: 2 repeats: 0 ndv: 2\n"+ - "num: 7 lower_bound: 2 upper_bound: 5 repeats: 0 ndv: 2\n"+ - "num: 4 lower_bound: 5 upper_bound: 7 repeats: 0 ndv: 1\n"+ - "num: 11 lower_bound: 10 upper_bound: 20 repeats: 0 ndv: 11\n"+ - "num: 19 lower_bound: 30 upper_bound: 49 repeats: 0 ndv: 19\n"+ - "num: 11 lower_bound: 50 upper_bound: 60 repeats: 0 ndv: 11", - UpdateHistogramWithBucketCount(q.Hist, q, Version2, 7).ToString(0)) -} - -func TestSplitBuckets(t *testing.T) { - // test bucket split - feedbacks := []Feedback{newFeedback(0, 1, 1, 1)} - for i := 0; i < 100; i++ { - feedbacks = append(feedbacks, newFeedback(10, 15, 5, 5)) - } - q := NewQueryFeedback(0, genHistogram(), 0, false) - q.Feedback = feedbacks - oldCnts := make([]int64, q.Hist.Len()) - for i := range q.Hist.Buckets { - oldCnts[i] = q.Hist.bucketCount(i) - } - oldNdvs := make([]int64, q.Hist.Len()) - for i := range q.Hist.Buckets { - oldNdvs[i] = q.Hist.Buckets[i].NDV - } - log.Warn("in test", zap.Int64s("ndvs", oldNdvs), zap.Int64s("cnts", oldCnts)) - buckets, isNewBuckets, totalCount := splitBuckets(q.Hist, q, defaultBucketCount) - ndvs := make([]int64, len(buckets)) - for i := range buckets { - ndvs[i] = buckets[i].Ndv - } - log.Warn("in test", zap.Int64s("ndvs", ndvs)) - require.Equal(t, - "column:0 ndv:0 totColSize:0\n"+ - "num: 1 lower_bound: 0 upper_bound: 1 repeats: 0 ndv: 1\n"+ - "num: 0 lower_bound: 2 upper_bound: 3 repeats: 0 ndv: 0\n"+ - "num: 0 lower_bound: 5 upper_bound: 7 repeats: 0 ndv: 0\n"+ - "num: 5 lower_bound: 10 upper_bound: 15 repeats: 0 ndv: 5\n"+ - "num: 0 lower_bound: 16 upper_bound: 20 repeats: 0 ndv: 0\n"+ - "num: 0 lower_bound: 30 upper_bound: 50 repeats: 0 ndv: 0", - buildNewHistogram(q.Hist, buckets).ToString(0)) - require.Equal(t, []bool{false, false, false, true, true, false}, isNewBuckets) - require.Equal(t, int64(6), totalCount) - - // test do not split if the bucket count is too small - feedbacks = []Feedback{newFeedback(0, 1, 100000, 1)} - for i := 0; i < 100; i++ { - feedbacks = append(feedbacks, newFeedback(10, 15, 1, 1)) - } - q = NewQueryFeedback(0, genHistogram(), 0, false) - q.Feedback = feedbacks - buckets, isNewBuckets, totalCount = splitBuckets(q.Hist, q, defaultBucketCount) - require.Equal(t, - "column:0 ndv:0 totColSize:0\n"+ - "num: 100000 lower_bound: 0 upper_bound: 1 repeats: 0 ndv: 1\n"+ - "num: 0 lower_bound: 2 upper_bound: 3 repeats: 0 ndv: 0\n"+ - "num: 0 lower_bound: 5 upper_bound: 7 repeats: 0 ndv: 0\n"+ - "num: 1 lower_bound: 10 upper_bound: 15 repeats: 0 ndv: 1\n"+ - "num: 0 lower_bound: 16 upper_bound: 20 repeats: 0 ndv: 0\n"+ - "num: 0 lower_bound: 30 upper_bound: 50 repeats: 0 ndv: 0", - buildNewHistogram(q.Hist, buckets).ToString(0)) - require.Equal(t, []bool{false, false, false, true, true, false}, isNewBuckets) - require.Equal(t, int64(100001), totalCount) - - // test do not split if the result bucket count is too small - h := NewHistogram(0, 0, 0, 0, types.NewFieldType(mysql.TypeLong), 5, 0) - appendBucket(h, 0, 1000000) - h.Buckets[0].Count = 1000000 - h.Buckets[0].NDV = 1000000 - feedbacks = feedbacks[:0] - for i := 0; i < 100; i++ { - feedbacks = append(feedbacks, newFeedback(0, 10, 1, 1)) - } - q = NewQueryFeedback(0, h, 0, false) - q.Feedback = feedbacks - buckets, isNewBuckets, totalCount = splitBuckets(q.Hist, q, defaultBucketCount) - require.Equal(t, - "column:0 ndv:0 totColSize:0\n"+ - "num: 1000000 lower_bound: 0 upper_bound: 1000000 repeats: 0 ndv: 1000000", - buildNewHistogram(q.Hist, buckets).ToString(0)) - require.Equal(t, []bool{false}, isNewBuckets) - require.Equal(t, int64(1000000), totalCount) - - // test split even if the feedback range is too small - h = NewHistogram(0, 0, 0, 0, types.NewFieldType(mysql.TypeLong), 5, 0) - appendBucket(h, 0, 1000000) - feedbacks = feedbacks[:0] - for i := 0; i < 100; i++ { - feedbacks = append(feedbacks, newFeedback(0, 10, 1, 1)) - } - q = NewQueryFeedback(0, h, 0, false) - q.Feedback = feedbacks - buckets, isNewBuckets, totalCount = splitBuckets(q.Hist, q, defaultBucketCount) - require.Equal(t, - "column:0 ndv:0 totColSize:0\n"+ - "num: 1 lower_bound: 0 upper_bound: 10 repeats: 0 ndv: 1\n"+ - "num: 0 lower_bound: 11 upper_bound: 1000000 repeats: 0 ndv: 0", - buildNewHistogram(q.Hist, buckets).ToString(0)) - require.Equal(t, []bool{true, true}, isNewBuckets) - require.Equal(t, int64(1), totalCount) - - // test merge the non-overlapped feedbacks. - h = NewHistogram(0, 0, 0, 0, types.NewFieldType(mysql.TypeLong), 5, 0) - appendBucket(h, 0, 10000) - feedbacks = feedbacks[:0] - feedbacks = append(feedbacks, newFeedback(0, 4000, 4000, 4000)) - feedbacks = append(feedbacks, newFeedback(4001, 9999, 1000, 1000)) - q = NewQueryFeedback(0, h, 0, false) - q.Feedback = feedbacks - buckets, isNewBuckets, totalCount = splitBuckets(q.Hist, q, defaultBucketCount) - require.Equal(t, - "column:0 ndv:0 totColSize:0\n"+ - "num: 5001 lower_bound: 0 upper_bound: 10000 repeats: 0 ndv: 5001", - buildNewHistogram(q.Hist, buckets).ToString(0)) - require.Equal(t, []bool{false}, isNewBuckets) - require.Equal(t, int64(5001), totalCount) -} - -func TestMergeBuckets(t *testing.T) { - tests := []struct { - points []int64 - counts []int64 - ndvs []int64 - isNewBuckets []bool - bucketCount int - result string - }{ - { - points: []int64{1, 2}, - counts: []int64{1}, - ndvs: []int64{1}, - isNewBuckets: []bool{false}, - bucketCount: 1, - result: "column:0 ndv:0 totColSize:0\nnum: 1 lower_bound: 1 upper_bound: 2 repeats: 0 ndv: 1", - }, - { - points: []int64{1, 2, 2, 3, 3, 4}, - counts: []int64{100000, 1, 1}, - ndvs: []int64{1, 1, 1}, - isNewBuckets: []bool{false, false, false}, - bucketCount: 2, - result: "column:0 ndv:0 totColSize:0\n" + - "num: 100000 lower_bound: 1 upper_bound: 2 repeats: 0 ndv: 1\n" + - "num: 2 lower_bound: 2 upper_bound: 4 repeats: 0 ndv: 2", - }, - // test do not Merge if the result bucket count is too large - { - points: []int64{1, 2, 2, 3, 3, 4, 4, 5}, - counts: []int64{1, 1, 100000, 100000}, - ndvs: []int64{1, 1, 1, 1}, - isNewBuckets: []bool{false, false, false, false}, - bucketCount: 3, - result: "column:0 ndv:0 totColSize:0\n" + - "num: 2 lower_bound: 1 upper_bound: 3 repeats: 0 ndv: 2\n" + - "num: 100000 lower_bound: 3 upper_bound: 4 repeats: 0 ndv: 1\n" + - "num: 100000 lower_bound: 4 upper_bound: 5 repeats: 0 ndv: 1", - }, - } - for _, tt := range tests { - require.Equal(t, len(tt.ndvs), len(tt.counts)) - bkts := make([]bucket, 0, len(tt.counts)) - totalCount := int64(0) - for i := 0; i < len(tt.counts); i++ { - lower, upper := types.NewIntDatum(tt.points[2*i]), types.NewIntDatum(tt.points[2*i+1]) - bkts = append(bkts, bucket{&lower, &upper, tt.counts[i], 0, tt.ndvs[i]}) - totalCount += tt.counts[i] - } - bkts = mergeBuckets(bkts, tt.isNewBuckets, tt.bucketCount, float64(totalCount)) - result := buildNewHistogram(&Histogram{Tp: types.NewFieldType(mysql.TypeLong)}, bkts).ToString(0) - require.Equal(t, tt.result, result) - } -} - -func encodeInt(v int64) *types.Datum { - val := codec.EncodeInt(nil, v) - d := types.NewBytesDatum(val) - return &d -} - -func TestFeedbackEncoding(t *testing.T) { - hist := NewHistogram(0, 0, 0, 0, types.NewFieldType(mysql.TypeLong), 0, 0) - q := &QueryFeedback{Hist: hist, Tp: PkType} - q.Feedback = append(q.Feedback, Feedback{encodeInt(0), encodeInt(3), 1, 0, 1}) - q.Feedback = append(q.Feedback, Feedback{encodeInt(0), encodeInt(5), 1, 0, 1}) - val, err := EncodeFeedback(q) - require.NoError(t, err) - rq := &QueryFeedback{} - require.NoError(t, DecodeFeedback(val, rq, nil, nil, hist.Tp)) - for _, fb := range rq.Feedback { - fb.Lower.SetBytes(codec.EncodeInt(nil, fb.Lower.GetInt64())) - fb.Upper.SetBytes(codec.EncodeInt(nil, fb.Upper.GetInt64())) - } - require.True(t, q.Equal(rq)) - - hist.Tp = types.NewFieldType(mysql.TypeBlob) - q = &QueryFeedback{Hist: hist} - q.Feedback = append(q.Feedback, Feedback{encodeInt(0), encodeInt(3), 1, 0, 1}) - q.Feedback = append(q.Feedback, Feedback{encodeInt(0), encodeInt(1), 1, 0, 1}) - val, err = EncodeFeedback(q) - require.NoError(t, err) - rq = &QueryFeedback{} - cms := NewCMSketch(4, 4) - require.NoError(t, DecodeFeedback(val, rq, cms, nil, hist.Tp)) - require.Equal(t, uint64(1), cms.QueryBytes(codec.EncodeInt(nil, 0))) - q.Feedback = q.Feedback[:1] - require.True(t, q.Equal(rq)) -} - -// Equal tests if two query feedback equal, it is only used in test. -func (q *QueryFeedback) Equal(rq *QueryFeedback) bool { - if len(q.Feedback) != len(rq.Feedback) { - return false - } - for i, fb := range q.Feedback { - rfb := rq.Feedback[i] - if fb.Count != rfb.Count { - return false - } - if fb.Ndv != rfb.Ndv { - return false - } - if fb.Lower.Kind() == types.KindInt64 { - if fb.Lower.GetInt64() != rfb.Lower.GetInt64() { - return false - } - if fb.Upper.GetInt64() != rfb.Upper.GetInt64() { - return false - } - } else { - if !bytes.Equal(fb.Lower.GetBytes(), rfb.Lower.GetBytes()) { - return false - } - if !bytes.Equal(fb.Upper.GetBytes(), rfb.Upper.GetBytes()) { - return false - } - } - } - return true -} diff --git a/statistics/handle/BUILD.bazel b/statistics/handle/BUILD.bazel index 651703064b9e8..a6c84ae37647d 100644 --- a/statistics/handle/BUILD.bazel +++ b/statistics/handle/BUILD.bazel @@ -35,18 +35,14 @@ go_library( "//types", "//util", "//util/chunk", - "//util/codec", - "//util/collate", "//util/logutil", "//util/mathutil", - "//util/ranger", "//util/sqlexec", "//util/syncutil", "//util/timeutil", "@com_github_ngaut_pools//:pools", "@com_github_pingcap_errors//:errors", "@com_github_pingcap_failpoint//:failpoint", - "@com_github_pingcap_log//:log", "@com_github_pingcap_tipb//go-tipb", "@com_github_tiancaiamao_gp//:gp", "@com_github_tikv_client_go_v2//oracle", diff --git a/statistics/handle/gc.go b/statistics/handle/gc.go index bd233db5b4aaf..b36860478e48c 100644 --- a/statistics/handle/gc.go +++ b/statistics/handle/gc.go @@ -290,9 +290,6 @@ func (h *Handle) DeleteTableStatsFromKV(statsIDs []int64) (err error) { if _, err = exec.ExecuteInternal(ctx, "delete from mysql.stats_top_n where table_id = %?", statsID); err != nil { return err } - if _, err = exec.ExecuteInternal(ctx, "delete from mysql.stats_feedback where table_id = %?", statsID); err != nil { - return err - } if _, err = exec.ExecuteInternal(ctx, "update mysql.stats_extended set version = %?, status = %? where table_id = %? and status in (%?, %?)", startTS, statistics.ExtendedStatsDeleted, statsID, statistics.ExtendedStatsAnalyzed, statistics.ExtendedStatsInited); err != nil { return err } diff --git a/statistics/handle/handle.go b/statistics/handle/handle.go index dbc665c72d098..8d594e9e8623e 100644 --- a/statistics/handle/handle.go +++ b/statistics/handle/handle.go @@ -93,12 +93,6 @@ type Handle struct { // written only after acquiring the lock. statsCache *cache.StatsCachePointer - // feedback is used to store query feedback info. - feedback struct { - data *statistics.QueryFeedbackMap - sync.Mutex - } - // globalMap contains all the delta map from collectors when we dump them to KV. globalMap struct { data tableDeltaMap @@ -119,8 +113,6 @@ type Handle struct { mu struct { ctx sessionctx.Context - // rateMap contains the error rate delta from feedback. - rateMap errorRateDeltaMap syncutil.RWMutex } @@ -460,9 +452,6 @@ func (h *Handle) Clear() { for len(h.ddlEventCh) > 0 { <-h.ddlEventCh } - h.feedback.Lock() - h.feedback.data = statistics.NewQueryFeedbackMap() - h.feedback.Unlock() h.mu.ctx.GetSessionVars().InitChunkSize = 1 h.mu.ctx.GetSessionVars().MaxChunkSize = 1 h.mu.ctx.GetSessionVars().EnableChunkRPC = false @@ -474,7 +463,6 @@ func (h *Handle) Clear() { h.colMap.Lock() h.colMap.data = make(colStatsUsageMap) h.colMap.Unlock() - h.mu.rateMap = make(errorRateDeltaMap) h.mu.Unlock() } @@ -499,14 +487,12 @@ func NewHandle(ctx, initStatsCtx sessionctx.Context, lease time.Duration, pool s handle.initStatsCtx = initStatsCtx handle.lease.Store(lease) handle.mu.ctx = ctx - handle.mu.rateMap = make(errorRateDeltaMap) statsCache, err := cache.NewStatsCachePointer() if err != nil { return nil, err } handle.statsCache = statsCache handle.globalMap.data = make(tableDeltaMap) - handle.feedback.data = statistics.NewQueryFeedbackMap() handle.colMap.data = make(colStatsUsageMap) handle.StatsLoad.SubCtxs = make([]sessionctx.Context, cfg.Performance.StatsLoadConcurrency) handle.StatsLoad.NeededItemsCh = make(chan *NeededItemTask, cfg.Performance.StatsLoadQueueSize) @@ -529,16 +515,6 @@ func (h *Handle) SetLease(lease time.Duration) { h.lease.Store(lease) } -// GetQueryFeedback gets the query feedback. It is only used in test. -func (h *Handle) GetQueryFeedback() *statistics.QueryFeedbackMap { - h.feedback.Lock() - defer func() { - h.feedback.data = statistics.NewQueryFeedbackMap() - h.feedback.Unlock() - }() - return h.feedback.data -} - // DurationToTS converts duration to timestamp. func DurationToTS(d time.Duration) uint64 { return oracle.ComposeTS(d.Nanoseconds()/int64(time.Millisecond), 0) @@ -1233,9 +1209,6 @@ func (h *Handle) FlushStats() { if err := h.DumpStatsDeltaToKV(DumpAll); err != nil { logutil.BgLogger().Error("dump stats delta fail", zap.String("category", "stats"), zap.Error(err)) } - if err := h.DumpStatsFeedbackToKV(); err != nil { - logutil.BgLogger().Error("dump stats feedback fail", zap.String("category", "stats"), zap.Error(err)) - } } // TableStatsFromStorage loads table stats info from storage. @@ -1261,16 +1234,6 @@ func (h *Handle) TableStatsFromStorage(tableInfo *model.TableInfo, physicalID in if reader.IsHistory() || statsTbl == nil { return statsTbl, nil } - for histID, idx := range statsTbl.Indices { - if statistics.IsAnalyzed(idx.Flag) { - h.mu.rateMap.clear(physicalID, histID, true) - } - } - for histID, col := range statsTbl.Columns { - if statistics.IsAnalyzed(col.Flag) { - h.mu.rateMap.clear(physicalID, histID, false) - } - } return statsTbl, nil } diff --git a/statistics/handle/handletest/globalstats/BUILD.bazel b/statistics/handle/handletest/globalstats/BUILD.bazel index fa94cf7546096..92ab53fa463ea 100644 --- a/statistics/handle/handletest/globalstats/BUILD.bazel +++ b/statistics/handle/handletest/globalstats/BUILD.bazel @@ -8,12 +8,10 @@ go_test( "main_test.go", ], flaky = True, - shard_count = 11, + shard_count = 10, deps = [ "//parser/model", - "//statistics", "//statistics/handle", - "//statistics/handle/internal", "//testkit", "//testkit/testsetup", "@com_github_stretchr_testify//require", diff --git a/statistics/handle/handletest/globalstats/globalstats_test.go b/statistics/handle/handletest/globalstats/globalstats_test.go index a4f4a2dc22795..3b8e6d95e4f1a 100644 --- a/statistics/handle/handletest/globalstats/globalstats_test.go +++ b/statistics/handle/handletest/globalstats/globalstats_test.go @@ -19,9 +19,7 @@ import ( "testing" "github.com/pingcap/tidb/parser/model" - "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/statistics/handle" - "github.com/pingcap/tidb/statistics/handle/internal" "github.com/pingcap/tidb/testkit" "github.com/stretchr/testify/require" ) @@ -855,100 +853,6 @@ func TestDDLPartition4GlobalStats(t *testing.T) { require.Equal(t, int64(7), globalStats.RealtimeCount) } -func TestFeedbackWithGlobalStats(t *testing.T) { - store, dom := testkit.CreateMockStoreAndDomain(t) - testKit := testkit.NewTestKit(t, store) - testKit.MustExec("use test") - testKit.MustExec("set @@tidb_analyze_version = 1") - - oriProbability := statistics.FeedbackProbability.Load() - oriNumber := statistics.MaxNumberOfRanges - oriMinLogCount := handle.MinLogScanCount.Load() - oriErrorRate := handle.MinLogErrorRate.Load() - defer func() { - statistics.FeedbackProbability.Store(oriProbability) - statistics.MaxNumberOfRanges = oriNumber - handle.MinLogScanCount.Store(oriMinLogCount) - handle.MinLogErrorRate.Store(oriErrorRate) - }() - // Case 1: You can't set tidb_analyze_version to 2 if feedback is enabled. - // Note: if we want to set @@tidb_partition_prune_mode = 'dynamic'. We must set tidb_analyze_version to 2 first. We have already tested this. - statistics.FeedbackProbability.Store(1) - testKit.MustQuery("select @@tidb_analyze_version").Check(testkit.Rows("1")) - testKit.MustExec("set @@tidb_analyze_version = 2") - testKit.MustQuery("show warnings").Check(testkit.Rows(`Error 1105 variable tidb_analyze_version not updated because analyze version 2 is incompatible with query feedback. Please consider setting feedback-probability to 0.0 in config file to disable query feedback`)) - testKit.MustQuery("select @@tidb_analyze_version").Check(testkit.Rows("1")) - - h := dom.StatsHandle() - var err error - // checkFeedbackOnPartitionTable is used to check whether the statistics are the same as before. - checkFeedbackOnPartitionTable := func(statsBefore *statistics.Table, tblInfo *model.TableInfo) { - h.UpdateStatsByLocalFeedback(dom.InfoSchema()) - err = h.DumpStatsFeedbackToKV() - require.NoError(t, err) - err = h.HandleUpdateStats(dom.InfoSchema()) - require.NoError(t, err) - statsTblAfter := h.GetTableStats(tblInfo) - // assert that statistics not changed - // the feedback can not work for the partition table in both static and dynamic mode - internal.AssertTableEqual(t, statsBefore, statsTblAfter) - } - - // Case 2: Feedback wouldn't be applied on version 2 and global-level statistics. - statistics.FeedbackProbability.Store(0) - testKit.MustExec("set @@tidb_analyze_version = 2") - testKit.MustExec("set @@tidb_partition_prune_mode = 'dynamic';") - testKit.MustQuery("select @@tidb_analyze_version").Check(testkit.Rows("2")) - testKit.MustExec("create table t (a bigint(64), b bigint(64), index idx(b)) PARTITION BY HASH(a) PARTITIONS 2;") - for i := 0; i < 200; i++ { - testKit.MustExec("insert into t values (1,2),(2,2),(4,5),(2,3),(3,4)") - } - testKit.MustExec("analyze table t with 0 topn") - is := dom.InfoSchema() - table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - require.NoError(t, err) - tblInfo := table.Meta() - testKit.MustExec("analyze table t") - err = h.Update(dom.InfoSchema()) - require.NoError(t, err) - statsTblBefore := h.GetTableStats(tblInfo) - statistics.FeedbackProbability.Store(1) - // make the statistics inaccurate. - for i := 0; i < 200; i++ { - testKit.MustExec("insert into t values (3,4), (3,4), (3,4), (3,4), (3,4)") - } - // trigger feedback - testKit.MustExec("select b from t partition(p0) use index(idx) where t.b <= 3;") - testKit.MustExec("select b from t partition(p1) use index(idx) where t.b <= 3;") - testKit.MustExec("select b from t use index(idx) where t.b <= 3 order by b;") - testKit.MustExec("select b from t use index(idx) where t.b <= 3;") - checkFeedbackOnPartitionTable(statsTblBefore, tblInfo) - - // Case 3: Feedback is also not effective on version 1 and partition-level statistics. - testKit.MustExec("set tidb_analyze_version = 1") - testKit.MustExec("set @@tidb_partition_prune_mode = 'static';") - testKit.MustExec("create table t1 (a bigint(64), b bigint(64), index idx(b)) PARTITION BY HASH(a) PARTITIONS 2") - for i := 0; i < 200; i++ { - testKit.MustExec("insert into t1 values (1,2),(2,2),(4,5),(2,3),(3,4)") - } - testKit.MustExec("analyze table t1 with 0 topn") - // make the statistics inaccurate. - for i := 0; i < 200; i++ { - testKit.MustExec("insert into t1 values (3,4), (3,4), (3,4), (3,4), (3,4)") - } - is = dom.InfoSchema() - table, err = is.TableByName(model.NewCIStr("test"), model.NewCIStr("t1")) - require.NoError(t, err) - tblInfo = table.Meta() - statsTblBefore = h.GetTableStats(tblInfo) - // trigger feedback - testKit.MustExec("select b from t1 partition(p0) use index(idx) where t1.b <= 3;") - testKit.MustExec("select b from t1 partition(p1) use index(idx) where t1.b <= 3;") - testKit.MustExec("select b from t1 use index(idx) where t1.b <= 3 order by b;") - testKit.MustExec("select b from t1 use index(idx) where t1.b <= 3;") - checkFeedbackOnPartitionTable(statsTblBefore, tblInfo) -} - func TestGlobalStatsNDV(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) diff --git a/statistics/handle/historical_stats_handler.go b/statistics/handle/historical_stats_handler.go index 43dff0e8acb05..7829e048b80d9 100644 --- a/statistics/handle/historical_stats_handler.go +++ b/statistics/handle/historical_stats_handler.go @@ -37,8 +37,6 @@ const ( StatsMetaHistorySourceExtendedStats = "extended stats" // StatsMetaHistorySourceSchemaChange indicates stats history meta source from schema change StatsMetaHistorySourceSchemaChange = "schema change" - // StatsMetaHistorySourceFeedBack indicates stats history meta source from feedback - StatsMetaHistorySourceFeedBack = "feedback" ) func recordHistoricalStatsMeta(sctx sessionctx.Context, tableID int64, version uint64, source string) error { diff --git a/statistics/handle/update.go b/statistics/handle/update.go index dfb2353b67df2..6885e3056e439 100644 --- a/statistics/handle/update.go +++ b/statistics/handle/update.go @@ -15,7 +15,6 @@ package handle import ( - "bytes" "cmp" "context" "fmt" @@ -28,27 +27,18 @@ import ( "time" "github.com/pingcap/errors" - "github.com/pingcap/log" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/parser/model" - "github.com/pingcap/tidb/parser/mysql" - "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/statistics/handle/cache" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" - "github.com/pingcap/tidb/util/chunk" - "github.com/pingcap/tidb/util/codec" - "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/logutil" - "github.com/pingcap/tidb/util/ranger" "github.com/pingcap/tidb/util/sqlexec" "github.com/pingcap/tidb/util/timeutil" - "go.uber.org/atomic" "go.uber.org/zap" ) @@ -147,24 +137,21 @@ func (m colStatsUsageMap) merge(other colStatsUsageMap) { } } -func merge(s *SessionStatsCollector, deltaMap tableDeltaMap, rateMap errorRateDeltaMap, feedback *statistics.QueryFeedbackMap, colMap colStatsUsageMap) { +func merge(s *SessionStatsCollector, deltaMap tableDeltaMap, rateMap errorRateDeltaMap, colMap colStatsUsageMap) { deltaMap.merge(s.mapper) s.mapper = make(tableDeltaMap) rateMap.merge(s.rateMap) s.rateMap = make(errorRateDeltaMap) - feedback.Merge(s.feedback) - s.feedback = statistics.NewQueryFeedbackMap() colMap.merge(s.colMap) s.colMap = make(colStatsUsageMap) } // SessionStatsCollector is a list item that holds the delta mapper. If you want to write or read mapper, you must lock it. type SessionStatsCollector struct { - mapper tableDeltaMap - feedback *statistics.QueryFeedbackMap - rateMap errorRateDeltaMap - colMap colStatsUsageMap - next *SessionStatsCollector + mapper tableDeltaMap + rateMap errorRateDeltaMap + colMap colStatsUsageMap + next *SessionStatsCollector sync.Mutex // deleted is set to true when a session is closed. Every time we sweep the list, we will remove the useless collector. @@ -185,60 +172,17 @@ func (s *SessionStatsCollector) Update(id int64, delta int64, count int64, colSi s.mapper.update(id, delta, count, colSize) } -// ClearForTest clears the mapper and feedback for test. +// ClearForTest clears the mapper for test. func (s *SessionStatsCollector) ClearForTest() { s.Lock() defer s.Unlock() s.mapper = make(tableDeltaMap) - s.feedback = statistics.NewQueryFeedbackMap() s.rateMap = make(errorRateDeltaMap) s.colMap = make(colStatsUsageMap) s.next = nil s.deleted = false } -var ( - // MinLogScanCount is the minimum scan count for a feedback to be logged. - MinLogScanCount = atomic.NewInt64(1000) - // MinLogErrorRate is the minimum error rate for a feedback to be logged. - MinLogErrorRate = atomic.NewFloat64(0.5) -) - -// StoreQueryFeedback merges the feedback into stats collector. Deprecated. -func (s *SessionStatsCollector) StoreQueryFeedback(feedback interface{}, h *Handle, enablePseudoForOutdatedStats bool) error { - q := feedback.(*statistics.QueryFeedback) - if !q.Valid.Load() || q.Hist == nil { - return nil - } - - // if table locked, skip - if h.IsTableLocked(q.PhysicalID) { - return nil - } - - err := h.RecalculateExpectCount(q, enablePseudoForOutdatedStats) - if err != nil { - return errors.Trace(err) - } - rate := q.CalcErrorRate() - minScanCnt := MinLogScanCount.Load() - minErrRate := MinLogErrorRate.Load() - if !(rate >= minErrRate && (q.Actual() >= minScanCnt || q.Expected >= minScanCnt)) { - return nil - } - metrics.SignificantFeedbackCounter.Inc() - metrics.StatsInaccuracyRate.Observe(rate) - if log.GetLevel() == zap.DebugLevel { - h.logDetailedInfo(q) - } - s.Lock() - defer s.Unlock() - isIndex := q.Tp == statistics.IndexType - s.rateMap.update(q.PhysicalID, q.Hist.ID, rate, isIndex) - s.feedback.Append(q) - return nil -} - // UpdateColStatsUsage updates the last time when the column stats are used(needed). func (s *SessionStatsCollector) UpdateColStatsUsage(colMap colStatsUsageMap) { s.Lock() @@ -251,11 +195,10 @@ func (h *Handle) NewSessionStatsCollector() *SessionStatsCollector { h.listHead.Lock() defer h.listHead.Unlock() newCollector := &SessionStatsCollector{ - mapper: make(tableDeltaMap), - rateMap: make(errorRateDeltaMap), - next: h.listHead.next, - feedback: statistics.NewQueryFeedbackMap(), - colMap: make(colStatsUsageMap), + mapper: make(tableDeltaMap), + rateMap: make(errorRateDeltaMap), + next: h.listHead.next, + colMap: make(colStatsUsageMap), } h.listHead.next = newCollector return newCollector @@ -466,14 +409,13 @@ const ( func (h *Handle) sweepList() { deltaMap := make(tableDeltaMap) errorRateMap := make(errorRateDeltaMap) - feedback := statistics.NewQueryFeedbackMap() colMap := make(colStatsUsageMap) prev := h.listHead prev.Lock() for curr := prev.next; curr != nil; curr = curr.next { curr.Lock() - // Merge the session stats into deltaMap, errorRateMap and feedback respectively. - merge(curr, deltaMap, errorRateMap, feedback, colMap) + // Merge the session stats into deltaMap, errorRateMap respectively. + merge(curr, deltaMap, errorRateMap, colMap) if curr.deleted { prev.next = curr.next // Since the session is already closed, we can safely unlock it here. @@ -488,13 +430,6 @@ func (h *Handle) sweepList() { h.globalMap.Lock() h.globalMap.data.merge(deltaMap) h.globalMap.Unlock() - h.mu.Lock() - h.mu.rateMap.merge(errorRateMap) - h.mu.Unlock() - h.feedback.Lock() - h.feedback.data.Merge(feedback) - h.feedback.data.SiftFeedbacks() - h.feedback.Unlock() h.colMap.Lock() h.colMap.data.merge(colMap) h.colMap.Unlock() @@ -641,304 +576,6 @@ func (h *Handle) dumpTableStatColSizeToKV(id int64, delta variable.TableDelta) e return errors.Trace(err) } -// DumpStatsFeedbackToKV dumps the stats feedback to KV. Deprecated. -func (h *Handle) DumpStatsFeedbackToKV() error { - h.feedback.Lock() - feedback := h.feedback.data - h.feedback.data = statistics.NewQueryFeedbackMap() - h.feedback.Unlock() - var err error - for _, fbs := range feedback.Feedbacks { - for _, fb := range fbs { - if fb.Tp == statistics.PkType { - err = h.DumpFeedbackToKV(fb) - } else { - t, ok := h.statsCache.Load().GetFromInternal(fb.PhysicalID) - if !ok { - continue - } - idx, ok := t.Indices[fb.Hist.ID] - if !ok { - continue - } - if idx.GetStatsVer() == statistics.Version1 { - err = h.DumpFeedbackForIndex(fb, t) - } else { - err = h.DumpFeedbackToKV(fb) - } - } - if err != nil { - // For simplicity, we just drop other feedbacks in case of error. - break - } - } - } - return errors.Trace(err) -} - -// DumpFeedbackToKV dumps the given feedback to physical kv layer. -func (h *Handle) DumpFeedbackToKV(fb *statistics.QueryFeedback) error { - vals, err := statistics.EncodeFeedback(fb) - if err != nil { - logutil.BgLogger().Debug("error occurred when encoding feedback", zap.Error(err)) - return nil - } - var isIndex int64 - if fb.Tp == statistics.IndexType { - isIndex = 1 - } - ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnStats) - const sql = "insert into mysql.stats_feedback (table_id, hist_id, is_index, feedback) values (%?, %?, %?, %?)" - h.mu.Lock() - _, err = h.mu.ctx.(sqlexec.SQLExecutor).ExecuteInternal(ctx, sql, fb.PhysicalID, fb.Hist.ID, isIndex, vals) - h.mu.Unlock() - if err != nil { - metrics.DumpFeedbackCounter.WithLabelValues(metrics.LblError).Inc() - } else { - metrics.DumpFeedbackCounter.WithLabelValues(metrics.LblOK).Inc() - } - return errors.Trace(err) -} - -// UpdateStatsByLocalFeedback will update statistics by the local feedback. -// Currently, we dump the feedback with the period of 10 minutes, which means -// it takes 10 minutes for a feedback to take effect. However, we can use the -// feedback locally on this tidb-server, so it could be used more timely. -func (h *Handle) UpdateStatsByLocalFeedback(is infoschema.InfoSchema) { - h.sweepList() - h.feedback.Lock() - feedback := h.feedback.data - h.feedback.data = statistics.NewQueryFeedbackMap() - h.feedback.Unlock() -OUTER: - for _, fbs := range feedback.Feedbacks { - for _, fb := range fbs { - table, ok := h.getTableByPhysicalID(is, fb.PhysicalID) - if !ok { - continue - } - if table.Meta().Partition != nil { - // If the table is partition table, the feedback will not work. - continue - } - tblStats := h.GetPartitionStats(table.Meta(), fb.PhysicalID) - newTblStats := tblStats.Copy() - if fb.Tp == statistics.IndexType { - idx, ok := tblStats.Indices[fb.Hist.ID] - if !ok || idx.Histogram.Len() == 0 { - continue - } - newIdx := *idx - eqFB, ranFB := statistics.SplitFeedbackByQueryType(fb.Feedback) - if idx.GetStatsVer() >= statistics.Version2 { - // // For StatsVersion higher than Version1, the topn is extracted out of histogram. So we don't update the histogram if the feedback overlaps with some topn. - // ranFB = statistics.CleanRangeFeedbackByTopN(ranFB, idx.TopN) - continue OUTER - } - newIdx.CMSketch, newIdx.TopN = statistics.UpdateCMSketchAndTopN(idx.CMSketch, idx.TopN, eqFB) - newIdx.Histogram = *statistics.UpdateHistogram(&idx.Histogram, &statistics.QueryFeedback{Feedback: ranFB}, int(idx.GetStatsVer())) - newIdx.Histogram.PreCalculateScalar() - newIdx.Flag = statistics.ResetAnalyzeFlag(newIdx.Flag) - newTblStats.Indices[fb.Hist.ID] = &newIdx - } else { - col, ok := tblStats.Columns[fb.Hist.ID] - if !ok || col.Histogram.Len() == 0 { - continue - } - if col.GetStatsVer() >= statistics.Version2 { - // // For StatsVersion higher than Version1, the topn is extracted out of histogram. So we don't update the histogram if the feedback overlaps with some topn. - // ranFB = statistics.CleanRangeFeedbackByTopN(ranFB, idx.TopN) - continue OUTER - } - newCol := *col - // only use the range query to update primary key - _, ranFB := statistics.SplitFeedbackByQueryType(fb.Feedback) - newFB := &statistics.QueryFeedback{Feedback: ranFB} - newFB = newFB.DecodeIntValues() - newCol.Histogram = *statistics.UpdateHistogram(&col.Histogram, newFB, statistics.Version1) - newCol.Flag = statistics.ResetAnalyzeFlag(newCol.Flag) - newTblStats.Columns[fb.Hist.ID] = &newCol - } - for retry := updateStatsCacheRetryCnt; retry > 0; retry-- { - oldCache := h.statsCache.Load() - if h.updateStatsCache(oldCache, []*statistics.Table{newTblStats}, nil) { - break - } - } - } - } -} - -// UpdateErrorRate updates the error rate of columns from h.rateMap to cache. -func (h *Handle) UpdateErrorRate(is infoschema.InfoSchema) { - h.mu.Lock() - tbls := make([]*statistics.Table, 0, len(h.mu.rateMap)) - for id, item := range h.mu.rateMap { - table, ok := h.getTableByPhysicalID(is, id) - if !ok { - continue - } - tbl := h.GetPartitionStats(table.Meta(), id).Copy() - if item.PkErrorRate != nil && tbl.Columns[item.PkID] != nil { - col := *tbl.Columns[item.PkID] - col.ErrorRate.Merge(item.PkErrorRate) - tbl.Columns[item.PkID] = &col - } - for key, val := range item.IdxErrorRate { - if tbl.Indices[key] == nil { - continue - } - idx := *tbl.Indices[key] - idx.ErrorRate.Merge(val) - tbl.Indices[key] = &idx - } - tbls = append(tbls, tbl) - delete(h.mu.rateMap, id) - } - h.mu.Unlock() - for retry := updateStatsCacheRetryCnt; retry > 0; retry-- { - oldCache := h.statsCache.Load() - if h.updateStatsCache(oldCache, tbls, nil) { - break - } - } -} - -// HandleUpdateStats update the stats using feedback. -func (h *Handle) HandleUpdateStats(is infoschema.InfoSchema) error { - ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnStats) - tables, _, err := h.execRestrictedSQL(ctx, "SELECT distinct table_id from mysql.stats_feedback") - if err != nil { - return errors.Trace(err) - } - if len(tables) == 0 { - return nil - } - - for _, ptbl := range tables { - tableID, histID, isIndex := ptbl.GetInt64(0), int64(-1), int64(-1) - for { - // fetch at most 100000 rows each time to avoid OOM - const sql = "select table_id, hist_id, is_index, feedback from mysql.stats_feedback where table_id = %? and is_index >= %? and hist_id > %? order by is_index, hist_id limit 100000" - rows, _, err := h.execRestrictedSQL(ctx, sql, tableID, histID, isIndex) - if err != nil { - return errors.Trace(err) - } - if len(rows) == 0 { - break - } - startIdx := 0 - for i, row := range rows { - if row.GetInt64(1) != histID || row.GetInt64(2) != isIndex { - if i > 0 { - if err = h.handleSingleHistogramUpdate(is, rows[startIdx:i]); err != nil { - return errors.Trace(err) - } - } - histID, isIndex = row.GetInt64(1), row.GetInt64(2) - startIdx = i - } - } - if err = h.handleSingleHistogramUpdate(is, rows[startIdx:]); err != nil { - return errors.Trace(err) - } - } - } - return nil -} - -// handleSingleHistogramUpdate updates the Histogram and CM Sketch using these feedbacks. All the feedbacks for -// the same index or column are gathered in `rows`. -func (h *Handle) handleSingleHistogramUpdate(is infoschema.InfoSchema, rows []chunk.Row) (err error) { - physicalTableID, histID, isIndex := rows[0].GetInt64(0), rows[0].GetInt64(1), rows[0].GetInt64(2) - defer func() { - if err == nil { - err = errors.Trace(h.deleteOutdatedFeedback(physicalTableID, histID, isIndex)) - } - }() - table, ok := h.getTableByPhysicalID(is, physicalTableID) - // The table has been deleted. - if !ok { - return nil - } - var tbl *statistics.Table - // 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 - var statsVer int64 = statistics.Version1 - if isIndex == 1 { - idx, ok := tbl.Indices[histID] - statsVer = idx.GetStatsVer() - if statsVer >= 2 { - logutil.BgLogger().Warn("Feedback is discarded because statistics on this table is version 2, which is incompatible with feedback. "+ - "Please consider setting feedback-probability to 0.0 in config file to disable query feedback.", - zap.String("category", "stats"), zap.Int64("table_id", physicalTableID), zap.Int64("hist_id", histID), zap.Int64("is_index", isIndex)) - return err - } - if ok && idx.Histogram.Len() > 0 { - statsVer = idx.GetStatsVer() - idxHist := idx.Histogram - hist = &idxHist - cms = idx.CMSketch.Copy() - topN = idx.TopN.Copy() - } - } else { - col, ok := tbl.Columns[histID] - if ok && col.GetStatsVer() >= 2 { - logutil.BgLogger().Warn("Feedback is discarded because statistics on this table is version 2, which is incompatible with feedback. "+ - "Please consider setting feedback-probability to 0.0 in config file to disable query feedback.", - zap.String("category", "stats"), zap.Int64("table_id", physicalTableID), zap.Int64("hist_id", histID), zap.Int64("is_index", isIndex)) - return err - } - if ok && col.Histogram.Len() > 0 { - colHist := col.Histogram - hist = &colHist - } - } - // The column or index has been deleted. - if hist == nil { - return nil - } - q := &statistics.QueryFeedback{} - for _, row := range rows { - err1 := statistics.DecodeFeedback(row.GetBytes(3), q, cms, topN, hist.Tp) - if err1 != nil { - logutil.BgLogger().Debug("decode feedback failed", zap.Error(err1)) - } - } - err = h.dumpStatsUpdateToKV(physicalTableID, isIndex, q, hist, cms, topN, statsVer) - return errors.Trace(err) -} - -func (h *Handle) deleteOutdatedFeedback(tableID, histID, isIndex int64) error { - h.mu.Lock() - defer h.mu.Unlock() - hasData := true - ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnStats) - for hasData { - sql := "delete from mysql.stats_feedback where table_id = %? and hist_id = %? and is_index = %? limit 10000" - _, err := h.mu.ctx.(sqlexec.SQLExecutor).ExecuteInternal(ctx, sql, tableID, histID, isIndex) - if err != nil { - return errors.Trace(err) - } - hasData = h.mu.ctx.GetSessionVars().StmtCtx.AffectedRows() > 0 - } - return nil -} - -func (h *Handle) dumpStatsUpdateToKV(tableID, isIndex int64, q *statistics.QueryFeedback, hist *statistics.Histogram, cms *statistics.CMSketch, topN *statistics.TopN, statsVersion int64) error { - hist = statistics.UpdateHistogram(hist, q, int(statsVersion)) - // feedback for partition is not ready. - err := h.SaveStatsToStorage(tableID, -1, 0, int(isIndex), hist, cms, topN, int(statsVersion), 0, false, StatsMetaHistorySourceFeedBack) - metrics.UpdateStatsCounter.WithLabelValues(metrics.RetLabel(err)).Inc() - return errors.Trace(err) -} - // DumpColStatsUsageToKV sweeps the whole list, updates the column stats usage map and dumps it to KV. func (h *Handle) DumpColStatsUsageToKV() error { if !variable.EnableColumnTracking.Load() { @@ -1320,351 +957,3 @@ func (h *Handle) execAutoAnalyze(statsVer int, analyzeSnapshot bool, sql string, metrics.AutoAnalyzeCounter.WithLabelValues("succ").Inc() } } - -// formatBuckets formats bucket from lowBkt to highBkt. -func formatBuckets(hg *statistics.Histogram, lowBkt, highBkt, idxCols int) string { - if lowBkt == highBkt { - return hg.BucketToString(lowBkt, idxCols) - } - if lowBkt+1 == highBkt { - return fmt.Sprintf("%s, %s", hg.BucketToString(lowBkt, idxCols), hg.BucketToString(highBkt, idxCols)) - } - // do not care the middle buckets - return fmt.Sprintf("%s, (%d buckets, total count %d), %s", hg.BucketToString(lowBkt, idxCols), - highBkt-lowBkt-1, hg.Buckets[highBkt-1].Count-hg.Buckets[lowBkt].Count, hg.BucketToString(highBkt, idxCols)) -} - -func colRangeToStr(c *statistics.Column, ran *ranger.Range, actual int64, factor float64) string { - lowCount, lowBkt := c.LessRowCountWithBktIdx(nil, ran.LowVal[0]) - highCount, highBkt := c.LessRowCountWithBktIdx(nil, ran.HighVal[0]) - return fmt.Sprintf("range: %s, actual: %d, expected: %d, buckets: {%s}", ran.String(), actual, - int64((highCount-lowCount)*factor), formatBuckets(&c.Histogram, lowBkt, highBkt, 0)) -} - -func logForIndexRange(idx *statistics.Index, ran *ranger.Range, actual int64, factor float64) string { - sc := &stmtctx.StatementContext{TimeZone: time.UTC} - lb, err := codec.EncodeKey(sc, nil, ran.LowVal...) - if err != nil { - return "" - } - rb, err := codec.EncodeKey(sc, nil, ran.HighVal...) - if err != nil { - return "" - } - if idx.CMSketch != nil && bytes.Compare(kv.Key(lb).PrefixNext(), rb) >= 0 { - str, err := types.DatumsToString(ran.LowVal, true) - if err != nil { - return "" - } - return fmt.Sprintf("value: %s, actual: %d, expected: %d", str, actual, int64(float64(idx.QueryBytes(nil, lb))*factor)) - } - l, r := types.NewBytesDatum(lb), types.NewBytesDatum(rb) - lowCount, lowBkt := idx.LessRowCountWithBktIdx(nil, l) - highCount, highBkt := idx.LessRowCountWithBktIdx(nil, r) - return fmt.Sprintf("range: %s, actual: %d, expected: %d, histogram: {%s}", ran.String(), actual, - int64((highCount-lowCount)*factor), formatBuckets(&idx.Histogram, lowBkt, highBkt, len(idx.Info.Columns))) -} - -func logForIndex(prefix string, t *statistics.Table, idx *statistics.Index, ranges []*ranger.Range, actual []int64, factor float64) { - sc := &stmtctx.StatementContext{TimeZone: time.UTC} - if idx.CMSketch == nil || idx.GetStatsVer() != statistics.Version1 { - for i, ran := range ranges { - logutil.BgLogger().Debug(prefix, zap.String("index", idx.Info.Name.O), zap.String("rangeStr", logForIndexRange(idx, ran, actual[i], factor))) - } - return - } - for i, ran := range ranges { - rangePosition := statistics.GetOrdinalOfRangeCond(sc, ran) - // only contains range or equality query - if rangePosition == 0 || rangePosition == len(ran.LowVal) { - logutil.BgLogger().Debug(prefix, zap.String("index", idx.Info.Name.O), zap.String("rangeStr", logForIndexRange(idx, ran, actual[i], factor))) - continue - } - equalityString, err := types.DatumsToString(ran.LowVal[:rangePosition], true) - if err != nil { - continue - } - bytes, err := codec.EncodeKey(sc, nil, ran.LowVal[:rangePosition]...) - if err != nil { - continue - } - equalityCount := idx.QueryBytes(nil, bytes) - rang := ranger.Range{ - LowVal: []types.Datum{ran.LowVal[rangePosition]}, - HighVal: []types.Datum{ran.HighVal[rangePosition]}, - Collators: collate.GetBinaryCollatorSlice(1), - } - colName := idx.Info.Columns[rangePosition].Name.L - // prefer index stats over column stats - if idxHist := t.IndexStartWithColumn(colName); idxHist != nil && idxHist.Histogram.Len() > 0 { - rangeString := logForIndexRange(idxHist, &rang, -1, factor) - logutil.BgLogger().Debug(prefix, zap.String("index", idx.Info.Name.O), zap.Int64("actual", actual[i]), - zap.String("equality", equalityString), zap.Uint64("expected equality", equalityCount), - zap.String("range", rangeString)) - } else if colHist := t.ColumnByName(colName); colHist != nil && colHist.Histogram.Len() > 0 { - err = convertRangeType(&rang, colHist.Tp, time.UTC) - if err == nil { - rangeString := colRangeToStr(colHist, &rang, -1, factor) - logutil.BgLogger().Debug(prefix, zap.String("index", idx.Info.Name.O), zap.Int64("actual", actual[i]), - zap.String("equality", equalityString), zap.Uint64("expected equality", equalityCount), - zap.String("range", rangeString)) - } - } else { - count, err := statistics.GetPseudoRowCountByColumnRanges(sc, float64(t.RealtimeCount), []*ranger.Range{&rang}, 0) - if err == nil { - logutil.BgLogger().Debug(prefix, zap.String("index", idx.Info.Name.O), zap.Int64("actual", actual[i]), - zap.String("equality", equalityString), zap.Uint64("expected equality", equalityCount), - zap.Stringer("range", &rang), zap.Float64("pseudo count", math.Round(count))) - } - } - } -} - -func (h *Handle) logDetailedInfo(q *statistics.QueryFeedback) { - t, ok := h.statsCache.Load().GetFromInternal(q.PhysicalID) - if !ok { - return - } - isIndex := q.Hist.IsIndexHist() - ranges, err := q.DecodeToRanges(isIndex) - if err != nil { - logutil.BgLogger().Debug("decode to ranges failed", zap.Error(err)) - return - } - actual := make([]int64, 0, len(q.Feedback)) - for _, fb := range q.Feedback { - actual = append(actual, fb.Count) - } - logPrefix := fmt.Sprintf("[stats-feedback] %s", t.Name) - if isIndex { - idx := t.Indices[q.Hist.ID] - if idx == nil || idx.Histogram.Len() == 0 { - return - } - logForIndex(logPrefix, t, idx, ranges, actual, idx.GetIncreaseFactor(t.RealtimeCount)) - } else { - c := t.Columns[q.Hist.ID] - if c == nil || c.Histogram.Len() == 0 { - return - } - logForPK(logPrefix, c, ranges, actual, c.GetIncreaseFactor(t.RealtimeCount)) - } -} - -func logForPK(prefix string, c *statistics.Column, ranges []*ranger.Range, actual []int64, factor float64) { - for i, ran := range ranges { - if ran.LowVal[0].GetInt64()+1 >= ran.HighVal[0].GetInt64() { - continue - } - logutil.BgLogger().Debug(prefix, zap.String("column", c.Info.Name.O), zap.String("rangeStr", colRangeToStr(c, ran, actual[i], factor))) - } -} - -// RecalculateExpectCount recalculates the expect row count if the origin row count is estimated by pseudo. Deprecated. -func (h *Handle) RecalculateExpectCount(q *statistics.QueryFeedback, enablePseudoForOutdatedStats bool) error { - t, ok := h.statsCache.Load().GetFromInternal(q.PhysicalID) - if !ok { - return nil - } - tablePseudo := t.Pseudo - if enablePseudoForOutdatedStats { - tablePseudo = t.Pseudo || t.IsOutdated() - } - if !tablePseudo { - return nil - } - isIndex := q.Hist.Tp.GetType() == mysql.TypeBlob - id := q.Hist.ID - if isIndex && (t.Indices[id] == nil || !t.Indices[id].NotAccurate()) { - return nil - } - if !isIndex && (t.Columns[id] == nil || !t.Columns[id].NotAccurate()) { - return nil - } - - se, err := h.pool.Get() - if err != nil { - return err - } - sctx := se.(sessionctx.Context) - timeZone := sctx.GetSessionVars().StmtCtx.TimeZone - defer func() { - sctx.GetSessionVars().StmtCtx.TimeZone = timeZone - h.pool.Put(se) - }() - sctx.GetSessionVars().StmtCtx.TimeZone = time.UTC - - ranges, err := q.DecodeToRanges(isIndex) - if err != nil { - return errors.Trace(err) - } - expected := 0.0 - if isIndex { - idx := t.Indices[id] - expected, err = idx.GetRowCount(sctx, nil, ranges, t.RealtimeCount, t.ModifyCount) - } else { - c := t.Columns[id] - expected, err = c.GetColumnRowCount(sctx, ranges, t.RealtimeCount, t.ModifyCount, true) - } - q.Expected = int64(expected) - return err -} - -func (h *Handle) dumpRangeFeedback(sc *stmtctx.StatementContext, ran *ranger.Range, rangeCount float64, q *statistics.QueryFeedback) error { - lowIsNull := ran.LowVal[0].IsNull() - if q.Tp == statistics.IndexType { - lower, err := codec.EncodeKey(sc, nil, ran.LowVal[0]) - if err != nil { - return errors.Trace(err) - } - upper, err := codec.EncodeKey(sc, nil, ran.HighVal[0]) - if err != nil { - return errors.Trace(err) - } - ran.LowVal[0].SetBytes(lower) - ran.HighVal[0].SetBytes(upper) - } else { - if !statistics.SupportColumnType(q.Hist.Tp) { - return nil - } - if ran.LowVal[0].Kind() == types.KindMinNotNull { - ran.LowVal[0] = types.GetMinValue(q.Hist.Tp) - } - if ran.HighVal[0].Kind() == types.KindMaxValue { - ran.HighVal[0] = types.GetMaxValue(q.Hist.Tp) - } - } - ranges, ok := q.Hist.SplitRange(sc, []*ranger.Range{ran}, q.Tp == statistics.IndexType) - if !ok { - logutil.BgLogger().Debug("type of histogram and ranges mismatch") - return nil - } - counts := make([]float64, 0, len(ranges)) - sum := 0.0 - for i, r := range ranges { - // Though after `SplitRange`, we may have ranges like `[l, r]`, we still use - // `betweenRowCount` to compute the estimation since the ranges of feedback are all in `[l, r)` - // form, that is to say, we ignore the exclusiveness of ranges from `SplitRange` and just use - // its result of boundary values. - count := q.Hist.BetweenRowCount(nil, r.LowVal[0], r.HighVal[0]) - // We have to include `NullCount` of histogram for [l, r) cases where l is null because `betweenRowCount` - // does not include null values of lower bound. - if i == 0 && lowIsNull { - count += float64(q.Hist.NullCount) - } - sum += count - counts = append(counts, count) - } - if sum <= 1 { - return nil - } - // We assume that each part contributes the same error rate. - adjustFactor := rangeCount / sum - for i, r := range ranges { - q.Feedback = append(q.Feedback, statistics.Feedback{Lower: &r.LowVal[0], Upper: &r.HighVal[0], Count: int64(counts[i] * adjustFactor)}) - } - return errors.Trace(h.DumpFeedbackToKV(q)) -} - -func convertRangeType(ran *ranger.Range, ft *types.FieldType, loc *time.Location) error { - err := statistics.ConvertDatumsType(ran.LowVal, ft, loc) - if err != nil { - return err - } - return statistics.ConvertDatumsType(ran.HighVal, ft, loc) -} - -// DumpFeedbackForIndex dumps the feedback for index. Deprecated. -// For queries that contains both equality and range query, we will split them and Update accordingly. -func (h *Handle) DumpFeedbackForIndex(q *statistics.QueryFeedback, t *statistics.Table) error { - idx, ok := t.Indices[q.Hist.ID] - if !ok { - return nil - } - - se, err := h.pool.Get() - if err != nil { - return err - } - sctx := se.(sessionctx.Context) - sc := sctx.GetSessionVars().StmtCtx - timeZone := sc.TimeZone - defer func() { - sctx.GetSessionVars().StmtCtx.TimeZone = timeZone - h.pool.Put(se) - }() - sc.TimeZone = time.UTC - - if idx.CMSketch == nil || idx.GetStatsVer() < statistics.Version1 { - return h.DumpFeedbackToKV(q) - } - ranges, err := q.DecodeToRanges(true) - if err != nil { - logutil.BgLogger().Debug("decode feedback ranges fail", zap.Error(err)) - return nil - } - for i, ran := range ranges { - rangePosition := statistics.GetOrdinalOfRangeCond(sc, ran) - // only contains range or equality query - if rangePosition == 0 || rangePosition == len(ran.LowVal) { - continue - } - bytes, err := codec.EncodeKey(sc, nil, ran.LowVal[:rangePosition]...) - if err != nil { - logutil.BgLogger().Debug("encode keys fail", zap.Error(err)) - continue - } - equalityCount := float64(idx.QueryBytes(nil, bytes)) * idx.GetIncreaseFactor(t.RealtimeCount) - rang := &ranger.Range{ - LowVal: []types.Datum{ran.LowVal[rangePosition]}, - HighVal: []types.Datum{ran.HighVal[rangePosition]}, - Collators: collate.GetBinaryCollatorSlice(1), - } - colName := idx.Info.Columns[rangePosition].Name.L - var rangeCount float64 - rangeFB := &statistics.QueryFeedback{PhysicalID: q.PhysicalID} - // prefer index stats over column stats - if idx := t.IndexStartWithColumn(colName); idx != nil && idx.Histogram.Len() != 0 { - rangeCount, err = t.GetRowCountByIndexRanges(sctx, idx.ID, []*ranger.Range{rang}) - rangeFB.Tp, rangeFB.Hist = statistics.IndexType, &idx.Histogram - } else if col := t.ColumnByName(colName); col != nil && col.Histogram.Len() != 0 { - err = convertRangeType(rang, col.Tp, time.UTC) - if err == nil { - rangeCount, err = t.GetRowCountByColumnRanges(sctx, col.ID, []*ranger.Range{rang}) - rangeFB.Tp, rangeFB.Hist = statistics.ColType, &col.Histogram - } - } else { - continue - } - if err != nil { - logutil.BgLogger().Debug("get row count by ranges fail", zap.Error(err)) - continue - } - - equalityCount, rangeCount = getNewCountForIndex(equalityCount, rangeCount, float64(t.RealtimeCount), float64(q.Feedback[i].Count)) - value := types.NewBytesDatum(bytes) - q.Feedback[i] = statistics.Feedback{Lower: &value, Upper: &value, Count: int64(equalityCount)} - err = h.dumpRangeFeedback(sc, rang, rangeCount, rangeFB) - if err != nil { - logutil.BgLogger().Debug("dump range feedback fail", zap.Error(err)) - continue - } - } - return errors.Trace(h.DumpFeedbackToKV(q)) -} - -// minAdjustFactor is the minimum adjust factor of each index feedback. -// We use it to avoid adjusting too much when the assumption of independence failed. -const minAdjustFactor = 0.7 - -// getNewCountForIndex adjust the estimated `eqCount` and `rangeCount` according to the real count. -// We assumes that `eqCount` and `rangeCount` contribute the same error rate. -func getNewCountForIndex(eqCount, rangeCount, totalCount, realCount float64) (equalityCount, rangeCnt float64) { - estimate := (eqCount / totalCount) * (rangeCount / totalCount) * totalCount - if estimate <= 1 { - return eqCount, rangeCount - } - adjustFactor := math.Sqrt(realCount / estimate) - adjustFactor = math.Max(adjustFactor, minAdjustFactor) - return eqCount * adjustFactor, rangeCount * adjustFactor -} diff --git a/statistics/handle/update_list_test.go b/statistics/handle/update_list_test.go index cbedae29c9bc3..1ee9ce256d672 100644 --- a/statistics/handle/update_list_test.go +++ b/statistics/handle/update_list_test.go @@ -17,7 +17,6 @@ package handle import ( "testing" - "github.com/pingcap/tidb/statistics" "github.com/stretchr/testify/require" ) @@ -25,7 +24,6 @@ func TestInsertAndDelete(t *testing.T) { h := Handle{ listHead: &SessionStatsCollector{mapper: make(tableDeltaMap)}, } - h.feedback.data = statistics.NewQueryFeedbackMap() var items []*SessionStatsCollector for i := 0; i < 5; i++ { items = append(items, h.NewSessionStatsCollector()) diff --git a/statistics/handle/updatetest/BUILD.bazel b/statistics/handle/updatetest/BUILD.bazel index dd36350965019..6419a5e719343 100644 --- a/statistics/handle/updatetest/BUILD.bazel +++ b/statistics/handle/updatetest/BUILD.bazel @@ -8,21 +8,17 @@ go_test( "update_test.go", ], flaky = True, - shard_count = 44, + shard_count = 27, deps = [ - "//metrics", "//parser/model", "//parser/mysql", - "//sessionctx/stmtctx", "//sessionctx/variable", "//statistics", "//statistics/handle", "//statistics/handle/cache", - "//statistics/handle/internal", "//testkit", "//testkit/testsetup", "//types", - "//util/codec", "//util/collate", "//util/ranger", "@com_github_prometheus_client_model//go", diff --git a/statistics/handle/updatetest/update_test.go b/statistics/handle/updatetest/update_test.go index 71849faa605b4..ff2e85259256b 100644 --- a/statistics/handle/updatetest/update_test.go +++ b/statistics/handle/updatetest/update_test.go @@ -22,18 +22,14 @@ import ( "testing" "time" - "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" - "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/statistics/handle" "github.com/pingcap/tidb/statistics/handle/cache" - "github.com/pingcap/tidb/statistics/handle/internal" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/ranger" dto "github.com/prometheus/client_model/go" @@ -704,148 +700,6 @@ func TestTableAnalyzed(t *testing.T) { require.True(t, handle.TableAnalyzed(statsTbl)) } -func TestUpdateErrorRate(t *testing.T) { - store, dom := testkit.CreateMockStoreAndDomain(t) - h := dom.StatsHandle() - is := dom.InfoSchema() - h.SetLease(0) - require.NoError(t, h.Update(is)) - oriProbability := statistics.FeedbackProbability.Load() - oriMinLogCount := handle.MinLogScanCount.Load() - oriErrorRate := handle.MinLogErrorRate.Load() - defer func() { - statistics.FeedbackProbability.Store(oriProbability) - handle.MinLogScanCount.Store(oriMinLogCount) - handle.MinLogErrorRate.Store(oriErrorRate) - }() - statistics.FeedbackProbability.Store(1) - handle.MinLogScanCount.Store(0) - handle.MinLogErrorRate.Store(0) - - testKit := testkit.NewTestKit(t, store) - testKit.MustExec("use test") - - // TODO(tiancaiamao): query feedback is broken when paging is on. - testKit.MustExec("set @@tidb_enable_paging = off") - - testKit.MustExec("set @@session.tidb_enable_pseudo_for_outdated_stats = 1") - testKit.MustExec("set @@session.tidb_analyze_version = 0") - testKit.MustExec("create table t (a bigint(64), b bigint(64), primary key(a), index idx(b))") - err := h.HandleDDLEvent(<-h.DDLEventCh()) - require.NoError(t, err) - - testKit.MustExec("insert into t values (1, 3)") - - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - testKit.MustExec("analyze table t") - - testKit.MustExec("insert into t values (2, 3)") - testKit.MustExec("insert into t values (5, 3)") - testKit.MustExec("insert into t values (8, 3)") - testKit.MustExec("insert into t values (12, 3)") - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - is = dom.InfoSchema() - require.NoError(t, h.Update(is)) - - table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - require.NoError(t, err) - tblInfo := table.Meta() - tbl := h.GetTableStats(tblInfo) - aID := tblInfo.Columns[0].ID - bID := tblInfo.Indices[0].ID - - // The statistic table is outdated now. - require.True(t, tbl.Columns[aID].NotAccurate()) - - testKit.MustQuery("select * from t where a between 1 and 10") - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - require.NoError(t, h.DumpStatsFeedbackToKV()) - require.NoError(t, h.HandleUpdateStats(is)) - h.UpdateErrorRate(is) - require.NoError(t, h.Update(is)) - tbl = h.GetTableStats(tblInfo) - - // The error rate of this column is not larger than MaxErrorRate now. - require.False(t, tbl.Columns[aID].NotAccurate()) - - require.True(t, tbl.Indices[bID].NotAccurate()) - testKit.MustQuery("select * from t where b between 2 and 10") - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - require.NoError(t, h.DumpStatsFeedbackToKV()) - require.NoError(t, h.HandleUpdateStats(is)) - h.UpdateErrorRate(is) - require.NoError(t, h.Update(is)) - tbl = h.GetTableStats(tblInfo) - require.False(t, tbl.Indices[bID].NotAccurate()) - require.Equal(t, int64(1), tbl.Indices[bID].QueryTotal) - - testKit.MustExec("analyze table t") - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - require.NoError(t, h.Update(is)) - tbl = h.GetTableStats(tblInfo) - require.Equal(t, int64(0), tbl.Indices[bID].QueryTotal) -} - -func TestUpdatePartitionErrorRate(t *testing.T) { - store, dom := testkit.CreateMockStoreAndDomain(t) - h := dom.StatsHandle() - is := dom.InfoSchema() - h.SetLease(0) - require.NoError(t, h.Update(is)) - oriProbability := statistics.FeedbackProbability.Load() - oriMinLogCount := handle.MinLogScanCount.Load() - oriErrorRate := handle.MinLogErrorRate.Load() - defer func() { - statistics.FeedbackProbability.Store(oriProbability) - handle.MinLogScanCount.Store(oriMinLogCount) - handle.MinLogErrorRate.Store(oriErrorRate) - }() - statistics.FeedbackProbability.Store(1) - handle.MinLogScanCount.Store(0) - handle.MinLogErrorRate.Store(0) - - testKit := testkit.NewTestKit(t, store) - testKit.MustExec("use test") - testKit.MustExec(`set @@tidb_partition_prune_mode='` + string(variable.Static) + `'`) - testKit.MustExec("create table t (a bigint(64), primary key(a)) partition by range (a) (partition p0 values less than (30))") - err := h.HandleDDLEvent(<-h.DDLEventCh()) - require.NoError(t, err) - - testKit.MustExec("insert into t values (1)") - - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - testKit.MustExec("analyze table t") - - testKit.MustExec("insert into t values (2)") - testKit.MustExec("insert into t values (5)") - testKit.MustExec("insert into t values (8)") - testKit.MustExec("insert into t values (12)") - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - is = dom.InfoSchema() - require.NoError(t, h.Update(is)) - - table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - require.NoError(t, err) - tblInfo := table.Meta() - pid := tblInfo.Partition.Definitions[0].ID - tbl := h.GetPartitionStats(tblInfo, pid) - aID := tblInfo.Columns[0].ID - - // The statistic table is outdated now. - require.True(t, tbl.Columns[aID].NotAccurate()) - - testKit.MustQuery("select * from t where a between 1 and 10") - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - require.NoError(t, h.DumpStatsFeedbackToKV()) - require.NoError(t, h.HandleUpdateStats(is)) - h.UpdateErrorRate(is) - require.NoError(t, h.Update(is)) - tbl = h.GetPartitionStats(tblInfo, pid) - - // Feedback will not take effect under partition table. - require.True(t, tbl.Columns[aID].NotAccurate()) -} - func appendBucket(h *statistics.Histogram, l, r int64) { lower, upper := types.NewIntDatum(l), types.NewIntDatum(r) h.AppendBucket(&lower, &upper, 0, 0) @@ -906,232 +760,6 @@ func TestSplitRange(t *testing.T) { } } -func TestQueryFeedback(t *testing.T) { - store, dom := testkit.CreateMockStoreAndDomain(t) - testKit := testkit.NewTestKit(t, store) - testKit.MustExec("use test") - - // TODO(tiancaiamao): query feedback is broken when paging is on. - testKit.MustExec("set @@tidb_enable_paging = off") - - testKit.MustExec("set @@session.tidb_analyze_version = 0") - testKit.MustExec("create table t (a bigint(64), b bigint(64), primary key(a), index idx(b))") - testKit.MustExec("insert into t values (1,2),(2,2),(4,5)") - testKit.MustExec("analyze table t with 0 topn") - testKit.MustExec("insert into t values (3,4)") - - h := dom.StatsHandle() - oriProbability := statistics.FeedbackProbability.Load() - oriNumber := statistics.MaxNumberOfRanges - oriMinLogCount := handle.MinLogScanCount.Load() - oriErrorRate := handle.MinLogErrorRate.Load() - defer func() { - statistics.FeedbackProbability.Store(oriProbability) - statistics.MaxNumberOfRanges = oriNumber - handle.MinLogScanCount.Store(oriMinLogCount) - handle.MinLogErrorRate.Store(oriErrorRate) - }() - statistics.FeedbackProbability.Store(1) - handle.MinLogScanCount.Store(0) - handle.MinLogErrorRate.Store(0) - tests := []struct { - sql string - hist string - idxCols int - }{ - { - // test primary key feedback - sql: "select * from t where t.a <= 5 order by a desc", - hist: "column:1 ndv:4 totColSize:0\n" + - "num: 1 lower_bound: -9223372036854775808 upper_bound: 2 repeats: 0 ndv: 0\n" + - "num: 2 lower_bound: 2 upper_bound: 4 repeats: 0 ndv: 0\n" + - "num: 1 lower_bound: 4 upper_bound: 4 repeats: 1 ndv: 0", - idxCols: 0, - }, - { - // test index feedback by double read - sql: "select * from t use index(idx) where t.b <= 5", - hist: "index:1 ndv:2\n" + - "num: 3 lower_bound: -inf upper_bound: 5 repeats: 0 ndv: 0\n" + - "num: 1 lower_bound: 5 upper_bound: 5 repeats: 1 ndv: 0", - idxCols: 1, - }, - { - // test index feedback by single read - sql: "select b from t use index(idx) where t.b <= 5", - hist: "index:1 ndv:2\n" + - "num: 3 lower_bound: -inf upper_bound: 5 repeats: 0 ndv: 0\n" + - "num: 1 lower_bound: 5 upper_bound: 5 repeats: 1 ndv: 0", - idxCols: 1, - }, - } - is := dom.InfoSchema() - table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - for i, test := range tests { - testKit.MustQuery(test.sql) - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - require.NoError(t, h.DumpStatsFeedbackToKV()) - require.NoError(t, h.HandleUpdateStats(dom.InfoSchema())) - require.NoError(t, err) - require.NoError(t, h.Update(is)) - tblInfo := table.Meta() - tbl := h.GetTableStats(tblInfo) - if test.idxCols == 0 { - require.Equal(t, tests[i].hist, tbl.Columns[tblInfo.Columns[0].ID].ToString(0)) - } else { - require.Equal(t, tests[i].hist, tbl.Indices[tblInfo.Indices[0].ID].ToString(1)) - } - } - - // Feedback from limit executor may not be accurate. - testKit.MustQuery("select * from t where t.a <= 5 limit 1") - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - feedback := h.GetQueryFeedback() - require.Equal(t, 0, feedback.Size) - - // Test only collect for max number of Ranges. - statistics.MaxNumberOfRanges = 0 - for _, test := range tests { - testKit.MustQuery(test.sql) - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - feedback := h.GetQueryFeedback() - require.Equal(t, 0, feedback.Size) - } - - // Test collect feedback by probability. - statistics.FeedbackProbability.Store(0) - statistics.MaxNumberOfRanges = oriNumber - for _, test := range tests { - testKit.MustQuery(test.sql) - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - feedback := h.GetQueryFeedback() - require.Equal(t, 0, feedback.Size) - } - - // Test that after drop stats, the feedback won't cause panic. - statistics.FeedbackProbability.Store(1) - for _, test := range tests { - testKit.MustQuery(test.sql) - } - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - require.NoError(t, h.DumpStatsFeedbackToKV()) - testKit.MustExec("drop stats t") - require.NoError(t, h.HandleUpdateStats(dom.InfoSchema())) - - // Test that the outdated feedback won't cause panic. - testKit.MustExec("analyze table t") - for _, test := range tests { - testKit.MustQuery(test.sql) - } - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - require.NoError(t, h.DumpStatsFeedbackToKV()) - testKit.MustExec("drop table t") - require.NoError(t, h.HandleUpdateStats(dom.InfoSchema())) -} - -func TestQueryFeedbackForPartition(t *testing.T) { - store, dom := testkit.CreateMockStoreAndDomain(t) - testKit := testkit.NewTestKit(t, store) - testKit.MustExec("use test") - testKit.MustExec("set @@session.tidb_analyze_version = 0") - testKit.MustExec(`set @@tidb_partition_prune_mode='` + string(variable.Static) + `'`) - testKit.MustExec(`create table t (a bigint(64), b bigint(64), primary key(a), index idx(b)) - partition by range (a) ( - partition p0 values less than (3), - partition p1 values less than (6))`) - testKit.MustExec("insert into t values (1,2),(2,2),(3,4),(4,1),(5,6)") - testKit.MustExec("analyze table t") - - oriProbability := statistics.FeedbackProbability.Load() - oriMinLogCount := handle.MinLogScanCount.Load() - oriErrorRate := handle.MinLogErrorRate.Load() - defer func() { - statistics.FeedbackProbability.Store(oriProbability) - handle.MinLogScanCount.Store(oriMinLogCount) - handle.MinLogErrorRate.Store(oriErrorRate) - }() - statistics.FeedbackProbability.Store(1) - handle.MinLogScanCount.Store(0) - handle.MinLogErrorRate.Store(0) - - h := dom.StatsHandle() - // Feedback will not take effect under partition table. - tests := []struct { - sql string - hist string - idxCols int - }{ - { - // test primary key feedback - sql: "select * from t where t.a <= 5", - hist: "column:1 ndv:2 totColSize:2\n" + - "num: 1 lower_bound: 1 upper_bound: 1 repeats: 1 ndv: 0\n" + - "num: 1 lower_bound: 2 upper_bound: 2 repeats: 1 ndv: 0", - idxCols: 0, - }, - { - // test index feedback by double read - sql: "select * from t use index(idx) where t.b <= 5", - hist: "index:1 ndv:1\n" + - "num: 2 lower_bound: 2 upper_bound: 2 repeats: 2 ndv: 0", - idxCols: 1, - }, - { - // test index feedback by single read - sql: "select b from t use index(idx) where t.b <= 5", - hist: "index:1 ndv:1\n" + - "num: 2 lower_bound: 2 upper_bound: 2 repeats: 2 ndv: 0", - idxCols: 1, - }, - } - is := dom.InfoSchema() - table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - require.NoError(t, err) - tblInfo := table.Meta() - pi := tblInfo.GetPartitionInfo() - require.NotNil(t, pi) - - // This test will check the result of partition p0. - var pid int64 - for _, def := range pi.Definitions { - if def.Name.L == "p0" { - pid = def.ID - break - } - } - - for i, test := range tests { - testKit.MustQuery(test.sql) - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - require.NoError(t, h.DumpStatsFeedbackToKV()) - require.NoError(t, h.HandleUpdateStats(dom.InfoSchema())) - require.NoError(t, err) - require.NoError(t, h.Update(is)) - tbl := h.GetPartitionStats(tblInfo, pid) - if test.idxCols == 0 { - require.Equal(t, tests[i].hist, tbl.Columns[tblInfo.Columns[0].ID].ToString(0)) - } else { - require.Equal(t, tests[i].hist, tbl.Indices[tblInfo.Indices[0].ID].ToString(1)) - } - } - testKit.MustExec("drop table t") -} - -func TestUpdateSystemTable(t *testing.T) { - store, dom := testkit.CreateMockStoreAndDomain(t) - testKit := testkit.NewTestKit(t, store) - testKit.MustExec("use test") - testKit.MustExec("create table t (a int, b int)") - testKit.MustExec("insert into t values (1,2)") - testKit.MustExec("analyze table t") - testKit.MustExec("analyze table mysql.stats_histograms") - h := dom.StatsHandle() - require.NoError(t, h.Update(dom.InfoSchema())) - feedback := h.GetQueryFeedback() - // We may have query feedback for system tables, but we do not need to store them. - require.Equal(t, 0, feedback.Size) -} - func TestOutOfOrderUpdate(t *testing.T) { store, dom := testkit.CreateMockStoreAndDomain(t) testKit := testkit.NewTestKit(t, store) @@ -1166,214 +794,6 @@ func TestOutOfOrderUpdate(t *testing.T) { testKit.MustQuery(fmt.Sprintf("select count from mysql.stats_meta where table_id = %d", tableInfo.ID)).Check(testkit.Rows("3")) } -func TestUpdateStatsByLocalFeedback(t *testing.T) { - store, dom := testkit.CreateMockStoreAndDomain(t) - testKit := testkit.NewTestKit(t, store) - testKit.MustExec("use test") - - // TODO(tiancaiamao): query feedback is broken when paging is on. - testKit.MustExec("set @@tidb_enable_paging = off") - - testKit.MustExec("set @@session.tidb_analyze_version = 0") - testKit.MustExec(`set @@tidb_partition_prune_mode='` + string(variable.Static) + `'`) - testKit.MustExec("create table t (a bigint(64), b bigint(64), primary key(a), index idx(b))") - testKit.MustExec("insert into t values (1,2),(2,2),(4,5)") - testKit.MustExec("analyze table t with 0 topn") - testKit.MustExec("insert into t values (3,5)") - h := dom.StatsHandle() - oriProbability := statistics.FeedbackProbability.Load() - oriMinLogCount := handle.MinLogScanCount.Load() - oriErrorRate := handle.MinLogErrorRate.Load() - oriNumber := statistics.MaxNumberOfRanges - defer func() { - statistics.FeedbackProbability.Store(oriProbability) - handle.MinLogScanCount.Store(oriMinLogCount) - handle.MinLogErrorRate.Store(oriErrorRate) - statistics.MaxNumberOfRanges = oriNumber - }() - statistics.FeedbackProbability.Store(1) - handle.MinLogScanCount.Store(0) - handle.MinLogErrorRate.Store(0) - - is := dom.InfoSchema() - table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - require.NoError(t, err) - - tblInfo := table.Meta() - h.GetTableStats(tblInfo) - - testKit.MustQuery("select * from t use index(idx) where b <= 5") - testKit.MustQuery("select * from t where a > 1") - testKit.MustQuery("select * from t use index(idx) where b = 5") - - h.UpdateStatsByLocalFeedback(dom.InfoSchema()) - tbl := h.GetTableStats(tblInfo) - - require.Equal(t, "column:1 ndv:3 totColSize:0\n"+ - "num: 1 lower_bound: 1 upper_bound: 1 repeats: 1 ndv: 0\n"+ - "num: 2 lower_bound: 2 upper_bound: 4 repeats: 0 ndv: 0\n"+ - "num: 1 lower_bound: 4 upper_bound: 9223372036854775807 repeats: 0 ndv: 0", tbl.Columns[tblInfo.Columns[0].ID].ToString(0)) - sc := &stmtctx.StatementContext{TimeZone: time.Local} - low, err := codec.EncodeKey(sc, nil, types.NewIntDatum(5)) - require.NoError(t, err) - - require.Equal(t, uint64(2), tbl.Indices[tblInfo.Indices[0].ID].CMSketch.QueryBytes(low)) - - require.Equal(t, "index:1 ndv:2\n"+ - "num: 2 lower_bound: -inf upper_bound: 5 repeats: 0 ndv: 0\n"+ - "num: 1 lower_bound: 5 upper_bound: 5 repeats: 1 ndv: 0", tbl.Indices[tblInfo.Indices[0].ID].ToString(1)) - - // Test that it won't cause panic after update. - testKit.MustQuery("select * from t use index(idx) where b > 0") - - // Test that after drop stats, it won't cause panic. - testKit.MustExec("drop stats t") - h.UpdateStatsByLocalFeedback(dom.InfoSchema()) -} - -func TestUpdatePartitionStatsByLocalFeedback(t *testing.T) { - store, dom := testkit.CreateMockStoreAndDomain(t) - testKit := testkit.NewTestKit(t, store) - testKit.MustExec("use test") - testKit.MustExec("set @@session.tidb_analyze_version = 0") - testKit.MustExec(`set @@tidb_partition_prune_mode='` + string(variable.Static) + `'`) - testKit.MustExec("create table t (a bigint(64), b bigint(64), primary key(a)) partition by range (a) (partition p0 values less than (6))") - testKit.MustExec("insert into t values (1,2),(2,2),(4,5)") - testKit.MustExec("analyze table t") - testKit.MustExec("insert into t values (3,5)") - h := dom.StatsHandle() - oriProbability := statistics.FeedbackProbability.Load() - oriMinLogCount := handle.MinLogScanCount.Load() - oriErrorRate := handle.MinLogErrorRate.Load() - defer func() { - statistics.FeedbackProbability.Store(oriProbability) - handle.MinLogScanCount.Store(oriMinLogCount) - handle.MinLogErrorRate.Store(oriErrorRate) - }() - statistics.FeedbackProbability.Store(1) - handle.MinLogScanCount.Store(0) - handle.MinLogErrorRate.Store(0) - - is := dom.InfoSchema() - table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - require.NoError(t, err) - - testKit.MustQuery("select * from t where a > 1").Check(testkit.Rows("2 2", "3 5", "4 5")) - - h.UpdateStatsByLocalFeedback(dom.InfoSchema()) - - tblInfo := table.Meta() - pid := tblInfo.Partition.Definitions[0].ID - tbl := h.GetPartitionStats(tblInfo, pid) - - // Feedback will not take effect under partition table. - require.Equal(t, "column:1 ndv:3 totColSize:0\n"+ - "num: 1 lower_bound: 1 upper_bound: 1 repeats: 1 ndv: 0\n"+ - "num: 1 lower_bound: 2 upper_bound: 2 repeats: 1 ndv: 0\n"+ - "num: 1 lower_bound: 4 upper_bound: 4 repeats: 1 ndv: 0", tbl.Columns[tblInfo.Columns[0].ID].ToString(0)) -} - -func TestFeedbackWithStatsVer2(t *testing.T) { - store, dom := testkit.CreateMockStoreAndDomain(t) - testKit := testkit.NewTestKit(t, store) - testKit.MustExec("use test") - testKit.MustExec("set global tidb_analyze_version = 1") - testKit.MustExec("set @@tidb_analyze_version = 1") - - oriProbability := statistics.FeedbackProbability.Load() - oriNumber := statistics.MaxNumberOfRanges - oriMinLogCount := handle.MinLogScanCount.Load() - oriErrorRate := handle.MinLogErrorRate.Load() - defer func() { - statistics.FeedbackProbability.Store(oriProbability) - statistics.MaxNumberOfRanges = oriNumber - handle.MinLogScanCount.Store(oriMinLogCount) - handle.MinLogErrorRate.Store(oriErrorRate) - }() - // Case 1: You can't set tidb_analyze_version to 2 if feedback is enabled. - statistics.FeedbackProbability.Store(1) - testKit.MustQuery("select @@tidb_analyze_version").Check(testkit.Rows("1")) - testKit.MustExec("set @@tidb_analyze_version = 2") - testKit.MustQuery("show warnings").Check(testkit.Rows(`Error 1105 variable tidb_analyze_version not updated because analyze version 2 is incompatible with query feedback. Please consider setting feedback-probability to 0.0 in config file to disable query feedback`)) - testKit.MustQuery("select @@tidb_analyze_version").Check(testkit.Rows("1")) - - // Case 2: Feedback wouldn't be applied on version 2 statistics. - statistics.FeedbackProbability.Store(0) - testKit.MustExec("set @@tidb_analyze_version = 2") - testKit.MustQuery("select @@tidb_analyze_version").Check(testkit.Rows("2")) - testKit.MustExec("create table t (a bigint(64), b bigint(64), index idx(b))") - for i := 0; i < 200; i++ { - testKit.MustExec("insert into t values (1,2),(2,2),(4,5),(2,3),(3,4)") - } - testKit.MustExec("analyze table t with 0 topn") - h := dom.StatsHandle() - is := dom.InfoSchema() - table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - require.NoError(t, err) - tblInfo := table.Meta() - testKit.MustExec("analyze table t") - err = h.Update(dom.InfoSchema()) - require.NoError(t, err) - statsTblBefore := h.GetTableStats(tblInfo) - statistics.FeedbackProbability.Store(1) - // make the statistics inaccurate. - for i := 0; i < 200; i++ { - testKit.MustExec("insert into t values (3,4), (3,4), (3,4), (3,4), (3,4)") - } - // trigger feedback - testKit.MustExec("select * from t where t.a <= 5 order by a desc") - testKit.MustExec("select b from t use index(idx) where t.b <= 5") - - h.UpdateStatsByLocalFeedback(dom.InfoSchema()) - err = h.DumpStatsFeedbackToKV() - require.NoError(t, err) - err = h.HandleUpdateStats(dom.InfoSchema()) - require.NoError(t, err) - statsTblAfter := h.GetTableStats(tblInfo) - // assert that statistics not changed - internal.AssertTableEqual(t, statsTblBefore, statsTblAfter) - - // Case 3: Feedback is still effective on version 1 statistics. - testKit.MustExec("set tidb_analyze_version = 1") - testKit.MustExec("create table t1 (a bigint(64), b bigint(64), index idx(b))") - for i := 0; i < 200; i++ { - testKit.MustExec("insert into t1 values (1,2),(2,2),(4,5),(2,3),(3,4)") - } - testKit.MustExec("analyze table t1 with 0 topn") - // make the statistics inaccurate. - for i := 0; i < 200; i++ { - testKit.MustExec("insert into t1 values (3,4), (3,4), (3,4), (3,4), (3,4)") - } - is = dom.InfoSchema() - table, err = is.TableByName(model.NewCIStr("test"), model.NewCIStr("t1")) - require.NoError(t, err) - tblInfo = table.Meta() - statsTblBefore = h.GetTableStats(tblInfo) - // trigger feedback - testKit.MustExec("select b from t1 use index(idx) where t1.b <= 5") - - h.UpdateStatsByLocalFeedback(dom.InfoSchema()) - err = h.DumpStatsFeedbackToKV() - require.NoError(t, err) - err = h.HandleUpdateStats(dom.InfoSchema()) - require.NoError(t, err) - statsTblAfter = h.GetTableStats(tblInfo) - // assert that statistics changed(feedback worked) - require.False(t, statistics.HistogramEqual(&statsTblBefore.Indices[1].Histogram, &statsTblAfter.Indices[1].Histogram, false)) - - // Case 4: When existing version 1 stats + tidb_analyze_version=2 + feedback enabled, explicitly running `analyze table` still results in version 1 stats. - statistics.FeedbackProbability.Store(0) - testKit.MustExec("set tidb_analyze_version = 2") - statistics.FeedbackProbability.Store(1) - testKit.MustExec("analyze table t1 with 0 topn") - testKit.MustQuery("show warnings").Check(testkit.Rows( - "Warning 1105 Use analyze version 1 on table `t1` because this table already has version 1 statistics and query feedback is also enabled." + - " 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.")) - testKit.MustQuery(fmt.Sprintf("select stats_ver from mysql.stats_histograms where table_id = %d", tblInfo.ID)).Check(testkit.Rows("1", "1", "1")) - - testKit.MustExec("set global tidb_analyze_version = 1") -} - func TestNeedAnalyzeTable(t *testing.T) { columns := map[int64]*statistics.Column{} columns[1] = &statistics.Column{StatsVer: statistics.Version2} @@ -1456,452 +876,6 @@ func TestNeedAnalyzeTable(t *testing.T) { } } -func TestIndexQueryFeedback(t *testing.T) { - t.Skip("support update the topn of index equal conditions") - store, dom := testkit.CreateMockStoreAndDomain(t) - testKit := testkit.NewTestKit(t, store) - - oriProbability := statistics.FeedbackProbability.Load() - defer func() { - statistics.FeedbackProbability.Store(oriProbability) - }() - statistics.FeedbackProbability.Store(1) - - testKit.MustExec("use test") - testKit.MustExec("create table t (a bigint(64), b bigint(64), c bigint(64), d float, e double, f decimal(17,2), " + - "g time, h date, index idx_b(b), index idx_ab(a,b), index idx_ac(a,c), index idx_ad(a, d), index idx_ae(a, e), index idx_af(a, f)," + - " index idx_ag(a, g), index idx_ah(a, h))") - for i := 0; i < 20; i++ { - testKit.MustExec(fmt.Sprintf(`insert into t values (1, %d, %d, %d, %d, %d, %d, "%s")`, i, i, i, i, i, i, fmt.Sprintf("1000-01-%02d", i+1))) - } - h := dom.StatsHandle() - err := h.HandleDDLEvent(<-h.DDLEventCh()) - require.NoError(t, err) - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - testKit.MustExec("analyze table t with 3 buckets") - for i := 0; i < 20; i++ { - testKit.MustExec(fmt.Sprintf(`insert into t values (1, %d, %d, %d, %d, %d, %d, "%s")`, i, i, i, i, i, i, fmt.Sprintf("1000-01-%02d", i+1))) - } - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - is := dom.InfoSchema() - require.NoError(t, h.Update(is)) - table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - require.NoError(t, err) - tblInfo := table.Meta() - tests := []struct { - sql string - hist string - idxCols int - rangeID int64 - idxID int64 - eqCount uint32 - }{ - { - sql: "select * from t use index(idx_ab) where a = 1 and b < 21", - hist: "index:1 ndv:20\n" + - "num: 16 lower_bound: -inf upper_bound: 7 repeats: 0\n" + - "num: 16 lower_bound: 8 upper_bound: 15 repeats: 0\n" + - "num: 9 lower_bound: 16 upper_bound: 21 repeats: 0", - rangeID: tblInfo.Indices[0].ID, - idxID: tblInfo.Indices[1].ID, - idxCols: 1, - eqCount: 32, - }, - { - sql: "select * from t use index(idx_ac) where a = 1 and c < 21", - hist: "column:3 ndv:20 totColSize:40\n" + - "num: 13 lower_bound: -9223372036854775808 upper_bound: 6 repeats: 0\n" + - "num: 13 lower_bound: 7 upper_bound: 13 repeats: 0\n" + - "num: 12 lower_bound: 14 upper_bound: 21 repeats: 0", - rangeID: tblInfo.Columns[2].ID, - idxID: tblInfo.Indices[2].ID, - idxCols: 0, - eqCount: 32, - }, - { - sql: "select * from t use index(idx_ad) where a = 1 and d < 21", - hist: "column:4 ndv:20 totColSize:320\n" + - "num: 13 lower_bound: -10000000000000 upper_bound: 6 repeats: 0\n" + - "num: 12 lower_bound: 7 upper_bound: 13 repeats: 0\n" + - "num: 10 lower_bound: 14 upper_bound: 21 repeats: 0", - rangeID: tblInfo.Columns[3].ID, - idxID: tblInfo.Indices[3].ID, - idxCols: 0, - eqCount: 32, - }, - { - sql: "select * from t use index(idx_ae) where a = 1 and e < 21", - hist: "column:5 ndv:20 totColSize:320\n" + - "num: 13 lower_bound: -100000000000000000000000 upper_bound: 6 repeats: 0\n" + - "num: 12 lower_bound: 7 upper_bound: 13 repeats: 0\n" + - "num: 10 lower_bound: 14 upper_bound: 21 repeats: 0", - rangeID: tblInfo.Columns[4].ID, - idxID: tblInfo.Indices[4].ID, - idxCols: 0, - eqCount: 32, - }, - { - sql: "select * from t use index(idx_af) where a = 1 and f < 21", - hist: "column:6 ndv:20 totColSize:400\n" + - "num: 13 lower_bound: -999999999999999.99 upper_bound: 6.00 repeats: 0\n" + - "num: 12 lower_bound: 7.00 upper_bound: 13.00 repeats: 0\n" + - "num: 10 lower_bound: 14.00 upper_bound: 21.00 repeats: 0", - rangeID: tblInfo.Columns[5].ID, - idxID: tblInfo.Indices[5].ID, - idxCols: 0, - eqCount: 32, - }, - { - sql: "select * from t use index(idx_ag) where a = 1 and g < 21", - hist: "column:7 ndv:20 totColSize:196\n" + - "num: 13 lower_bound: -838:59:59 upper_bound: 00:00:06 repeats: 0\n" + - "num: 12 lower_bound: 00:00:07 upper_bound: 00:00:13 repeats: 0\n" + - "num: 10 lower_bound: 00:00:14 upper_bound: 00:00:21 repeats: 0", - rangeID: tblInfo.Columns[6].ID, - idxID: tblInfo.Indices[6].ID, - idxCols: 0, - eqCount: 30, - }, - { - sql: `select * from t use index(idx_ah) where a = 1 and h < "1000-01-21"`, - hist: "column:8 ndv:20 totColSize:360\n" + - "num: 13 lower_bound: 1000-01-01 upper_bound: 1000-01-07 repeats: 0\n" + - "num: 12 lower_bound: 1000-01-08 upper_bound: 1000-01-14 repeats: 0\n" + - "num: 10 lower_bound: 1000-01-15 upper_bound: 1000-01-21 repeats: 0", - rangeID: tblInfo.Columns[7].ID, - idxID: tblInfo.Indices[7].ID, - idxCols: 0, - eqCount: 32, - }, - } - for i, test := range tests { - testKit.MustQuery(test.sql) - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - require.NoError(t, h.DumpStatsFeedbackToKV()) - require.NoError(t, h.HandleUpdateStats(dom.InfoSchema())) - require.NoError(t, h.Update(is)) - tbl := h.GetTableStats(tblInfo) - if test.idxCols == 0 { - require.Equal(t, tests[i].hist, tbl.Columns[test.rangeID].ToString(0)) - } else { - require.Equal(t, tests[i].hist, tbl.Indices[test.rangeID].ToString(1)) - } - val, err := codec.EncodeKey(testKit.Session().GetSessionVars().StmtCtx, nil, types.NewIntDatum(1)) - require.NoError(t, err) - require.Equal(t, uint64(test.eqCount), tbl.Indices[test.idxID].CMSketch.QueryBytes(val)) - } -} - -func TestIndexQueryFeedback4TopN(t *testing.T) { - store, dom := testkit.CreateMockStoreAndDomain(t) - testKit := testkit.NewTestKit(t, store) - - oriProbability := statistics.FeedbackProbability.Load() - oriMinLogCount := handle.MinLogScanCount.Load() - oriErrorRate := handle.MinLogErrorRate.Load() - defer func() { - statistics.FeedbackProbability.Store(oriProbability) - handle.MinLogScanCount.Store(oriMinLogCount) - handle.MinLogErrorRate.Store(oriErrorRate) - }() - statistics.FeedbackProbability.Store(1) - handle.MinLogScanCount.Store(0) - handle.MinLogErrorRate.Store(0) - - testKit.MustExec("use test") - - // TODO(tiancaiamao): query feedback is broken when paging is on. - testKit.MustExec("set @@tidb_enable_paging = off") - - testKit.MustExec("set @@session.tidb_analyze_version = 0") - testKit.MustExec("create table t (a bigint(64), index idx(a))") - for i := 0; i < 20; i++ { - testKit.MustExec(`insert into t values (1)`) - } - h := dom.StatsHandle() - err := h.HandleDDLEvent(<-h.DDLEventCh()) - require.NoError(t, err) - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - testKit.MustExec("set @@session.tidb_analyze_version = 1") - testKit.MustExec("set @@tidb_enable_fast_analyze = 1") - testKit.MustExec("analyze table t with 3 buckets") - for i := 0; i < 20; i++ { - testKit.MustExec(`insert into t values (1)`) - } - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - is := dom.InfoSchema() - require.NoError(t, h.Update(is)) - table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - require.NoError(t, err) - tblInfo := table.Meta() - - testKit.MustQuery("select * from t use index(idx) where a = 1") - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - require.NoError(t, h.DumpStatsFeedbackToKV()) - require.NoError(t, h.HandleUpdateStats(dom.InfoSchema())) - require.NoError(t, h.Update(is)) - tbl := h.GetTableStats(tblInfo) - val, err := codec.EncodeKey(testKit.Session().GetSessionVars().StmtCtx, nil, types.NewIntDatum(1)) - require.NoError(t, err) - require.Equal(t, uint64(40), tbl.Indices[1].CMSketch.QueryBytes(val)) -} - -func TestAbnormalIndexFeedback(t *testing.T) { - store, dom := testkit.CreateMockStoreAndDomain(t) - testKit := testkit.NewTestKit(t, store) - - oriProbability := statistics.FeedbackProbability.Load() - oriMinLogCount := handle.MinLogScanCount.Load() - oriErrorRate := handle.MinLogErrorRate.Load() - defer func() { - statistics.FeedbackProbability.Store(oriProbability) - handle.MinLogScanCount.Store(oriMinLogCount) - handle.MinLogErrorRate.Store(oriErrorRate) - }() - statistics.FeedbackProbability.Store(1) - handle.MinLogScanCount.Store(0) - handle.MinLogErrorRate.Store(0) - - testKit.MustExec("use test") - - // TODO(tiancaiamao): query feedback is broken when paging is on. - testKit.MustExec("set @@tidb_enable_paging = off") - - testKit.MustExec("create table t (a bigint(64), b bigint(64), index idx_ab(a,b))") - for i := 0; i < 20; i++ { - testKit.MustExec(fmt.Sprintf("insert into t values (%d, %d)", i/5, i)) - } - testKit.MustExec("set @@session.tidb_analyze_version = 1") - testKit.MustExec("analyze table t with 3 buckets, 0 topn") - testKit.MustExec("delete from t where a = 1") - testKit.MustExec("delete from t where b > 10") - is := dom.InfoSchema() - table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - require.NoError(t, err) - tblInfo := table.Meta() - h := dom.StatsHandle() - tests := []struct { - sql string - hist string - rangeID int64 - idxID int64 - eqCount uint32 - }{ - { - // The real count of `a = 1` is 0. - sql: "select * from t where a = 1 and b < 21", - hist: "column:2 ndv:20 totColSize:20\n" + - "num: 5 lower_bound: -9223372036854775808 upper_bound: 7 repeats: 0 ndv: 0\n" + - "num: 4 lower_bound: 7 upper_bound: 14 repeats: 0 ndv: 0\n" + - "num: 4 lower_bound: 14 upper_bound: 21 repeats: 0 ndv: 0", - rangeID: tblInfo.Columns[1].ID, - idxID: tblInfo.Indices[0].ID, - eqCount: 3, - }, - { - // The real count of `b > 10` is 0. - sql: "select * from t where a = 2 and b > 10", - hist: "column:2 ndv:20 totColSize:20\n" + - "num: 5 lower_bound: -9223372036854775808 upper_bound: 7 repeats: 0 ndv: 0\n" + - "num: 6 lower_bound: 7 upper_bound: 14 repeats: 0 ndv: 0\n" + - "num: 8 lower_bound: 14 upper_bound: 9223372036854775807 repeats: 0 ndv: 0", - rangeID: tblInfo.Columns[1].ID, - idxID: tblInfo.Indices[0].ID, - eqCount: 3, - }, - } - for i, test := range tests { - testKit.MustQuery(test.sql) - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - require.NoError(t, h.DumpStatsFeedbackToKV()) - require.NoError(t, h.HandleUpdateStats(dom.InfoSchema())) - require.NoError(t, h.Update(is)) - tbl := h.GetTableStats(tblInfo) - require.Equal(t, tests[i].hist, tbl.Columns[test.rangeID].ToString(0)) - val, err := codec.EncodeKey(testKit.Session().GetSessionVars().StmtCtx, nil, types.NewIntDatum(1)) - require.NoError(t, err) - require.Equal(t, uint64(test.eqCount), tbl.Indices[test.idxID].CMSketch.QueryBytes(val)) - } -} - -func TestFeedbackRanges(t *testing.T) { - store, dom := testkit.CreateMockStoreAndDomain(t) - testKit := testkit.NewTestKit(t, store) - h := dom.StatsHandle() - oriProbability := statistics.FeedbackProbability.Load() - oriNumber := statistics.MaxNumberOfRanges - oriMinLogCount := handle.MinLogScanCount.Load() - oriErrorRate := handle.MinLogErrorRate.Load() - defer func() { - statistics.FeedbackProbability.Store(oriProbability) - statistics.MaxNumberOfRanges = oriNumber - handle.MinLogScanCount.Store(oriMinLogCount) - handle.MinLogErrorRate.Store(oriErrorRate) - }() - statistics.FeedbackProbability.Store(1) - handle.MinLogScanCount.Store(0) - handle.MinLogErrorRate.Store(0) - - testKit.MustExec("use test") - - // TODO(tiancaiamao): query feedback is broken when paging is on. - testKit.MustExec("set @@tidb_enable_paging = off") - - testKit.MustExec("create table t (a tinyint, b tinyint, primary key(a), index idx(a, b))") - for i := 0; i < 20; i++ { - testKit.MustExec(fmt.Sprintf("insert into t values (%d, %d)", i, i)) - } - err := h.HandleDDLEvent(<-h.DDLEventCh()) - require.NoError(t, err) - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - testKit.MustExec("set @@session.tidb_analyze_version=1") - testKit.MustExec("analyze table t with 3 buckets") - for i := 30; i < 40; i++ { - testKit.MustExec(fmt.Sprintf("insert into t values (%d, %d)", i, i)) - } - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - tests := []struct { - sql string - hist string - colID int64 - }{ - { - sql: "select * from t where a <= 50 or (a > 130 and a < 140)", - hist: "column:1 ndv:30 totColSize:0\n" + - "num: 8 lower_bound: -128 upper_bound: 8 repeats: 0 ndv: 0\n" + - "num: 8 lower_bound: 8 upper_bound: 16 repeats: 0 ndv: 0\n" + - "num: 14 lower_bound: 16 upper_bound: 50 repeats: 0 ndv: 0", - colID: 1, - }, - { - sql: "select * from t where a >= 10", - hist: "column:1 ndv:30 totColSize:0\n" + - "num: 8 lower_bound: -128 upper_bound: 8 repeats: 0 ndv: 0\n" + - "num: 8 lower_bound: 8 upper_bound: 16 repeats: 0 ndv: 0\n" + - "num: 14 lower_bound: 16 upper_bound: 127 repeats: 0 ndv: 0", - colID: 1, - }, - { - sql: "select * from t use index(idx) where a = 1 and (b <= 50 or (b > 130 and b < 140))", - hist: "column:2 ndv:20 totColSize:30\n" + - "num: 8 lower_bound: -128 upper_bound: 7 repeats: 0 ndv: 0\n" + - "num: 8 lower_bound: 7 upper_bound: 14 repeats: 0 ndv: 0\n" + - "num: 7 lower_bound: 14 upper_bound: 51 repeats: 0 ndv: 0", - colID: 2, - }, - } - is := dom.InfoSchema() - table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - for _, test := range tests { - testKit.MustQuery(test.sql) - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - require.NoError(t, h.DumpStatsFeedbackToKV()) - require.NoError(t, h.HandleUpdateStats(dom.InfoSchema())) - require.NoError(t, err) - require.NoError(t, h.Update(is)) - tblInfo := table.Meta() - tbl := h.GetTableStats(tblInfo) - require.Equal(t, test.hist, tbl.Columns[test.colID].ToString(0)) - } -} - -func TestUnsignedFeedbackRanges(t *testing.T) { - store, dom := testkit.CreateMockStoreAndDomain(t) - testKit := testkit.NewTestKit(t, store) - h := dom.StatsHandle() - - oriProbability := statistics.FeedbackProbability.Load() - oriMinLogCount := handle.MinLogScanCount.Load() - oriErrorRate := handle.MinLogErrorRate.Load() - oriNumber := statistics.MaxNumberOfRanges - defer func() { - statistics.FeedbackProbability.Store(oriProbability) - handle.MinLogScanCount.Store(oriMinLogCount) - handle.MinLogErrorRate.Store(oriErrorRate) - statistics.MaxNumberOfRanges = oriNumber - }() - statistics.FeedbackProbability.Store(1) - handle.MinLogScanCount.Store(0) - handle.MinLogErrorRate.Store(0) - - testKit.MustExec("use test") - - // TODO(tiancaiamao): query feedback is broken when paging is on. - testKit.MustExec("set @@tidb_enable_paging = off") - - testKit.MustExec("set @@session.tidb_analyze_version = 0") - testKit.MustExec("create table t (a tinyint unsigned, primary key(a))") - testKit.MustExec("create table t1 (a bigint unsigned, primary key(a))") - for i := 0; i < 20; i++ { - testKit.MustExec(fmt.Sprintf("insert into t values (%d)", i)) - testKit.MustExec(fmt.Sprintf("insert into t1 values (%d)", i)) - } - err := h.HandleDDLEvent(<-h.DDLEventCh()) - require.NoError(t, err) - err = h.HandleDDLEvent(<-h.DDLEventCh()) - require.NoError(t, err) - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - testKit.MustExec("analyze table t, t1 with 3 buckets") - for i := 30; i < 40; i++ { - testKit.MustExec(fmt.Sprintf("insert into t values (%d)", i)) - testKit.MustExec(fmt.Sprintf("insert into t1 values (%d)", i)) - } - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - tests := []struct { - sql string - hist string - tblName string - }{ - { - sql: "select * from t where a <= 50", - hist: "column:1 ndv:30 totColSize:10\n" + - "num: 8 lower_bound: 0 upper_bound: 8 repeats: 0 ndv: 0\n" + - "num: 8 lower_bound: 8 upper_bound: 16 repeats: 0 ndv: 0\n" + - "num: 14 lower_bound: 16 upper_bound: 50 repeats: 0 ndv: 0", - tblName: "t", - }, - { - sql: "select count(*) from t", - hist: "column:1 ndv:30 totColSize:10\n" + - "num: 8 lower_bound: 0 upper_bound: 8 repeats: 0 ndv: 0\n" + - "num: 8 lower_bound: 8 upper_bound: 16 repeats: 0 ndv: 0\n" + - "num: 14 lower_bound: 16 upper_bound: 255 repeats: 0 ndv: 0", - tblName: "t", - }, - { - sql: "select * from t1 where a <= 50", - hist: "column:1 ndv:30 totColSize:10\n" + - "num: 8 lower_bound: 0 upper_bound: 8 repeats: 0 ndv: 0\n" + - "num: 8 lower_bound: 8 upper_bound: 16 repeats: 0 ndv: 0\n" + - "num: 14 lower_bound: 16 upper_bound: 50 repeats: 0 ndv: 0", - tblName: "t1", - }, - { - sql: "select count(*) from t1", - hist: "column:1 ndv:30 totColSize:10\n" + - "num: 8 lower_bound: 0 upper_bound: 8 repeats: 0 ndv: 0\n" + - "num: 8 lower_bound: 8 upper_bound: 16 repeats: 0 ndv: 0\n" + - "num: 14 lower_bound: 16 upper_bound: 18446744073709551615 repeats: 0 ndv: 0", - tblName: "t1", - }, - } - is := dom.InfoSchema() - require.NoError(t, h.Update(is)) - for _, test := range tests { - table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr(test.tblName)) - require.NoError(t, err) - testKit.MustQuery(test.sql) - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - require.NoError(t, h.DumpStatsFeedbackToKV()) - require.NoError(t, h.HandleUpdateStats(dom.InfoSchema())) - require.NoError(t, err) - require.NoError(t, h.Update(is)) - tblInfo := table.Meta() - tbl := h.GetTableStats(tblInfo) - require.Equal(t, test.hist, tbl.Columns[1].ToString(0)) - } -} - func TestLoadHistCorrelation(t *testing.T) { store, dom := testkit.CreateMockStoreAndDomain(t) testKit := testkit.NewTestKit(t, store) @@ -1927,36 +901,6 @@ func TestLoadHistCorrelation(t *testing.T) { require.Equal(t, "1", result.Rows()[0][9]) } -func TestDeleteUpdateFeedback(t *testing.T) { - store, dom := testkit.CreateMockStoreAndDomain(t) - testKit := testkit.NewTestKit(t, store) - - oriProbability := statistics.FeedbackProbability.Load() - defer func() { - statistics.FeedbackProbability.Store(oriProbability) - }() - statistics.FeedbackProbability.Store(1) - - h := dom.StatsHandle() - testKit.MustExec("use test") - testKit.MustExec("create table t (a bigint(64), b bigint(64), index idx_ab(a,b))") - for i := 0; i < 20; i++ { - testKit.MustExec(fmt.Sprintf("insert into t values (%d, %d)", i/5, i)) - } - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - testKit.MustExec("analyze table t with 3 buckets") - - testKit.MustExec("delete from t where a = 1") - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - require.Equal(t, 0, h.GetQueryFeedback().Size) - testKit.MustExec("update t set a = 6 where a = 2") - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - require.Equal(t, 0, h.GetQueryFeedback().Size) - testKit.MustExec("explain analyze delete from t where a = 3") - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - require.Equal(t, 0, h.GetQueryFeedback().Size) -} - func BenchmarkHandleAutoAnalyze(b *testing.B) { store, dom := testkit.CreateMockStoreAndDomain(b) testKit := testkit.NewTestKit(b, store) @@ -1974,62 +918,6 @@ func subtraction(newMetric *dto.Metric, oldMetric *dto.Metric) int { return int(*(newMetric.Counter.Value) - *(oldMetric.Counter.Value)) } -func TestDisableFeedback(t *testing.T) { - store := testkit.CreateMockStore(t) - testKit := testkit.NewTestKit(t, store) - - oriProbability := statistics.FeedbackProbability.Load() - defer func() { - statistics.FeedbackProbability.Store(oriProbability) - }() - statistics.FeedbackProbability.Store(0.0) - oldNum := &dto.Metric{} - err := metrics.StoreQueryFeedbackCounter.WithLabelValues(metrics.LblOK).Write(oldNum) - require.NoError(t, err) - testKit.MustExec("use test") - testKit.MustExec("create table t (a int, b int, index idx_a(a))") - testKit.MustExec("insert into t values (1, 1), (2, 2), (3, 3), (5, 5)") - testKit.MustExec("analyze table t with 0 topn") - for i := 0; i < 20; i++ { - testKit.MustQuery("select /*+ use_index(t, idx_a) */ * from t where a < 4") - } - - newNum := &dto.Metric{} - err = metrics.StoreQueryFeedbackCounter.WithLabelValues(metrics.LblOK).Write(newNum) - require.NoError(t, err) - require.Equal(t, 0, subtraction(newNum, oldNum)) -} - -func TestFeedbackCounter(t *testing.T) { - store := testkit.CreateMockStore(t) - testKit := testkit.NewTestKit(t, store) - - oriProbability := statistics.FeedbackProbability.Load() - defer func() { - statistics.FeedbackProbability.Store(oriProbability) - }() - statistics.FeedbackProbability.Store(1) - oldNum := &dto.Metric{} - err := metrics.StoreQueryFeedbackCounter.WithLabelValues(metrics.LblOK).Write(oldNum) - require.NoError(t, err) - testKit.MustExec("use test") - - // TODO(tiancaiamao): query feedback is broken when paging is on. - testKit.MustExec("set @@tidb_enable_paging = off") - - testKit.MustExec("create table t (a int, b int, index idx_a(a))") - testKit.MustExec("insert into t values (1, 1), (2, 2), (3, 3), (5, 5)") - testKit.MustExec("analyze table t with 0 topn") - for i := 0; i < 20; i++ { - testKit.MustQuery("select /*+ use_index(t, idx_a) */ * from t where a < 4") - } - - newNum := &dto.Metric{} - err = metrics.StoreQueryFeedbackCounter.WithLabelValues(metrics.LblOK).Write(newNum) - require.NoError(t, err) - require.Equal(t, 20, subtraction(newNum, oldNum)) -} - func TestMergeTopN(t *testing.T) { // Move this test to here to avoid race test. tests := []struct { @@ -2475,98 +1363,6 @@ func TestStatsLockUnlockForAutoAnalyze(t *testing.T) { require.Equal(t, int64(15), tblStats2.RealtimeCount) } -func TestStatsLockForFeedback(t *testing.T) { - store, dom := testkit.CreateMockStoreAndDomain(t) - testKit := testkit.NewTestKit(t, store) - testKit.MustExec("use test") - - // TODO(tiancaiamao): query feedback is broken when paging is on. - testKit.MustExec("set @@tidb_enable_paging = off") - - testKit.MustExec("set @@session.tidb_analyze_version = 0") - testKit.MustExec("create table t (a bigint(64), b bigint(64), primary key(a), index idx(b))") - testKit.MustExec("insert into t values (1,2),(2,2),(4,5)") - testKit.MustExec("analyze table t with 0 topn") - testKit.MustExec("insert into t values (3,4)") - for i := 5; i < 20; i++ { - testKit.MustExec(fmt.Sprintf("insert into t values(%d, %d)", i, i+1)) - } - - h := dom.StatsHandle() - oriProbability := statistics.FeedbackProbability.Load() - oriNumber := statistics.MaxNumberOfRanges - oriMinLogCount := handle.MinLogScanCount.Load() - oriErrorRate := handle.MinLogErrorRate.Load() - defer func() { - statistics.FeedbackProbability.Store(oriProbability) - statistics.MaxNumberOfRanges = oriNumber - handle.MinLogScanCount.Store(oriMinLogCount) - handle.MinLogErrorRate.Store(oriErrorRate) - }() - statistics.FeedbackProbability.Store(1) - handle.MinLogScanCount.Store(0) - handle.MinLogErrorRate.Store(0) - tests := []struct { - sql string - hist string - }{ - { - // test primary key feedback - sql: "select * from t where t.a <= 4 order by a desc", - hist: "column:1 ndv:4 totColSize:0\n" + - "num: 1 lower_bound: -9223372036854775808 upper_bound: 2 repeats: 0 ndv: 0\n" + - "num: 2 lower_bound: 2 upper_bound: 4 repeats: 0 ndv: 0\n" + - "num: 1 lower_bound: 4 upper_bound: 4 repeats: 1 ndv: 0", - }, - //run 1st sql after table locked, hist should not changed - { - sql: "select * from t where t.a <= 8 order by a desc", - hist: "column:1 ndv:4 totColSize:0\n" + - "num: 1 lower_bound: -9223372036854775808 upper_bound: 2 repeats: 0 ndv: 0\n" + - "num: 2 lower_bound: 2 upper_bound: 4 repeats: 0 ndv: 0\n" + - "num: 1 lower_bound: 4 upper_bound: 4 repeats: 1 ndv: 0", - }, - //run 2nd sql after table unlocked, hist should not changed - { - sql: "select * from t where t.a <= 12 order by a desc", - hist: "column:1 ndv:12 totColSize:0\n" + - "num: 1 lower_bound: -9223372036854775808 upper_bound: 2 repeats: 0 ndv: 0\n" + - "num: 2 lower_bound: 2 upper_bound: 4 repeats: 0 ndv: 0\n" + - "num: 9 lower_bound: 4 upper_bound: 12 repeats: 0 ndv: 0", - }, - //run 4th sql after table locked, hist should not changed - { - sql: "select * from t", - hist: "column:1 ndv:12 totColSize:0\n" + - "num: 1 lower_bound: -9223372036854775808 upper_bound: 2 repeats: 0 ndv: 0\n" + - "num: 2 lower_bound: 2 upper_bound: 4 repeats: 0 ndv: 0\n" + - "num: 9 lower_bound: 4 upper_bound: 12 repeats: 0 ndv: 0", - }, - } - is := dom.InfoSchema() - table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - for i, test := range tests { - testKit.MustQuery(test.sql) - require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) - require.NoError(t, h.DumpStatsFeedbackToKV()) - require.NoError(t, h.HandleUpdateStats(dom.InfoSchema())) - require.NoError(t, err) - require.NoError(t, h.Update(is)) - tblInfo := table.Meta() - tbl := h.GetTableStats(tblInfo) - //fmt.Printf("\n i: %d, exp: %s, \nact: %s\n", i, tests[i].hist, tbl.Columns[tblInfo.Columns[0].ID].ToString(0)) - require.Equal(t, tests[i].hist, tbl.Columns[tblInfo.Columns[0].ID].ToString(0)) - // add table lock after 2nd - if i == 0 { - testKit.MustExec("lock stats t") - } else if i == 1 { - testKit.MustExec("unlock stats t") - } else if i == 2 { - testKit.MustExec("lock stats t") - } - } -} - func TestStatsLockForDelta(t *testing.T) { store, dom := testkit.CreateMockStoreAndDomain(t) testKit := testkit.NewTestKit(t, store) diff --git a/statistics/histogram.go b/statistics/histogram.go index 2b757d087f784..d690bfd555f34 100644 --- a/statistics/histogram.go +++ b/statistics/histogram.go @@ -237,7 +237,7 @@ const ( Version2 = 2 ) -// AnalyzeFlag is set when the statistics comes from analyze and has not been modified by feedback. +// AnalyzeFlag is set when the statistics comes from analyze. const AnalyzeFlag = 1 // IsAnalyzed checks whether this flag contains AnalyzeFlag. @@ -245,11 +245,6 @@ func IsAnalyzed(flag int64) bool { return (flag & AnalyzeFlag) > 0 } -// ResetAnalyzeFlag resets the AnalyzeFlag because it has been modified by feedback. -func ResetAnalyzeFlag(flag int64) int64 { - return flag &^ AnalyzeFlag -} - // ValueToString converts a possible encoded value to a formatted string. If the value is encoded, then // idxCols equals to number of origin values, else idxCols is 0. func ValueToString(vars *variable.SessionVars, value *types.Datum, idxCols int, idxColumnTypes []byte) (string, error) { diff --git a/types/datum.go b/types/datum.go index d03aa62464fda..8d08f73d9798b 100644 --- a/types/datum.go +++ b/types/datum.go @@ -2271,7 +2271,6 @@ func (ds *datumsSorter) Len() int { } func (ds *datumsSorter) Less(i, j int) bool { - // TODO: set collation explicitly when rewrites feedback. cmp, err := ds.datums[i].Compare(ds.sc, &ds.datums[j], collate.GetCollator(ds.datums[i].Collation())) if err != nil { ds.err = errors.Trace(err) diff --git a/util/mock/context.go b/util/mock/context.go index f2dab9d9e0faa..d4e489d17ab26 100644 --- a/util/mock/context.go +++ b/util/mock/context.go @@ -331,9 +331,6 @@ func (c *Context) GoCtx() context.Context { return c.ctx } -// StoreQueryFeedback stores the query feedback. -func (*Context) StoreQueryFeedback(_ interface{}) {} - // UpdateColStatsUsage updates the column stats usage. func (*Context) UpdateColStatsUsage(_ []model.TableItemID) {}