From a43bc8c8ea65752fc5c333be2f1dd594fa315df5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=8E=8B=E8=B6=85?= Date: Thu, 25 Aug 2022 11:48:21 +0800 Subject: [PATCH 1/4] *: fix data race in `TestTiFlashGroupIndexWhenStartup` (#37371) close pingcap/tidb#37370 --- ddl/ddl_tiflash_test.go | 9 +++++---- domain/infosync/tiflash_manager.go | 20 +++++++++++++++++--- 2 files changed, 22 insertions(+), 7 deletions(-) diff --git a/ddl/ddl_tiflash_test.go b/ddl/ddl_tiflash_test.go index 2fb42d76dd507..b86c420d605fb 100644 --- a/ddl/ddl_tiflash_test.go +++ b/ddl/ddl_tiflash_test.go @@ -998,18 +998,19 @@ func TestTiFlashBatchUnsupported(t *testing.T) { func TestTiFlashGroupIndexWhenStartup(t *testing.T) { s, teardown := createTiFlashContext(t) + tiflash := s.tiflash defer teardown() _ = testkit.NewTestKit(t, s.store) timeout := time.Now().Add(10 * time.Second) errMsg := "time out" for time.Now().Before(timeout) { time.Sleep(100 * time.Millisecond) - if s.tiflash.GroupIndex != 0 { + if tiflash.GetRuleGroupIndex() != 0 { errMsg = "invalid group index" break } } - require.Equal(t, placement.RuleIndexTiFlash, s.tiflash.GroupIndex, errMsg) - require.Greater(t, s.tiflash.GroupIndex, placement.RuleIndexTable) - require.Greater(t, s.tiflash.GroupIndex, placement.RuleIndexPartition) + require.Equal(t, placement.RuleIndexTiFlash, tiflash.GetRuleGroupIndex(), errMsg) + require.Greater(t, tiflash.GetRuleGroupIndex(), placement.RuleIndexTable) + require.Greater(t, tiflash.GetRuleGroupIndex(), placement.RuleIndexPartition) } diff --git a/domain/infosync/tiflash_manager.go b/domain/infosync/tiflash_manager.go index 7040eefee837b..eb929a776ae58 100644 --- a/domain/infosync/tiflash_manager.go +++ b/domain/infosync/tiflash_manager.go @@ -305,7 +305,7 @@ func (m *mockTiFlashTableInfo) String() string { // MockTiFlash mocks a TiFlash, with necessary Pd support. type MockTiFlash struct { sync.Mutex - GroupIndex int + groupIndex int StatusAddr string StatusServer *httptest.Server SyncStatus map[int]mockTiFlashTableInfo @@ -373,7 +373,7 @@ func NewMockTiFlash() *MockTiFlash { func (tiflash *MockTiFlash) HandleSetPlacementRule(rule placement.TiFlashRule) error { tiflash.Lock() defer tiflash.Unlock() - tiflash.GroupIndex = placement.RuleIndexTiFlash + tiflash.groupIndex = placement.RuleIndexTiFlash if !tiflash.PdEnabled { logutil.BgLogger().Info("pd server is manually disabled, just quit") return nil @@ -492,6 +492,20 @@ func (tiflash *MockTiFlash) HandleGetStoresStat() *helper.StoresStat { } } +// SetRuleGroupIndex sets the group index of tiflash +func (tiflash *MockTiFlash) SetRuleGroupIndex(groupIndex int) { + tiflash.Lock() + defer tiflash.Unlock() + tiflash.groupIndex = groupIndex +} + +// GetRuleGroupIndex gets the group index of tiflash +func (tiflash *MockTiFlash) GetRuleGroupIndex() int { + tiflash.Lock() + defer tiflash.Unlock() + return tiflash.groupIndex +} + // Compare supposed rule, and we actually get from TableInfo func isRuleMatch(rule placement.TiFlashRule, startKey string, endKey string, count int, labels []string) bool { // Compute startKey @@ -598,7 +612,7 @@ func (m *mockTiFlashPlacementManager) SetTiFlashGroupConfig(_ context.Context) e if m.tiflash == nil { return nil } - m.tiflash.GroupIndex = placement.RuleIndexTiFlash + m.tiflash.SetRuleGroupIndex(placement.RuleIndexTiFlash) return nil } From d4352676007c405036951118c5136c91537c21d0 Mon Sep 17 00:00:00 2001 From: Song Gao Date: Thu, 25 Aug 2022 12:14:21 +0800 Subject: [PATCH 2/4] statistics: track used stats status during query (#37101) ref pingcap/tidb#36481 --- executor/adapter.go | 39 +++++++++++++++++++++++++++++ executor/executor.go | 1 + planner/core/plan_stats_test.go | 24 ++++++++++++++++++ sessionctx/stmtctx/stmtctx.go | 2 ++ sessionctx/variable/session.go | 23 ++++++++++++++++- sessionctx/variable/session_test.go | 10 ++++++-- statistics/selectivity.go | 5 ++-- statistics/table.go | 29 ++++++++++++++++++--- 8 files changed, 124 insertions(+), 9 deletions(-) diff --git a/executor/adapter.go b/executor/adapter.go index 4a648f4029c42..d946d8dc3c9c7 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -1296,6 +1296,7 @@ func (a *ExecStmt) LogSlowQuery(txnTS uint64, succ bool, hasMoreResults bool) { ExecRetryCount: a.retryCount, IsExplicitTxn: sessVars.TxnCtx.IsExplicit, IsWriteCacheTable: stmtCtx.WaitLockLeaseTime > 0, + StatsLoadStatus: convertStatusIntoString(a.Ctx, stmtCtx.StatsLoadStatus), IsSyncStatsFailed: stmtCtx.IsSyncStatsFailed, } failpoint.Inject("assertSyncStatsFailed", func(val failpoint.Value) { @@ -1667,3 +1668,41 @@ func (a *ExecStmt) getSQLPlanDigest() ([]byte, []byte) { } return sqlDigest, planDigest } + +func convertStatusIntoString(sctx sessionctx.Context, statsLoadStatus map[model.TableItemID]string) map[string]map[string]string { + if len(statsLoadStatus) < 1 { + return nil + } + is := domain.GetDomain(sctx).InfoSchema() + // tableName -> name -> status + r := make(map[string]map[string]string) + for item, status := range statsLoadStatus { + t, ok := is.TableByID(item.TableID) + if !ok { + t, _, _ = is.FindTableByPartitionID(item.TableID) + } + if t == nil { + logutil.BgLogger().Warn("record table item load status failed due to not finding table", + zap.Int64("tableID", item.TableID)) + continue + } + tableName := t.Meta().Name.O + itemName := "" + if item.IsIndex { + itemName = t.Meta().FindIndexNameByID(item.ID) + } else { + itemName = t.Meta().FindColumnNameByID(item.ID) + } + if itemName == "" { + logutil.BgLogger().Warn("record table item load status failed due to not finding item", + zap.Int64("tableID", item.TableID), + zap.Int64("id", item.ID), zap.Bool("isIndex", item.IsIndex)) + continue + } + if r[tableName] == nil { + r[tableName] = make(map[string]string) + } + r[tableName][itemName] = status + } + return r +} diff --git a/executor/executor.go b/executor/executor.go index fc177713d70c9..e836ab6d522d3 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -1925,6 +1925,7 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) { sc.EnableOptimizeTrace = false sc.OptimizeTracer = nil sc.OptimizerCETrace = nil + sc.StatsLoadStatus = make(map[model.TableItemID]string) sc.IsSyncStatsFailed = false sc.SysdateIsNow = ctx.GetSessionVars().SysdateIsNow diff --git a/planner/core/plan_stats_test.go b/planner/core/plan_stats_test.go index 8f23858cdabc4..b9836f126424e 100644 --- a/planner/core/plan_stats_test.go +++ b/planner/core/plan_stats_test.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/model" @@ -293,3 +294,26 @@ func TestPlanStatsLoadTimeout(t *testing.T) { t.Error("unexpected plan:", pp) } } + +func TestPlanStatsStatusRecord(t *testing.T) { + restore := config.RestoreFunc() + defer restore() + config.UpdateGlobal(func(conf *config.Config) { + conf.Performance.EnableStatsCacheMemQuota = true + }) + store, _ := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec(`create table t (b int,key b(b))`) + tk.MustExec("insert into t (b) values (1)") + tk.MustExec("analyze table t") + tk.MustQuery("select * from t where b >= 1") + require.Len(t, tk.Session().GetSessionVars().StmtCtx.StatsLoadStatus, 0) + // drop stats in order to change status + domain.GetDomain(tk.Session()).StatsHandle().SetStatsCacheCapacity(1) + tk.MustQuery("select * from t where b >= 1") + require.Len(t, tk.Session().GetSessionVars().StmtCtx.StatsLoadStatus, 2) + for _, status := range tk.Session().GetSessionVars().StmtCtx.StatsLoadStatus { + require.Equal(t, status, "allEvicted") + } +} diff --git a/sessionctx/stmtctx/stmtctx.go b/sessionctx/stmtctx/stmtctx.go index 98f2c6b27a244..f62df41de3615 100644 --- a/sessionctx/stmtctx/stmtctx.go +++ b/sessionctx/stmtctx/stmtctx.go @@ -312,6 +312,8 @@ type StatementContext struct { // IsSQLAndPlanRegistered uses to indicate whether the SQL and plan has been registered for TopSQL. IsSQLAndPlanRegistered atomic2.Bool + // StatsLoadStatus records StatsLoadedStatus for the index/column which is used in query + StatsLoadStatus map[model.TableItemID]string // IsSyncStatsFailed indicates whether any failure happened during sync stats IsSyncStatsFailed bool } diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 4a03cc331773d..02b909fc31da5 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -2570,6 +2570,8 @@ type SlowQueryLogItems struct { ResultRows int64 IsExplicitTxn bool IsWriteCacheTable bool + // table -> name -> status + StatsLoadStatus map[string]map[string]string IsSyncStatsFailed bool } @@ -2665,6 +2667,9 @@ func (s *SessionVars) SlowLogFormat(logItems *SlowQueryLogItems) string { buf.WriteString(k + ":" + vStr) firstComma = true } + if v != 0 && len(logItems.StatsLoadStatus[k]) > 0 { + writeStatsLoadStatusItems(&buf, logItems.StatsLoadStatus[k]) + } } buf.WriteString("\n") } @@ -2748,7 +2753,6 @@ func (s *SessionVars) SlowLogFormat(logItems *SlowQueryLogItems) string { if len(logItems.BinaryPlan) != 0 { writeSlowLogItem(&buf, SlowLogBinaryPlan, logItems.BinaryPlan) } - if logItems.PrevStmt != "" { writeSlowLogItem(&buf, SlowLogPrevStmt, logItems.PrevStmt) } @@ -2762,9 +2766,26 @@ func (s *SessionVars) SlowLogFormat(logItems *SlowQueryLogItems) string { if len(logItems.SQL) == 0 || logItems.SQL[len(logItems.SQL)-1] != ';' { buf.WriteString(";") } + return buf.String() } +func writeStatsLoadStatusItems(buf *bytes.Buffer, loadStatus map[string]string) { + if len(loadStatus) > 0 { + buf.WriteString("[") + firstComma := false + for name, status := range loadStatus { + if firstComma { + buf.WriteString("," + name + ":" + status) + } else { + buf.WriteString(name + ":" + status) + firstComma = true + } + } + buf.WriteString("]") + } +} + // writeSlowLogItem writes a slow log item in the form of: "# ${key}:${value}" func writeSlowLogItem(buf *bytes.Buffer, key, value string) { buf.WriteString(SlowLogRowPrefixStr + key + SlowLogSpaceMarkStr + value + "\n") diff --git a/sessionctx/variable/session_test.go b/sessionctx/variable/session_test.go index 5e47a9da45197..6dc6812f85a03 100644 --- a/sessionctx/variable/session_test.go +++ b/sessionctx/variable/session_test.go @@ -166,7 +166,12 @@ func TestSlowLogFormat(t *testing.T) { }, } statsInfos := make(map[string]uint64) - statsInfos["t1"] = 0 + statsInfos["t1"] = 123 + loadStatus := make(map[string]map[string]string) + loadStatus["t1"] = map[string]string{ + "col1": "unInitialized", + } + copTasks := &stmtctx.CopTasksDetails{ NumCopTasks: 10, AvgProcessTime: time.Second, @@ -212,7 +217,7 @@ func TestSlowLogFormat(t *testing.T) { # Index_names: [t1:a,t2:b] # Is_internal: true # Digest: 01d00e6e93b28184beae487ac05841145d2a2f6a7b16de32a763bed27967e83d -# Stats: t1:pseudo +# Stats: t1:123[col1:unInitialized] # Num_cop_tasks: 10 # Cop_proc_avg: 1 Cop_proc_p90: 2 Cop_proc_max: 3 Cop_proc_addr: 10.6.131.78 # Cop_wait_avg: 0.01 Cop_wait_p90: 0.02 Cop_wait_max: 0.03 Cop_wait_addr: 10.6.131.79 @@ -270,6 +275,7 @@ func TestSlowLogFormat(t *testing.T) { ExecRetryTime: 5*time.Second + time.Millisecond*100, IsExplicitTxn: true, IsWriteCacheTable: true, + StatsLoadStatus: loadStatus, } logString := seVar.SlowLogFormat(logItems) require.Equal(t, resultFields+"\n"+sql, logString) diff --git a/statistics/selectivity.go b/statistics/selectivity.go index ba847473fde85..74027019a75c7 100644 --- a/statistics/selectivity.go +++ b/statistics/selectivity.go @@ -209,12 +209,11 @@ func (coll *HistColl) Selectivity(ctx sessionctx.Context, exprs []expression.Exp continue } - if colHist := coll.Columns[c.UniqueID]; colHist == nil || colHist.IsInvalid(ctx, coll.Pseudo) { + colHist := coll.Columns[c.UniqueID] + if colHist == nil || colHist.IsInvalid(ctx, coll.Pseudo) { ret *= 1.0 / pseudoEqualRate continue } - - colHist := coll.Columns[c.UniqueID] if colHist.Histogram.NDV > 0 { ret *= 1 / float64(colHist.Histogram.NDV) } else { diff --git a/statistics/table.go b/statistics/table.go index 3a13480489c95..384bf3c4c62b0 100644 --- a/statistics/table.go +++ b/statistics/table.go @@ -531,10 +531,12 @@ func (t *Table) ColumnEqualRowCount(sctx sessionctx.Context, value types.Datum, } // GetRowCountByIntColumnRanges estimates the row count by a slice of IntColumnRange. -func (coll *HistColl) GetRowCountByIntColumnRanges(sctx sessionctx.Context, colID int64, intRanges []*ranger.Range) (float64, error) { +func (coll *HistColl) GetRowCountByIntColumnRanges(sctx sessionctx.Context, colID int64, intRanges []*ranger.Range) (result float64, err error) { sc := sctx.GetSessionVars().StmtCtx - var result float64 c, ok := coll.Columns[colID] + if c != nil { + recordUsedItemStatsStatus(sctx, c.StatsLoadedStatus, coll.PhysicalID, colID, false) + } if !ok || c.IsInvalid(sctx, coll.Pseudo) { if len(intRanges) == 0 { return 0, nil @@ -549,7 +551,7 @@ func (coll *HistColl) GetRowCountByIntColumnRanges(sctx sessionctx.Context, colI } return result, nil } - result, err := c.GetColumnRowCount(sctx, intRanges, coll.Count, true) + result, err = c.GetColumnRowCount(sctx, intRanges, coll.Count, true) if sc.EnableOptimizerCETrace { CETraceRange(sctx, coll.PhysicalID, []string{c.Info.Name.O}, intRanges, "Column Stats", uint64(result)) } @@ -560,6 +562,9 @@ func (coll *HistColl) GetRowCountByIntColumnRanges(sctx sessionctx.Context, colI func (coll *HistColl) GetRowCountByColumnRanges(sctx sessionctx.Context, colID int64, colRanges []*ranger.Range) (float64, error) { sc := sctx.GetSessionVars().StmtCtx c, ok := coll.Columns[colID] + if c != nil { + recordUsedItemStatsStatus(sctx, c.StatsLoadedStatus, coll.PhysicalID, colID, false) + } if !ok || c.IsInvalid(sctx, coll.Pseudo) { result, err := GetPseudoRowCountByColumnRanges(sc, float64(coll.Count), colRanges, 0) if err == nil && sc.EnableOptimizerCETrace && ok { @@ -584,6 +589,9 @@ func (coll *HistColl) GetRowCountByIndexRanges(sctx sessionctx.Context, idxID in colNames = append(colNames, col.Name.O) } } + if idx != nil { + recordUsedItemStatsStatus(sctx, idx.StatsLoadedStatus, coll.PhysicalID, idxID, true) + } if !ok || idx.IsInvalid(coll.Pseudo) { colsLen := -1 if idx != nil && idx.Info.Unique { @@ -1378,3 +1386,18 @@ func CheckAnalyzeVerOnTable(tbl *Table, version *int) bool { // This table has no statistics yet. We can directly return true. return true } + +// recordUsedItemStatsStatus only records un-FullLoad item load status during user query +func recordUsedItemStatsStatus(sctx sessionctx.Context, loadStatus StatsLoadedStatus, + tableID, id int64, isIndex bool) { + if loadStatus.IsFullLoad() { + return + } + stmtCtx := sctx.GetSessionVars().StmtCtx + item := model.TableItemID{TableID: tableID, ID: id, IsIndex: isIndex} + // For some testcases, it skips ResetContextOfStmt to init StatsLoadStatus + if stmtCtx.StatsLoadStatus == nil { + stmtCtx.StatsLoadStatus = make(map[model.TableItemID]string) + } + stmtCtx.StatsLoadStatus[item] = loadStatus.StatusToString() +} From 16f015aa25b28fbae30dfe890ddc7f8bd4b0ae3c Mon Sep 17 00:00:00 2001 From: Yifan Xu <30385241+xuyifangreeneyes@users.noreply.github.com> Date: Thu, 25 Aug 2022 14:00:21 +0800 Subject: [PATCH 3/4] util/ranger: avoid building ranges twice in detachCNFCondAndBuildRangeForIndex (#37177) ref pingcap/tidb#37176 --- util/ranger/bench_test.go | 133 ++++++++++++++++++++++++++++++++++++++ util/ranger/detacher.go | 40 ++++++++---- util/ranger/ranger.go | 16 +++-- 3 files changed, 172 insertions(+), 17 deletions(-) create mode 100644 util/ranger/bench_test.go diff --git a/util/ranger/bench_test.go b/util/ranger/bench_test.go new file mode 100644 index 0000000000000..11032abb15240 --- /dev/null +++ b/util/ranger/bench_test.go @@ -0,0 +1,133 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package ranger_test + +import ( + "context" + "testing" + + "github.com/pingcap/tidb/expression" + plannercore "github.com/pingcap/tidb/planner/core" + "github.com/pingcap/tidb/session" + "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/util/ranger" + "github.com/stretchr/testify/require" +) + +func BenchmarkDetachCondAndBuildRangeForIndex(b *testing.B) { + store := testkit.CreateMockStore(b) + testKit := testkit.NewTestKit(b, store) + testKit.MustExec("USE test") + testKit.MustExec("DROP TABLE IF EXISTS t") + testKit.MustExec(` +CREATE TABLE t ( + id bigint(20) NOT NULL, + flag tinyint(1) NOT NULL, + start_time bigint(20) NOT NULL, + end_time bigint(20) NOT NULL, + org_id bigint(20) NOT NULL, + work_type varchar(20) CHARACTER SET latin1 DEFAULT NULL, + KEY idx (work_type, flag, org_id, end_time) +)`) + const longInListQuery = ` +SELECT + * +FROM + test.t +WHERE + flag = false + AND work_type = 'ART' + AND start_time < 1437233819011 + AND org_id IN ( + 6914445279216,13370400597894,23573357446410,25887731612857,47914138043476,55587486395595,81718641401629,116900287316705,145561135354847,146149695520705,150737101640717, + 156461412195734,159282589442914,172009309011258,177247737846954,186431984199126,192702036012003,198028383844036,203822128217766,204361899227892,222307981616040, + 223676791310456,239710781952147,284504079630201,293577747630605,294033490212963,295616321229667,296595862564733,306758307660237,312453546107536,341725138386801, + 365564508529854,367823101074613,372136375994705,379986503803645,391811651805728,400703265358976,402101198418969,418909618067022,431707392441212,433127041410999, + 453871162499705,459423393488794,468411967568789,470295426508062,474431880357867,486212302560292,491505628336549,501258676986205,512885499269076,541562269064881, + 573472004072939,588733301257768,615894236361514,615983097695492,632401842501612,647142957465854,718424118160213,727429900786891,730535283550760,738198024914914, + 739163331366841,752647396960214,756654620675819,771528730062551,780002237988119,819197371932508,841300462416232,850412457085478,874244695288775,881570046324012, + 888197716219703,891639557310370,891974587948562,894056936863746,896673560435518,907536217395121,911688951561329,915825484067466,955508673412475,970655461426927, + 972378619997200,995940920336141,1001616070548723,1002328390838858,1008841511217759,1027157044666161,1048868628313964,1066571897813670,1077847212985966,1080168008312256, + 1096474319900512,1102806753403857,1117700255389200,1121170709282654,1160894105829436,1173396752247253,1174823990738043,1176756778119821,1183639496028232,1224377491431012, + 1240466389734677,1265076660322902,1269569491282720,1272581574031499,1294849094933502,1295099688484884,1298881176988511,1299992642087679,1307669929489952,1338074385647926, + 1342415156765380,1360218717881999,1377658957083475,1392922543432192,1407444098594050,1438256495838735,1445134088155147,1486929584894557,1494681055271250,1500940354882616, + 1530516421560327,1532590570974295,1544648947254643,1561923580960451,1563587565476473,1565067823935596,1573069534228327,1573167213450271,1573297960637648,1577324450386436, + 1595234199645128,1595320706602899,1595934401297767,1616085587597345,1652295812716667,1655495487920136,1663475672388133,1668352817492466,1681094348275341,1689623403182214, + 1701682724927093,1705012823832699,1710393138044599,1716535649128474,1733575964270463,1750190609804974,1754580077690816,1756061776687456,1758058273255859,1775158332937577, + 1786728287430927,1816461420376899,1828580334315536,1858008005313000,1878841219054602,1878932921719554,1904081347331116,1904820184794904,1913069596895373,1941380005857044, + 1954836070968071,1955618820347782,1983296066429676,1987819713385690,1998098943250021,2013403425656222,2026046763398088,2026621786756595,2042249014990205,2046741004470190, + 2054370107316514,2082578854015062,2087183461591329,2087192311688265,2095277921868705,2103249621417272,2105159282073449,2106385538583803,2115025577188264,2115892671475192, + 2144855844328126,2145526421460724,2155282047243675,2170620433275766,2189596848694026,2194468311513858,2213049505974092,2213713514793282,2213911591088204,2215904332099994, + 2220235232411590,2226449138693468,2281727415070895,2288760906462231,2292201263531973,2292800860074434,2305015986098173,2311091146951322,2345959699274993,2356192386220089, + 2358508761766515,2372253813770528,2372994254274287,2374279183118353,2382395547546489,2397593929551113,2414904392525386,2416320859150803,2432407185506251,2439133858011514, + 2456909988876966,2490057196713078,2491000908839300,2501843499019962,2539856169115632,2542691236689315,2554165079560216,2565016421484028,2579373020942520,2583310536030502, + 2601385761450563,2603243552830302,2609138752573551,2628285706286347,2631734058797888,2638342575912817,2640167419150776,2651402302096646,2652627728219962,2677347449018607, + 2680209147172480,2683108753662485,2697695717735514,2699481485241986,2706019556864874,2707225343321107,2707841703322085,2724535386459144,2751805187614069,2759827465036125, + 2761426575202406,2768997857008184,2780782950787373,2789872834409453,2801402100587551,2806075464399632,2811893029385689,2816433481597910,2817217705468440,2817327738406355, + 2834191662385443,2838661299874842,2841835162527294,2842790844846179,2873548708258327,2879581389559553,2881798195775557,2943715564248539,2951224334569493,2958397742216527, + 2962117760574537,2977264272542363,2982743447396379,3009349759215772,3010012117130847,3010096874529870,3010115852485268,3012535704934837,3019483201458596,3036314602733381, + 3036837746136599,3039514679368453,3052506678397436,3059152178913030,3061095276596365,3063233426258797,3085115838323303,3093308450248420,3101208987984055,3106059275305341, + 3107559394454149,3110050933644367,3138545183162445,3140678914029638,3142109724205254,3142527081183238,3160670144726081,3160829106050702,3172902656372574,3181996104936476, + 3185563316673259,3189751274857877,3211729550313903,3213599609783958,3213659878256809,3215918533522245,3225137047012900,3231023124763183,3237856608607317,3247879729114765, + 3253596244997212,3256221808530784,3259756319203329,3260467132021662,3270706364495298,3273179837457878,3281263986182695,3296686215236784,3315584830022847,3317781471030011, + 3338120669693067,3342373672540130,3348941534532661,3359715878023917,3383970565360566,3397868551281252,3428808017724584,3449238661063482,3450439330534765,3471286956110444, + 3472931370488350,3473088266701087,3474445823605231,3477506057124755,3477541077050002,3488929919781737,3502330373943603,3504844077112105,3508329172255490,3510690524473209, + 3512324709745663,3514007754921924,3558011694836438,3559330712604565,3562145253615132,3581148885399840,3591312468185522,3597256243542091,3618160287855458,3625686898155792, + 3650194338715713,3650610878210125,3668345904483321,3668494673430762,3677776463522008,3694233081600122,3709632134619598,3725419513943971,3730099951927002,3734961839633730, + 3788380331922768,3811246425255446,3815582365292463,3829203122180282,3847292141308076,3877507310017402,3882136043994493,3887872593644033,3891280433757250,3924114035682754, + 3930635798027692,3940634174809349,3955900287161214,3957844806020309,3973236219509940,3981294421878412,3984846206013660,3993802859865565,3996764828980278,4023461371356880, + 4030830455174294,4032434581680299,4041625011713530,4041957068079946,4058955264781991,4083680454731905,4095581705542542,4112420671677445,4121292361441010,4133226631387396, + 4139004365572538,4161250756754756,4163706760683594,4170471653379067,4171134267004311,4177796537235481,4180802682160942,4203191696595400,4209464578955045,4223422057959415, + 4237541104444937,4257691911774311,4260020795088571,4301574030764989,4301922471400280,4304478206038048,4314941265701364,4320330498423583,4339739390410992,4342413486284428, + 4347136230283432,4351145740656078,4358837874704787,4362622126951624,4364582223851552,4366497646764759,4385815379876479,4406431382404050,4409339407622327,4411432076559821, + 4449849757340102,4453892102562139,4466153465045159,4479272299804907,4486938493241801,4500505590495671,4502678993390350,4508608408879950,4515778758013390,4516708986545589, + 4534650929880461,4537213242644302,4537280160911816,4544757774741059,4544796825985803,4567299173366450,4586846032456054,4607174945068672,4613242210966642,4625035792839901, + 4634756593700865,4635123568273790,4659875963875224,4670322637424975,4683035442855866,4689033135620199,4714163851388857,4714214015125119,4744848633979578,4750937375207328, + 4776805289846989,4804031931975645,4837280540915990,4840957238353452,4868526553354967,4875063418864529,4909882543513230,4912268119820614,4919123728900332,4929754602909549, + 4941072993543698,4941174020949904,4948032640819331,4955057670206957,4989823480030237,4990195550280933,5011619499820208,5013143564325843,5016786248387969,5019292677276101, + 5045230878000223,5053158166772953,5058611677018883,5083565032770599,5100847523417394,5105223137691724,5116076462386020,5117104556161083,5137323839372187,5159591155175114, + 5174145884534911,5182009696238822,5207529188216676,5229363397364492,5231081768308950,5233170527517877,5238337785560206,5259149223152958,5259347545688076,5265284989590946, + 5288711160650011,5297955243309025,5300535720011230,5311590629866593,5313248950470856,5313960220028487,5321218075331795,5334031591400346,5340934779949776,5380276587818617, + 5380604989545853,5392427172834832,5419648071490001,5436430269421440,5438720576124743,5442272167466546,5443531545450195,5462404261617760,5484761325677647 + ) +` + sctx := testKit.Session().(sessionctx.Context) + stmts, err := session.Parse(sctx, longInListQuery) + require.NoError(b, err) + require.Len(b, stmts, 1) + ret := &plannercore.PreprocessorReturn{} + err = plannercore.Preprocess(sctx, stmts[0], plannercore.WithPreprocessorReturn(ret)) + require.NoError(b, err) + ctx := context.Background() + p, _, err := plannercore.BuildLogicalPlanForTest(ctx, sctx, stmts[0], ret.InfoSchema) + require.NoError(b, err) + selection := p.(plannercore.LogicalPlan).Children()[0].(*plannercore.LogicalSelection) + tbl := selection.Children()[0].(*plannercore.DataSource).TableInfo() + require.NotNil(b, selection) + conds := make([]expression.Expression, len(selection.Conditions)) + for i, cond := range selection.Conditions { + conds[i] = expression.PushDownNot(sctx, cond) + } + cols, lengths := expression.IndexInfo2PrefixCols(tbl.Columns, selection.Schema().Columns, tbl.Indices[0]) + require.NotNil(b, cols) + + b.ResetTimer() + for i := 0; i < b.N; i++ { + _, err = ranger.DetachCondAndBuildRangeForIndex(sctx, conds, cols, lengths) + require.NoError(b, err) + } + b.StopTimer() +} diff --git a/util/ranger/detacher.go b/util/ranger/detacher.go index fc03ce7bea2eb..2ccda74e5fe19 100644 --- a/util/ranger/detacher.go +++ b/util/ranger/detacher.go @@ -283,22 +283,38 @@ func (d *rangeDetacher) detachCNFCondAndBuildRangeForIndex(conditions []expressi eqOrInCount := len(accessConds) res.EqCondCount = eqCount res.EqOrInCount = eqOrInCount - ranges, err = d.buildCNFIndexRange(tpSlice, eqOrInCount, accessConds) + ranges, err = d.buildRangeOnColsByCNFCond(tpSlice, eqOrInCount, accessConds) if err != nil { return nil, err } - - // Though ranges are built from equal/in conditions, some range may not be a single point after UnionRanges in buildCNFIndexRange. - // In order to prepare for the following appendRanges2PointRanges, we set d.mergeConsecutive to false and call buildCNFIndexRange - // again to get pointRanges, in which each range must be a single point. If we use ranges rather than pointRanges when calling - // appendRanges2PointRanges, wrong ranges would be calculated as issue https://github.com/pingcap/tidb/issues/26029 describes. - mergeConsecutive := d.mergeConsecutive - d.mergeConsecutive = false - pointRanges, err := d.buildCNFIndexRange(tpSlice, eqOrInCount, accessConds) - if err != nil { - return nil, err + // If index has prefix column and d.mergeConsecutive is true, ranges may not be point ranges anymore after UnionRanges. + // Therefore, we need to calculate pointRanges separately so that it can be used to append tail ranges in considerDNF branch. + // See https://github.com/pingcap/tidb/issues/26029 for details. + var pointRanges []*Range + if hasPrefix(d.lengths) && fixPrefixColRange(ranges, d.lengths, tpSlice) { + if d.mergeConsecutive { + pointRanges = make([]*Range, 0, len(ranges)) + for _, ran := range ranges { + pointRanges = append(pointRanges, ran.Clone()) + } + ranges, err = UnionRanges(d.sctx, ranges, d.mergeConsecutive) + if err != nil { + return nil, errors.Trace(err) + } + pointRanges, err = UnionRanges(d.sctx, pointRanges, false) + if err != nil { + return nil, errors.Trace(err) + } + } else { + ranges, err = UnionRanges(d.sctx, ranges, d.mergeConsecutive) + if err != nil { + return nil, errors.Trace(err) + } + pointRanges = ranges + } + } else { + pointRanges = ranges } - d.mergeConsecutive = mergeConsecutive res.Ranges = ranges res.AccessConds = accessConds diff --git a/util/ranger/ranger.go b/util/ranger/ranger.go index b9e83ea9a0cf9..3912027a0bfbe 100644 --- a/util/ranger/ranger.go +++ b/util/ranger/ranger.go @@ -366,17 +366,13 @@ func BuildColumnRange(conds []expression.Expression, sctx sessionctx.Context, tp return buildColumnRange(conds, sctx, tp, false, colLen) } -// buildCNFIndexRange builds the range for index where the top layer is CNF. -func (d *rangeDetacher) buildCNFIndexRange(newTp []*types.FieldType, +func (d *rangeDetacher) buildRangeOnColsByCNFCond(newTp []*types.FieldType, eqAndInCount int, accessCondition []expression.Expression) ([]*Range, error) { rb := builder{sc: d.sctx.GetSessionVars().StmtCtx} var ( ranges []*Range err error ) - for _, col := range d.cols { - newTp = append(newTp, newFieldType(col.RetType)) - } for i := 0; i < eqAndInCount; i++ { // Build ranges for equal or in access conditions. point := rb.build(accessCondition[i], collate.GetCollator(newTp[i].GetCollate())) @@ -409,6 +405,16 @@ func (d *rangeDetacher) buildCNFIndexRange(newTp []*types.FieldType, if err != nil { return nil, errors.Trace(err) } + return ranges, nil +} + +// buildCNFIndexRange builds the range for index where the top layer is CNF. +func (d *rangeDetacher) buildCNFIndexRange(newTp []*types.FieldType, + eqAndInCount int, accessCondition []expression.Expression) ([]*Range, error) { + ranges, err := d.buildRangeOnColsByCNFCond(newTp, eqAndInCount, accessCondition) + if err != nil { + return nil, err + } // Take prefix index into consideration. if hasPrefix(d.lengths) { From 7d9c68446aa526772c12ea65e4abfb91d831b517 Mon Sep 17 00:00:00 2001 From: Song Gao Date: Thu, 25 Aug 2022 14:30:22 +0800 Subject: [PATCH 4/4] executor: fix unstable TestGlobalMemoryTrackerOnCleanUp (#37372) close pingcap/tidb#36585 --- executor/insert.go | 1 + executor/memtest/mem_test.go | 10 +++++----- executor/update.go | 2 +- 3 files changed, 7 insertions(+), 6 deletions(-) diff --git a/executor/insert.go b/executor/insert.go index f533cc4155aef..7c4b84f5f1c3c 100644 --- a/executor/insert.go +++ b/executor/insert.go @@ -315,6 +315,7 @@ func (e *InsertExec) Next(ctx context.Context, req *chunk.Chunk) error { // Close implements the Executor Close interface. func (e *InsertExec) Close() error { + defer e.memTracker.ReplaceBytesUsed(0) e.ctx.GetSessionVars().CurrInsertValues = chunk.Row{} e.ctx.GetSessionVars().CurrInsertBatchExtraCols = e.ctx.GetSessionVars().CurrInsertBatchExtraCols[0:0:0] e.setMessage() diff --git a/executor/memtest/mem_test.go b/executor/memtest/mem_test.go index 5f3acb4443a7a..43cb738f07997 100644 --- a/executor/memtest/mem_test.go +++ b/executor/memtest/mem_test.go @@ -17,30 +17,30 @@ package memtest import ( "testing" - "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/testkit" "github.com/stretchr/testify/require" ) -func TestGlobalMemoryTrackerOnCleanUp(t *testing.T) { - originConsume := executor.GlobalMemoryUsageTracker.BytesConsumed() +func TestInsertUpdateTrackerOnCleanUp(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t (id int)") + originConsume := tk.Session().GetSessionVars().StmtCtx.MemTracker.BytesConsumed() // assert insert tk.MustExec("insert t (id) values (1)") tk.MustExec("insert t (id) values (2)") tk.MustExec("insert t (id) values (3)") - afterConsume := executor.GlobalMemoryUsageTracker.BytesConsumed() + afterConsume := tk.Session().GetSessionVars().StmtCtx.MemTracker.BytesConsumed() require.Equal(t, afterConsume, originConsume) + originConsume = tk.Session().GetSessionVars().StmtCtx.MemTracker.BytesConsumed() // assert update tk.MustExec("update t set id = 4 where id = 1") tk.MustExec("update t set id = 5 where id = 2") tk.MustExec("update t set id = 6 where id = 3") - afterConsume = executor.GlobalMemoryUsageTracker.BytesConsumed() + afterConsume = tk.Session().GetSessionVars().StmtCtx.MemTracker.BytesConsumed() require.Equal(t, afterConsume, originConsume) } diff --git a/executor/update.go b/executor/update.go index 196f737aa057f..e8c65d18f924e 100644 --- a/executor/update.go +++ b/executor/update.go @@ -427,6 +427,7 @@ func (e *UpdateExec) composeGeneratedColumns(rowIdx int, newRowData []types.Datu // Close implements the Executor Close interface. func (e *UpdateExec) Close() error { + defer e.memTracker.ReplaceBytesUsed(0) e.setMessage() if e.runtimeStats != nil && e.stats != nil { txn, err := e.ctx.Txn(false) @@ -434,7 +435,6 @@ func (e *UpdateExec) Close() error { txn.GetSnapshot().SetOption(kv.CollectRuntimeStats, nil) } } - defer e.memTracker.ReplaceBytesUsed(0) return e.children[0].Close() }