diff --git a/pkg/executor/BUILD.bazel b/pkg/executor/BUILD.bazel index 70f00ec079621..f631762b21423 100644 --- a/pkg/executor/BUILD.bazel +++ b/pkg/executor/BUILD.bazel @@ -478,6 +478,7 @@ go_test( "//pkg/util/set", "//pkg/util/sqlexec", "//pkg/util/sqlkiller", + "//pkg/util/stmtsummary", "//pkg/util/stmtsummary/v2:stmtsummary", "//pkg/util/stringutil", "//pkg/util/syncutil", diff --git a/pkg/executor/adapter.go b/pkg/executor/adapter.go index a1c9261acf3c2..c1e7e461055f2 100644 --- a/pkg/executor/adapter.go +++ b/pkg/executor/adapter.go @@ -1420,7 +1420,7 @@ func (a *ExecStmt) FinishExecuteStmt(txnTS uint64, err error, hasMoreResults boo } // `LowSlowQuery` and `SummaryStmt` must be called before recording `PrevStmt`. a.LogSlowQuery(txnTS, succ, hasMoreResults) - a.SummaryStmt(succ) + a.SummaryStmt(succ, execDetail) a.observeStmtFinishedForTopSQL() a.UpdatePlanCacheRuntimeInfo() if sessVars.StmtCtx.IsTiFlash.Load() { @@ -1895,7 +1895,7 @@ func getEncodedPlan(stmtCtx *stmtctx.StatementContext, genHint bool) (encodedPla } // SummaryStmt collects statements for information_schema.statements_summary -func (a *ExecStmt) SummaryStmt(succ bool) { +func (a *ExecStmt) SummaryStmt(succ bool, execDetail execdetails.ExecDetails) { sessVars := a.Ctx.GetSessionVars() var userString string if sessVars.User != nil { @@ -1940,7 +1940,6 @@ func (a *ExecStmt) SummaryStmt(succ bool) { planDigest = tmp.String() } - execDetail := stmtCtx.GetExecDetails() copTaskInfo := stmtCtx.CopTasksDetails() memMax := sessVars.MemTracker.MaxConsumed() diskMax := sessVars.DiskTracker.MaxConsumed() @@ -2020,7 +2019,17 @@ func (a *ExecStmt) SummaryStmt(succ bool) { if a.retryCount > 0 { stmtExecInfo.ExecRetryTime = costTime - sessVars.DurationParse - sessVars.DurationCompile - time.Since(a.retryStartTime) } - stmtsummaryv2.Add(stmtExecInfo) + if sessVars.CacheStmtDigestKey == nil { + sessVars.CacheStmtDigestKey = &stmtsummary.StmtDigestKey{} + } + key := sessVars.CacheStmtDigestKey + key.ResetHash() + key.SchemaName = stmtExecInfo.SchemaName + key.Digest = stmtExecInfo.Digest + key.PrevDigest = stmtExecInfo.PrevSQLDigest + key.PlanDigest = stmtExecInfo.PlanDigest + key.ResourceGroupName = stmtExecInfo.ResourceGroupName + stmtsummaryv2.Add(key, stmtExecInfo) } // GetOriginalSQL implements StmtExecLazyInfo interface. diff --git a/pkg/executor/stmtsummary_test.go b/pkg/executor/stmtsummary_test.go index 9ef4960b4798d..55fce0d1d373b 100644 --- a/pkg/executor/stmtsummary_test.go +++ b/pkg/executor/stmtsummary_test.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util" "github.com/pingcap/tidb/pkg/util/mock" + "github.com/pingcap/tidb/pkg/util/stmtsummary" stmtsummaryv2 "github.com/pingcap/tidb/pkg/util/stmtsummary/v2" "github.com/stretchr/testify/require" ) @@ -43,12 +44,16 @@ func TestStmtSummaryRetriverV2_TableStatementsSummary(t *testing.T) { stmtSummary := stmtsummaryv2.NewStmtSummary4Test(1000) defer stmtSummary.Close() - stmtSummary.Add(stmtsummaryv2.GenerateStmtExecInfo4Test("digest1")) - stmtSummary.Add(stmtsummaryv2.GenerateStmtExecInfo4Test("digest1")) - stmtSummary.Add(stmtsummaryv2.GenerateStmtExecInfo4Test("digest2")) - stmtSummary.Add(stmtsummaryv2.GenerateStmtExecInfo4Test("digest2")) - stmtSummary.Add(stmtsummaryv2.GenerateStmtExecInfo4Test("digest3")) - stmtSummary.Add(stmtsummaryv2.GenerateStmtExecInfo4Test("digest3")) + stmtSummaryAdd := func(info *stmtsummary.StmtExecInfo) { + k := genStmtSummaryByDigestKey(info) + stmtSummary.Add(k, info) + } + stmtSummaryAdd(stmtsummaryv2.GenerateStmtExecInfo4Test("digest1")) + stmtSummaryAdd(stmtsummaryv2.GenerateStmtExecInfo4Test("digest1")) + stmtSummaryAdd(stmtsummaryv2.GenerateStmtExecInfo4Test("digest2")) + stmtSummaryAdd(stmtsummaryv2.GenerateStmtExecInfo4Test("digest2")) + stmtSummaryAdd(stmtsummaryv2.GenerateStmtExecInfo4Test("digest3")) + stmtSummaryAdd(stmtsummaryv2.GenerateStmtExecInfo4Test("digest3")) retriever := stmtSummaryRetrieverV2{ stmtSummary: stmtSummary, @@ -88,12 +93,16 @@ func TestStmtSummaryRetriverV2_TableStatementsSummaryEvicted(t *testing.T) { stmtSummary := stmtsummaryv2.NewStmtSummary4Test(1) defer stmtSummary.Close() - stmtSummary.Add(stmtsummaryv2.GenerateStmtExecInfo4Test("digest1")) - stmtSummary.Add(stmtsummaryv2.GenerateStmtExecInfo4Test("digest1")) - stmtSummary.Add(stmtsummaryv2.GenerateStmtExecInfo4Test("digest2")) - stmtSummary.Add(stmtsummaryv2.GenerateStmtExecInfo4Test("digest2")) - stmtSummary.Add(stmtsummaryv2.GenerateStmtExecInfo4Test("digest3")) - stmtSummary.Add(stmtsummaryv2.GenerateStmtExecInfo4Test("digest3")) + stmtSummaryAdd := func(info *stmtsummary.StmtExecInfo) { + k := genStmtSummaryByDigestKey(info) + stmtSummary.Add(k, info) + } + stmtSummaryAdd(stmtsummaryv2.GenerateStmtExecInfo4Test("digest1")) + stmtSummaryAdd(stmtsummaryv2.GenerateStmtExecInfo4Test("digest1")) + stmtSummaryAdd(stmtsummaryv2.GenerateStmtExecInfo4Test("digest2")) + stmtSummaryAdd(stmtsummaryv2.GenerateStmtExecInfo4Test("digest2")) + stmtSummaryAdd(stmtsummaryv2.GenerateStmtExecInfo4Test("digest3")) + stmtSummaryAdd(stmtsummaryv2.GenerateStmtExecInfo4Test("digest3")) retriever := stmtSummaryRetrieverV2{ stmtSummary: stmtSummary, @@ -150,12 +159,16 @@ func TestStmtSummaryRetriverV2_TableStatementsSummaryHistory(t *testing.T) { stmtSummary := stmtsummaryv2.NewStmtSummary4Test(2) defer stmtSummary.Close() - stmtSummary.Add(stmtsummaryv2.GenerateStmtExecInfo4Test("digest1")) - stmtSummary.Add(stmtsummaryv2.GenerateStmtExecInfo4Test("digest1")) - stmtSummary.Add(stmtsummaryv2.GenerateStmtExecInfo4Test("digest2")) - stmtSummary.Add(stmtsummaryv2.GenerateStmtExecInfo4Test("digest2")) - stmtSummary.Add(stmtsummaryv2.GenerateStmtExecInfo4Test("digest3")) - stmtSummary.Add(stmtsummaryv2.GenerateStmtExecInfo4Test("digest3")) + stmtSummaryAdd := func(info *stmtsummary.StmtExecInfo) { + k := genStmtSummaryByDigestKey(info) + stmtSummary.Add(k, info) + } + stmtSummaryAdd(stmtsummaryv2.GenerateStmtExecInfo4Test("digest1")) + stmtSummaryAdd(stmtsummaryv2.GenerateStmtExecInfo4Test("digest1")) + stmtSummaryAdd(stmtsummaryv2.GenerateStmtExecInfo4Test("digest2")) + stmtSummaryAdd(stmtsummaryv2.GenerateStmtExecInfo4Test("digest2")) + stmtSummaryAdd(stmtsummaryv2.GenerateStmtExecInfo4Test("digest3")) + stmtSummaryAdd(stmtsummaryv2.GenerateStmtExecInfo4Test("digest3")) data := infoschema.NewData() infoSchemaBuilder := infoschema.NewBuilder(nil, nil, data, variable.SchemaCacheSize.Load() > 0) @@ -191,3 +204,13 @@ func TestStmtSummaryRetriverV2_TableStatementsSummaryHistory(t *testing.T) { } require.Len(t, results, 7) } + +func genStmtSummaryByDigestKey(info *stmtsummary.StmtExecInfo) *stmtsummary.StmtDigestKey { + return &stmtsummary.StmtDigestKey{ + SchemaName: info.SchemaName, + Digest: info.Digest, + PrevDigest: info.PrevSQLDigest, + PlanDigest: info.PlanDigest, + ResourceGroupName: info.ResourceGroupName, + } +} diff --git a/pkg/sessionctx/variable/session.go b/pkg/sessionctx/variable/session.go index d28ff3da2ef5b..68c5d7b99242e 100644 --- a/pkg/sessionctx/variable/session.go +++ b/pkg/sessionctx/variable/session.go @@ -1714,7 +1714,8 @@ type SessionVars struct { ScatterRegion string // CacheStmtExecInfo is a cache for the statement execution information, used to reduce the overhead of memory allocation. - CacheStmtExecInfo *stmtsummary.StmtExecInfo + CacheStmtExecInfo *stmtsummary.StmtExecInfo + CacheStmtDigestKey *stmtsummary.StmtDigestKey } // GetSessionVars implements the `SessionVarsProvider` interface. diff --git a/pkg/util/stmtsummary/evicted.go b/pkg/util/stmtsummary/evicted.go index ec84ad802751b..afcbc8a75cd9b 100644 --- a/pkg/util/stmtsummary/evicted.go +++ b/pkg/util/stmtsummary/evicted.go @@ -70,7 +70,7 @@ func newStmtSummaryByDigestEvictedElement(beginTime int64, endTime int64) *stmtS } // AddEvicted is used add an evicted record to stmtSummaryByDigestEvicted -func (ssbde *stmtSummaryByDigestEvicted) AddEvicted(evictedKey *stmtSummaryByDigestKey, evictedValue *stmtSummaryByDigest, historySize int) { +func (ssbde *stmtSummaryByDigestEvicted) AddEvicted(evictedKey *StmtDigestKey, evictedValue *stmtSummaryByDigest, historySize int) { if evictedValue == nil { return } @@ -152,7 +152,7 @@ func (ssbde *stmtSummaryByDigestEvicted) Clear() { } // add an evicted record to stmtSummaryByDigestEvictedElement -func (seElement *stmtSummaryByDigestEvictedElement) addEvicted(digestKey *stmtSummaryByDigestKey, digestValue *stmtSummaryByDigestElement) { +func (seElement *stmtSummaryByDigestEvictedElement) addEvicted(digestKey *StmtDigestKey, digestValue *stmtSummaryByDigestElement) { if digestKey != nil { seElement.count++ addInfo(seElement.otherSummary, digestValue) @@ -169,7 +169,7 @@ const ( // if matches, it will add the digest and return enum match // if digest too old, it will return enum tooOld and do nothing // if digest too young, it will return enum tooYoung and do nothing -func (seElement *stmtSummaryByDigestEvictedElement) matchAndAdd(digestKey *stmtSummaryByDigestKey, digestValue *stmtSummaryByDigestElement) (statement int) { +func (seElement *stmtSummaryByDigestEvictedElement) matchAndAdd(digestKey *StmtDigestKey, digestValue *stmtSummaryByDigestElement) (statement int) { if seElement == nil || digestValue == nil { return isTooYoung } diff --git a/pkg/util/stmtsummary/evicted_test.go b/pkg/util/stmtsummary/evicted_test.go index ff93702f0a0d3..0ef26d66af705 100644 --- a/pkg/util/stmtsummary/evicted_test.go +++ b/pkg/util/stmtsummary/evicted_test.go @@ -49,10 +49,10 @@ func newInduceSsbde(beginTime int64, endTime int64) *stmtSummaryByDigestElement return newSsbde } -// generate new stmtSummaryByDigestKey and stmtSummaryByDigest -func generateStmtSummaryByDigestKeyValue(schema string, beginTime int64, endTime int64) (*stmtSummaryByDigestKey, *stmtSummaryByDigest) { - key := &stmtSummaryByDigestKey{ - schemaName: schema, +// generate new StmtDigestKey and stmtSummaryByDigest +func generateStmtSummaryByDigestKeyValue(schema string, beginTime int64, endTime int64) (*StmtDigestKey, *stmtSummaryByDigest) { + key := &StmtDigestKey{ + SchemaName: schema, } value := newInduceSsbd(beginTime, endTime) return key, value @@ -77,10 +77,10 @@ func TestMapToEvictedCountDatum(t *testing.T) { sei1 := generateAnyExecInfo() sei0.SchemaName = "I'll occupy this cache! :(" - ssMap.AddStatement(sei0) + ssMap.AddStatement(genStmtSummaryByDigestKey(sei0), sei0) n := ssMap.beginTimeForCurInterval sei1.SchemaName = "sorry, it's mine now. =)" - ssMap.AddStatement(sei1) + ssMap.AddStatement(genStmtSummaryByDigestKey(sei1), sei1) expectedEvictedCount := []any{ types.NewTime(types.FromGoTime(time.Unix(n, 0)), mysql.TypeTimestamp, types.DefaultFsp), @@ -105,7 +105,8 @@ func TestMapToEvictedCountDatum(t *testing.T) { ssMap.beginTimeForCurInterval = now + interval // insert one statement per interval. for range 50 { - ssMap.AddStatement(generateAnyExecInfo()) + info := generateAnyExecInfo() + ssMap.AddStatement(genStmtSummaryByDigestKey(info), info) ssMap.beginTimeForCurInterval += interval * 2 } require.Equal(t, 1, ssMap.summaryMap.Size()) @@ -120,14 +121,14 @@ func TestMapToEvictedCountDatum(t *testing.T) { ssMap.beginTimeForCurInterval += interval * 2 banditSei := generateAnyExecInfo() banditSei.SchemaName = "Kick you out >:(" - ssMap.AddStatement(banditSei) + ssMap.AddStatement(genStmtSummaryByDigestKey(banditSei), banditSei) evictedCountDatums := ssMap.ToEvictedCountDatum() require.Equal(t, 25, len(evictedCountDatums)) // update begin time banditSei.SchemaName = "Yet another kicker" - ssMap.AddStatement(banditSei) + ssMap.AddStatement(genStmtSummaryByDigestKey(banditSei), banditSei) evictedCountDatums = ssMap.ToEvictedCountDatum() // test young digest @@ -191,7 +192,7 @@ func TestSimpleStmtSummaryByDigestEvicted(t *testing.T) { ssbde.AddEvicted(evictedKey, evictedValue, 3) require.Equal(t, "{begin: 8, end: 9, count: 1}, {begin: 5, end: 6, count: 1}, {begin: 2, end: 3, count: 1}", getAllEvicted(ssbde)) - evictedKey = &stmtSummaryByDigestKey{schemaName: "b"} + evictedKey = &StmtDigestKey{SchemaName: "b"} ssbde.AddEvicted(evictedKey, evictedValue, 4) require.Equal(t, "{begin: 8, end: 9, count: 2}, {begin: 5, end: 6, count: 2}, {begin: 2, end: 3, count: 2}, {begin: 1, end: 2, count: 1}", getAllEvicted(ssbde)) @@ -273,14 +274,15 @@ func TestEvictedCountDetailed(t *testing.T) { digest := val.(*stmtSummaryByDigest) require.Equal(t, i, digest.history.Len()) } - ssMap.AddStatement(generateAnyExecInfo()) + info := generateAnyExecInfo() + ssMap.AddStatement(genStmtSummaryByDigestKey(info), info) ssMap.beginTimeForCurInterval += interval } ssMap.beginTimeForCurInterval -= interval banditSei := generateAnyExecInfo() banditSei.SchemaName = "kick you out >:(" - ssMap.AddStatement(banditSei) + ssMap.AddStatement(genStmtSummaryByDigestKey(banditSei), banditSei) evictedCountDatums := ssMap.ToEvictedCountDatum() n := ssMap.beginTimeForCurInterval for _, evictedCountDatum := range evictedCountDatums { @@ -301,13 +303,13 @@ func TestEvictedCountDetailed(t *testing.T) { types.NewTime(types.FromGoTime(time.Unix(n+60, 0)), mysql.TypeTimestamp, types.DefaultFsp), int64(2), } - ssMap.AddStatement(banditSei) + ssMap.AddStatement(genStmtSummaryByDigestKey(banditSei), banditSei) evictedCountDatums = ssMap.ToEvictedCountDatum() match(t, evictedCountDatums[0], expectedDatum...) ssMap.Clear() other := ssMap.other // test poisoning with empty-history digestValue - other.AddEvicted(new(stmtSummaryByDigestKey), new(stmtSummaryByDigest), 100) + other.AddEvicted(new(StmtDigestKey), new(stmtSummaryByDigest), 100) require.Equal(t, 0, other.history.Len()) } diff --git a/pkg/util/stmtsummary/statement_summary.go b/pkg/util/stmtsummary/statement_summary.go index 297eefa33f345..7edd72251a982 100644 --- a/pkg/util/stmtsummary/statement_summary.go +++ b/pkg/util/stmtsummary/statement_summary.go @@ -40,17 +40,17 @@ import ( atomic2 "go.uber.org/atomic" ) -// stmtSummaryByDigestKey defines key for stmtSummaryByDigestMap.summaryMap. -type stmtSummaryByDigestKey struct { +// StmtDigestKey defines key for stmtSummaryByDigestMap.summaryMap. +type StmtDigestKey struct { // Same statements may appear in different schema, but they refer to different tables. - schemaName string - digest string + SchemaName string + Digest string // The digest of the previous statement. - prevDigest string + PrevDigest string // The digest of the plan of this SQL. - planDigest string - // `resourceGroupName` is the resource group's name of this statement is bind to. - resourceGroupName string + PlanDigest string + // `ResourceGroupName` is the resource group's name of this statement is bind to. + ResourceGroupName string // `hash` is the hash value of this object. hash []byte } @@ -58,18 +58,40 @@ type stmtSummaryByDigestKey struct { // Hash implements SimpleLRUCache.Key. // Only when current SQL is `commit` do we record `prevSQL`. Otherwise, `prevSQL` is empty. // `prevSQL` is included in the key To distinguish different transactions. -func (key *stmtSummaryByDigestKey) Hash() []byte { +func (key *StmtDigestKey) Hash() []byte { if len(key.hash) == 0 { - key.hash = make([]byte, 0, len(key.schemaName)+len(key.digest)+len(key.prevDigest)+len(key.planDigest)+len(key.resourceGroupName)) - key.hash = append(key.hash, hack.Slice(key.digest)...) - key.hash = append(key.hash, hack.Slice(key.schemaName)...) - key.hash = append(key.hash, hack.Slice(key.prevDigest)...) - key.hash = append(key.hash, hack.Slice(key.planDigest)...) - key.hash = append(key.hash, hack.Slice(key.resourceGroupName)...) + length := len(key.SchemaName) + len(key.Digest) + len(key.PrevDigest) + len(key.PlanDigest) + len(key.ResourceGroupName) + if cap(key.hash) < length { + key.hash = make([]byte, 0, length) + } + key.hash = append(key.hash, hack.Slice(key.Digest)...) + key.hash = append(key.hash, hack.Slice(key.SchemaName)...) + key.hash = append(key.hash, hack.Slice(key.PrevDigest)...) + key.hash = append(key.hash, hack.Slice(key.PlanDigest)...) + key.hash = append(key.hash, hack.Slice(key.ResourceGroupName)...) } return key.hash } +// ResetHash resets the hash value. +func (key *StmtDigestKey) ResetHash() { + key.hash = key.hash[:0] +} + +// Clone returns a new cloned StmtDigestKey. +func (key *StmtDigestKey) Clone() *StmtDigestKey { + k := &StmtDigestKey{ + SchemaName: key.SchemaName, + Digest: key.Digest, + PrevDigest: key.PrevDigest, + PlanDigest: key.PlanDigest, + ResourceGroupName: key.ResourceGroupName, + hash: make([]byte, 0, len(key.hash)), + } + k.hash = append(k.hash, key.hash...) + return k +} + // stmtSummaryByDigestMap is a LRU cache that stores statement summaries. type stmtSummaryByDigestMap struct { // It's rare to read concurrently, so RWMutex is not needed. @@ -308,13 +330,13 @@ func newStmtSummaryByDigestMap() *stmtSummaryByDigestMap { } newSsMap.summaryMap.SetOnEvict(func(k kvcache.Key, v kvcache.Value) { historySize := newSsMap.historySize() - newSsMap.other.AddEvicted(k.(*stmtSummaryByDigestKey), v.(*stmtSummaryByDigest), historySize) + newSsMap.other.AddEvicted(k.(*StmtDigestKey), v.(*stmtSummaryByDigest), historySize) }) return newSsMap } // AddStatement adds a statement to StmtSummaryByDigestMap. -func (ssMap *stmtSummaryByDigestMap) AddStatement(sei *StmtExecInfo) { +func (ssMap *stmtSummaryByDigestMap) AddStatement(key *StmtDigestKey, sei *StmtExecInfo) { // All times are counted in seconds. now := time.Now().Unix() @@ -335,13 +357,6 @@ func (ssMap *stmtSummaryByDigestMap) AddStatement(sei *StmtExecInfo) { historySize = ssMap.historySize() } - key := &stmtSummaryByDigestKey{ - schemaName: sei.SchemaName, - digest: sei.Digest, - prevDigest: sei.PrevSQLDigest, - planDigest: sei.PlanDigest, - resourceGroupName: sei.ResourceGroupName, - } // Calculate hash value in advance, to reduce the time holding the lock. key.Hash() @@ -370,7 +385,7 @@ func (ssMap *stmtSummaryByDigestMap) AddStatement(sei *StmtExecInfo) { if !ok { // Lazy initialize it to release ssMap.mutex ASAP. summary = new(stmtSummaryByDigest) - ssMap.summaryMap.Put(key, summary) + ssMap.summaryMap.Put(key.Clone(), summary) } else { summary = value.(*stmtSummaryByDigest) } diff --git a/pkg/util/stmtsummary/statement_summary_test.go b/pkg/util/stmtsummary/statement_summary_test.go index f07ce4ad86c8a..5be78797b63fd 100644 --- a/pkg/util/stmtsummary/statement_summary_test.go +++ b/pkg/util/stmtsummary/statement_summary_test.go @@ -75,11 +75,11 @@ func TestAddStatement(t *testing.T) { // first statement stmtExecInfo1 := generateAnyExecInfo() stmtExecInfo1.ExecDetail.CommitDetail.Mu.PrewriteBackoffTypes = make([]string, 0) - key := &stmtSummaryByDigestKey{ - schemaName: stmtExecInfo1.SchemaName, - digest: stmtExecInfo1.Digest, - planDigest: stmtExecInfo1.PlanDigest, - resourceGroupName: stmtExecInfo1.ResourceGroupName, + key := &StmtDigestKey{ + SchemaName: stmtExecInfo1.SchemaName, + Digest: stmtExecInfo1.Digest, + PlanDigest: stmtExecInfo1.PlanDigest, + ResourceGroupName: stmtExecInfo1.ResourceGroupName, } samplePlan, _, _ := stmtExecInfo1.LazyInfo.GetEncodedPlan() stmtExecInfo1.ExecDetail.CommitDetail.Mu.Lock() @@ -166,7 +166,7 @@ func TestAddStatement(t *testing.T) { tableNames: "db1.tb1,db2.tb2", history: history, } - ssMap.AddStatement(stmtExecInfo1) + ssMap.AddStatement(genStmtSummaryByDigestKey(stmtExecInfo1), stmtExecInfo1) summary, ok := ssMap.summaryMap.Get(key) require.True(t, ok) require.True(t, matchStmtSummaryByDigest(summary.(*stmtSummaryByDigest), &expectedSummary)) @@ -318,7 +318,7 @@ func TestAddStatement(t *testing.T) { expectedSummaryElement.SumRUWaitDuration += stmtExecInfo2.RUDetail.RUWaitDuration() expectedSummaryElement.MaxRUWaitDuration = stmtExecInfo2.RUDetail.RUWaitDuration() - ssMap.AddStatement(stmtExecInfo2) + ssMap.AddStatement(genStmtSummaryByDigestKey(stmtExecInfo2), stmtExecInfo2) summary, ok = ssMap.summaryMap.Get(key) require.True(t, ok) require.True(t, matchStmtSummaryByDigest(summary.(*stmtSummaryByDigest), &expectedSummary)) @@ -445,7 +445,7 @@ func TestAddStatement(t *testing.T) { expectedSummaryElement.SumWRU += stmtExecInfo3.RUDetail.WRU() expectedSummaryElement.SumRUWaitDuration += stmtExecInfo3.RUDetail.RUWaitDuration() - ssMap.AddStatement(stmtExecInfo3) + ssMap.AddStatement(genStmtSummaryByDigestKey(stmtExecInfo3), stmtExecInfo3) summary, ok = ssMap.summaryMap.Get(key) require.True(t, ok) require.True(t, matchStmtSummaryByDigest(summary.(*stmtSummaryByDigest), &expectedSummary)) @@ -454,13 +454,13 @@ func TestAddStatement(t *testing.T) { stmtExecInfo4 := stmtExecInfo1 stmtExecInfo4.SchemaName = "schema2" stmtExecInfo4.ExecDetail.CommitDetail = nil - key = &stmtSummaryByDigestKey{ - schemaName: stmtExecInfo4.SchemaName, - digest: stmtExecInfo4.Digest, - planDigest: stmtExecInfo4.PlanDigest, - resourceGroupName: stmtExecInfo4.ResourceGroupName, + key = &StmtDigestKey{ + SchemaName: stmtExecInfo4.SchemaName, + Digest: stmtExecInfo4.Digest, + PlanDigest: stmtExecInfo4.PlanDigest, + ResourceGroupName: stmtExecInfo4.ResourceGroupName, } - ssMap.AddStatement(stmtExecInfo4) + ssMap.AddStatement(genStmtSummaryByDigestKey(stmtExecInfo4), stmtExecInfo4) require.Equal(t, 2, ssMap.summaryMap.Size()) _, ok = ssMap.summaryMap.Get(key) require.True(t, ok) @@ -468,13 +468,13 @@ func TestAddStatement(t *testing.T) { // Fifth statement has a different digest. stmtExecInfo5 := stmtExecInfo1 stmtExecInfo5.Digest = "digest2" - key = &stmtSummaryByDigestKey{ - schemaName: stmtExecInfo5.SchemaName, - digest: stmtExecInfo5.Digest, - planDigest: stmtExecInfo4.PlanDigest, - resourceGroupName: stmtExecInfo5.ResourceGroupName, + key = &StmtDigestKey{ + SchemaName: stmtExecInfo5.SchemaName, + Digest: stmtExecInfo5.Digest, + PlanDigest: stmtExecInfo4.PlanDigest, + ResourceGroupName: stmtExecInfo5.ResourceGroupName, } - ssMap.AddStatement(stmtExecInfo5) + ssMap.AddStatement(genStmtSummaryByDigestKey(stmtExecInfo5), stmtExecInfo5) require.Equal(t, 3, ssMap.summaryMap.Size()) _, ok = ssMap.summaryMap.Get(key) require.True(t, ok) @@ -482,13 +482,13 @@ func TestAddStatement(t *testing.T) { // Sixth statement has a different plan digest. stmtExecInfo6 := stmtExecInfo1 stmtExecInfo6.PlanDigest = "plan_digest2" - key = &stmtSummaryByDigestKey{ - schemaName: stmtExecInfo6.SchemaName, - digest: stmtExecInfo6.Digest, - planDigest: stmtExecInfo6.PlanDigest, - resourceGroupName: stmtExecInfo6.ResourceGroupName, + key = &StmtDigestKey{ + SchemaName: stmtExecInfo6.SchemaName, + Digest: stmtExecInfo6.Digest, + PlanDigest: stmtExecInfo6.PlanDigest, + ResourceGroupName: stmtExecInfo6.ResourceGroupName, } - ssMap.AddStatement(stmtExecInfo6) + ssMap.AddStatement(genStmtSummaryByDigestKey(stmtExecInfo6), stmtExecInfo6) require.Equal(t, 4, ssMap.summaryMap.Size()) _, ok = ssMap.summaryMap.Get(key) require.True(t, ok) @@ -507,18 +507,18 @@ func TestAddStatement(t *testing.T) { binPlan: "", planDigest: "", } - key = &stmtSummaryByDigestKey{ - schemaName: stmtExecInfo7.SchemaName, - digest: stmtExecInfo7.Digest, - planDigest: stmtExecInfo7.PlanDigest, - resourceGroupName: stmtExecInfo7.ResourceGroupName, + key = &StmtDigestKey{ + SchemaName: stmtExecInfo7.SchemaName, + Digest: stmtExecInfo7.Digest, + PlanDigest: stmtExecInfo7.PlanDigest, + ResourceGroupName: stmtExecInfo7.ResourceGroupName, } - ssMap.AddStatement(stmtExecInfo7) + ssMap.AddStatement(genStmtSummaryByDigestKey(stmtExecInfo7), stmtExecInfo7) require.Equal(t, 5, ssMap.summaryMap.Size()) v, ok := ssMap.summaryMap.Get(key) require.True(t, ok) stmt := v.(*stmtSummaryByDigest) - require.Equal(t, key.digest, stmt.digest) + require.Equal(t, key.Digest, stmt.digest) e := stmt.history.Back() ssElement := e.Value.(*stmtSummaryByDigestElement) require.Equal(t, plancodec.PlanDiscardedEncoded, ssElement.samplePlan) @@ -735,6 +735,16 @@ func generateAnyExecInfo() *StmtExecInfo { return stmtExecInfo } +func genStmtSummaryByDigestKey(info *StmtExecInfo) *StmtDigestKey { + return &StmtDigestKey{ + SchemaName: info.SchemaName, + Digest: info.Digest, + PrevDigest: info.PrevSQLDigest, + PlanDigest: info.PlanDigest, + ResourceGroupName: info.ResourceGroupName, + } +} + type mockLazyInfo struct { originalSQL string plan string @@ -883,7 +893,7 @@ func TestToDatum(t *testing.T) { ssMap.beginTimeForCurInterval = now + 60 stmtExecInfo1 := generateAnyExecInfo() - ssMap.AddStatement(stmtExecInfo1) + ssMap.AddStatement(genStmtSummaryByDigestKey(stmtExecInfo1), stmtExecInfo1) reader := newStmtSummaryReaderForTest(ssMap) datums := reader.GetStmtSummaryCurrentRows() require.Equal(t, 1, len(datums)) @@ -938,7 +948,7 @@ func TestToDatum(t *testing.T) { require.NoError(t, err) stmtExecInfo2 := stmtExecInfo1 stmtExecInfo2.Digest = "bandit sei" - ssMap.AddStatement(stmtExecInfo2) + ssMap.AddStatement(genStmtSummaryByDigestKey(stmtExecInfo2), stmtExecInfo2) require.Equal(t, 1, ssMap.summaryMap.Size()) datums = reader.GetStmtSummaryCurrentRows() expectedEvictedDatum := []any{n, e, "", "", "", "", @@ -996,7 +1006,7 @@ func TestAddStatementParallel(t *testing.T) { // Add 32 times with different digest. for i := range loops { stmtExecInfo1.Digest = fmt.Sprintf("digest%d", i) - ssMap.AddStatement(stmtExecInfo1) + ssMap.AddStatement(genStmtSummaryByDigestKey(stmtExecInfo1), stmtExecInfo1) } // There would be 32 summaries. @@ -1035,7 +1045,7 @@ func TestMaxStmtCount(t *testing.T) { loops := 100 for i := range loops { stmtExecInfo1.Digest = fmt.Sprintf("digest%d", i) - ssMap.AddStatement(stmtExecInfo1) + ssMap.AddStatement(genStmtSummaryByDigestKey(stmtExecInfo1), stmtExecInfo1) } // Summary count should be MaxStmtCount. @@ -1044,11 +1054,11 @@ func TestMaxStmtCount(t *testing.T) { // LRU cache should work. for i := loops - 10; i < loops; i++ { - key := &stmtSummaryByDigestKey{ - schemaName: stmtExecInfo1.SchemaName, - digest: fmt.Sprintf("digest%d", i), - planDigest: stmtExecInfo1.PlanDigest, - resourceGroupName: stmtExecInfo1.ResourceGroupName, + key := &StmtDigestKey{ + SchemaName: stmtExecInfo1.SchemaName, + Digest: fmt.Sprintf("digest%d", i), + PlanDigest: stmtExecInfo1.PlanDigest, + ResourceGroupName: stmtExecInfo1.ResourceGroupName, } key.Hash() _, ok := sm.Get(key) @@ -1059,7 +1069,7 @@ func TestMaxStmtCount(t *testing.T) { require.Nil(t, ssMap.SetMaxStmtCount(50)) for i := range loops { stmtExecInfo1.Digest = fmt.Sprintf("digest%d", i) - ssMap.AddStatement(stmtExecInfo1) + ssMap.AddStatement(genStmtSummaryByDigestKey(stmtExecInfo1), stmtExecInfo1) } require.Equal(t, 50, sm.Size()) @@ -1067,7 +1077,7 @@ func TestMaxStmtCount(t *testing.T) { require.Nil(t, ssMap.SetMaxStmtCount(10)) for i := range loops { stmtExecInfo1.Digest = fmt.Sprintf("digest%d", i) - ssMap.AddStatement(stmtExecInfo1) + ssMap.AddStatement(genStmtSummaryByDigestKey(stmtExecInfo1), stmtExecInfo1) } require.Equal(t, 10, sm.Size()) } @@ -1090,14 +1100,14 @@ func TestMaxSQLLength(t *testing.T) { stmtExecInfo1 := generateAnyExecInfo() stmtExecInfo1.LazyInfo.(*mockLazyInfo).originalSQL = str stmtExecInfo1.NormalizedSQL = str - ssMap.AddStatement(stmtExecInfo1) - - key := &stmtSummaryByDigestKey{ - schemaName: stmtExecInfo1.SchemaName, - digest: stmtExecInfo1.Digest, - planDigest: stmtExecInfo1.PlanDigest, - prevDigest: stmtExecInfo1.PrevSQLDigest, - resourceGroupName: stmtExecInfo1.ResourceGroupName, + ssMap.AddStatement(genStmtSummaryByDigestKey(stmtExecInfo1), stmtExecInfo1) + + key := &StmtDigestKey{ + SchemaName: stmtExecInfo1.SchemaName, + Digest: stmtExecInfo1.Digest, + PlanDigest: stmtExecInfo1.PlanDigest, + PrevDigest: stmtExecInfo1.PrevSQLDigest, + ResourceGroupName: stmtExecInfo1.ResourceGroupName, } value, ok := ssMap.summaryMap.Get(key) require.True(t, ok) @@ -1133,7 +1143,7 @@ func TestSetMaxStmtCountParallel(t *testing.T) { // Add 32 times with different digest. for i := range loops { stmtExecInfo1.Digest = fmt.Sprintf("digest%d", i) - ssMap.AddStatement(stmtExecInfo1) + ssMap.AddStatement(genStmtSummaryByDigestKey(stmtExecInfo1), stmtExecInfo1) } } for range threads { @@ -1173,7 +1183,7 @@ func TestDisableStmtSummary(t *testing.T) { ssMap.beginTimeForCurInterval = now + 60 stmtExecInfo1 := generateAnyExecInfo() - ssMap.AddStatement(stmtExecInfo1) + ssMap.AddStatement(genStmtSummaryByDigestKey(stmtExecInfo1), stmtExecInfo1) reader := newStmtSummaryReaderForTest(ssMap) datums := reader.GetStmtSummaryCurrentRows() require.Len(t, datums, 0) @@ -1181,7 +1191,7 @@ func TestDisableStmtSummary(t *testing.T) { err = ssMap.SetEnabled(true) require.NoError(t, err) - ssMap.AddStatement(stmtExecInfo1) + ssMap.AddStatement(genStmtSummaryByDigestKey(stmtExecInfo1), stmtExecInfo1) datums = reader.GetStmtSummaryCurrentRows() require.Equal(t, 1, len(datums)) @@ -1191,7 +1201,7 @@ func TestDisableStmtSummary(t *testing.T) { stmtExecInfo2.LazyInfo.(*mockLazyInfo).originalSQL = "original_sql2" stmtExecInfo2.NormalizedSQL = "normalized_sql2" stmtExecInfo2.Digest = "digest2" - ssMap.AddStatement(stmtExecInfo2) + ssMap.AddStatement(genStmtSummaryByDigestKey(stmtExecInfo2), stmtExecInfo2) datums = reader.GetStmtSummaryCurrentRows() require.Equal(t, 2, len(datums)) @@ -1199,7 +1209,7 @@ func TestDisableStmtSummary(t *testing.T) { err = ssMap.SetEnabled(false) require.NoError(t, err) ssMap.beginTimeForCurInterval = now + 60 - ssMap.AddStatement(stmtExecInfo2) + ssMap.AddStatement(genStmtSummaryByDigestKey(stmtExecInfo2), stmtExecInfo2) datums = reader.GetStmtSummaryCurrentRows() require.Len(t, datums, 0) @@ -1211,7 +1221,7 @@ func TestDisableStmtSummary(t *testing.T) { require.NoError(t, err) ssMap.beginTimeForCurInterval = now + 60 - ssMap.AddStatement(stmtExecInfo1) + ssMap.AddStatement(genStmtSummaryByDigestKey(stmtExecInfo1), stmtExecInfo1) datums = reader.GetStmtSummaryCurrentRows() require.Equal(t, 1, len(datums)) @@ -1239,7 +1249,7 @@ func TestEnableSummaryParallel(t *testing.T) { // Sometimes enable it and sometimes disable it. err := ssMap.SetEnabled(i%2 == 0) require.NoError(t, err) - ssMap.AddStatement(stmtExecInfo1) + ssMap.AddStatement(genStmtSummaryByDigestKey(stmtExecInfo1), stmtExecInfo1) // Try to read it. reader.GetStmtSummaryHistoryRows() } @@ -1265,18 +1275,18 @@ func TestGetMoreThanCntBindableStmt(t *testing.T) { stmtExecInfo1.LazyInfo.(*mockLazyInfo).originalSQL = "insert 1" stmtExecInfo1.NormalizedSQL = "insert ?" stmtExecInfo1.StmtCtx.StmtType = "Insert" - ssMap.AddStatement(stmtExecInfo1) + ssMap.AddStatement(genStmtSummaryByDigestKey(stmtExecInfo1), stmtExecInfo1) stmts := ssMap.GetMoreThanCntBindableStmt(1) require.Equal(t, 0, len(stmts)) stmtExecInfo1.NormalizedSQL = "select ?" stmtExecInfo1.Digest = "digest1" stmtExecInfo1.StmtCtx.StmtType = "Select" - ssMap.AddStatement(stmtExecInfo1) + ssMap.AddStatement(genStmtSummaryByDigestKey(stmtExecInfo1), stmtExecInfo1) stmts = ssMap.GetMoreThanCntBindableStmt(1) require.Equal(t, 0, len(stmts)) - ssMap.AddStatement(stmtExecInfo1) + ssMap.AddStatement(genStmtSummaryByDigestKey(stmtExecInfo1), stmtExecInfo1) stmts = ssMap.GetMoreThanCntBindableStmt(1) require.Equal(t, 1, len(stmts)) } @@ -1301,13 +1311,13 @@ func TestRefreshCurrentSummary(t *testing.T) { ssMap.beginTimeForCurInterval = now + 10 stmtExecInfo1 := generateAnyExecInfo() - key := &stmtSummaryByDigestKey{ - schemaName: stmtExecInfo1.SchemaName, - digest: stmtExecInfo1.Digest, - planDigest: stmtExecInfo1.PlanDigest, - resourceGroupName: stmtExecInfo1.ResourceGroupName, + key := &StmtDigestKey{ + SchemaName: stmtExecInfo1.SchemaName, + Digest: stmtExecInfo1.Digest, + PlanDigest: stmtExecInfo1.PlanDigest, + ResourceGroupName: stmtExecInfo1.ResourceGroupName, } - ssMap.AddStatement(stmtExecInfo1) + ssMap.AddStatement(genStmtSummaryByDigestKey(stmtExecInfo1), stmtExecInfo1) require.Equal(t, 1, ssMap.summaryMap.Size()) value, ok := ssMap.summaryMap.Get(key) require.True(t, ok) @@ -1317,7 +1327,7 @@ func TestRefreshCurrentSummary(t *testing.T) { ssMap.beginTimeForCurInterval = now - 1900 ssElement.beginTime = now - 1900 - ssMap.AddStatement(stmtExecInfo1) + ssMap.AddStatement(genStmtSummaryByDigestKey(stmtExecInfo1), stmtExecInfo1) require.Equal(t, 1, ssMap.summaryMap.Size()) value, ok = ssMap.summaryMap.Get(key) require.True(t, ok) @@ -1330,7 +1340,7 @@ func TestRefreshCurrentSummary(t *testing.T) { require.NoError(t, err) ssMap.beginTimeForCurInterval = now - 20 ssElement.beginTime = now - 20 - ssMap.AddStatement(stmtExecInfo1) + ssMap.AddStatement(genStmtSummaryByDigestKey(stmtExecInfo1), stmtExecInfo1) require.Equal(t, 3, value.(*stmtSummaryByDigest).history.Len()) } @@ -1352,15 +1362,15 @@ func TestSummaryHistory(t *testing.T) { }() stmtExecInfo1 := generateAnyExecInfo() - key := &stmtSummaryByDigestKey{ - schemaName: stmtExecInfo1.SchemaName, - digest: stmtExecInfo1.Digest, - planDigest: stmtExecInfo1.PlanDigest, - resourceGroupName: stmtExecInfo1.ResourceGroupName, + key := &StmtDigestKey{ + SchemaName: stmtExecInfo1.SchemaName, + Digest: stmtExecInfo1.Digest, + PlanDigest: stmtExecInfo1.PlanDigest, + ResourceGroupName: stmtExecInfo1.ResourceGroupName, } for i := range 11 { ssMap.beginTimeForCurInterval = now + int64(i+1)*10 - ssMap.AddStatement(stmtExecInfo1) + ssMap.AddStatement(genStmtSummaryByDigestKey(stmtExecInfo1), stmtExecInfo1) require.Equal(t, 1, ssMap.summaryMap.Size()) value, ok := ssMap.summaryMap.Get(key) require.True(t, ok) @@ -1398,14 +1408,14 @@ func TestSummaryHistory(t *testing.T) { // insert first digest for i := range 6 { ssMap.beginTimeForCurInterval = now + int64(i)*10 - ssMap.AddStatement(stmtExecInfo1) + ssMap.AddStatement(genStmtSummaryByDigestKey(stmtExecInfo1), stmtExecInfo1) require.Equal(t, 1, ssMap.summaryMap.Size()) require.Equal(t, 0, ssMap.other.history.Len()) } // insert another digest to evict it stmtExecInfo2 := stmtExecInfo1 stmtExecInfo2.Digest = "bandit digest" - ssMap.AddStatement(stmtExecInfo2) + ssMap.AddStatement(genStmtSummaryByDigestKey(stmtExecInfo2), stmtExecInfo2) require.Equal(t, 1, ssMap.summaryMap.Size()) // length of `other` should not longer than historySize. require.Equal(t, 5, ssMap.other.history.Len()) @@ -1424,28 +1434,28 @@ func TestPrevSQL(t *testing.T) { stmtExecInfo1 := generateAnyExecInfo() stmtExecInfo1.PrevSQL = "prevSQL" stmtExecInfo1.PrevSQLDigest = "prevSQLDigest" - ssMap.AddStatement(stmtExecInfo1) - key := &stmtSummaryByDigestKey{ - schemaName: stmtExecInfo1.SchemaName, - digest: stmtExecInfo1.Digest, - planDigest: stmtExecInfo1.PlanDigest, - prevDigest: stmtExecInfo1.PrevSQLDigest, - resourceGroupName: stmtExecInfo1.ResourceGroupName, + ssMap.AddStatement(genStmtSummaryByDigestKey(stmtExecInfo1), stmtExecInfo1) + key := &StmtDigestKey{ + SchemaName: stmtExecInfo1.SchemaName, + Digest: stmtExecInfo1.Digest, + PlanDigest: stmtExecInfo1.PlanDigest, + PrevDigest: stmtExecInfo1.PrevSQLDigest, + ResourceGroupName: stmtExecInfo1.ResourceGroupName, } require.Equal(t, 1, ssMap.summaryMap.Size()) _, ok := ssMap.summaryMap.Get(key) require.True(t, ok) // same prevSQL - ssMap.AddStatement(stmtExecInfo1) + ssMap.AddStatement(genStmtSummaryByDigestKey(stmtExecInfo1), stmtExecInfo1) require.Equal(t, 1, ssMap.summaryMap.Size()) // different prevSQL stmtExecInfo2 := stmtExecInfo1 stmtExecInfo2.PrevSQL = "prevSQL1" stmtExecInfo2.PrevSQLDigest = "prevSQLDigest1" - key.prevDigest = stmtExecInfo2.PrevSQLDigest - ssMap.AddStatement(stmtExecInfo2) + key.PrevDigest = stmtExecInfo2.PrevSQLDigest + ssMap.AddStatement(genStmtSummaryByDigestKey(stmtExecInfo2), stmtExecInfo2) require.Equal(t, 2, ssMap.summaryMap.Size()) _, ok = ssMap.summaryMap.Get(key) require.True(t, ok) @@ -1457,12 +1467,12 @@ func TestEndTime(t *testing.T) { ssMap.beginTimeForCurInterval = now - 100 stmtExecInfo1 := generateAnyExecInfo() - ssMap.AddStatement(stmtExecInfo1) - key := &stmtSummaryByDigestKey{ - schemaName: stmtExecInfo1.SchemaName, - digest: stmtExecInfo1.Digest, - planDigest: stmtExecInfo1.PlanDigest, - resourceGroupName: stmtExecInfo1.ResourceGroupName, + ssMap.AddStatement(genStmtSummaryByDigestKey(stmtExecInfo1), stmtExecInfo1) + key := &StmtDigestKey{ + SchemaName: stmtExecInfo1.SchemaName, + Digest: stmtExecInfo1.Digest, + PlanDigest: stmtExecInfo1.PlanDigest, + ResourceGroupName: stmtExecInfo1.ResourceGroupName, } require.Equal(t, 1, ssMap.summaryMap.Size()) value, ok := ssMap.summaryMap.Get(key) @@ -1478,7 +1488,7 @@ func TestEndTime(t *testing.T) { err := ssMap.SetRefreshInterval(1800) require.NoError(t, err) }() - ssMap.AddStatement(stmtExecInfo1) + ssMap.AddStatement(genStmtSummaryByDigestKey(stmtExecInfo1), stmtExecInfo1) require.Equal(t, 1, ssbd.history.Len()) ssElement = ssbd.history.Back().Value.(*stmtSummaryByDigestElement) require.Equal(t, now-100, ssElement.beginTime) @@ -1486,7 +1496,7 @@ func TestEndTime(t *testing.T) { err = ssMap.SetRefreshInterval(60) require.NoError(t, err) - ssMap.AddStatement(stmtExecInfo1) + ssMap.AddStatement(genStmtSummaryByDigestKey(stmtExecInfo1), stmtExecInfo1) require.Equal(t, 2, ssbd.history.Len()) now2 := time.Now().Unix() ssElement = ssbd.history.Front().Value.(*stmtSummaryByDigestElement) @@ -1507,12 +1517,12 @@ func TestPointGet(t *testing.T) { stmtExecInfo1 := generateAnyExecInfo() stmtExecInfo1.PlanDigest = "" stmtExecInfo1.LazyInfo.(*mockLazyInfo).plan = fakePlanDigestGenerator() - ssMap.AddStatement(stmtExecInfo1) - key := &stmtSummaryByDigestKey{ - schemaName: stmtExecInfo1.SchemaName, - digest: stmtExecInfo1.Digest, - planDigest: "", - resourceGroupName: stmtExecInfo1.ResourceGroupName, + ssMap.AddStatement(genStmtSummaryByDigestKey(stmtExecInfo1), stmtExecInfo1) + key := &StmtDigestKey{ + SchemaName: stmtExecInfo1.SchemaName, + Digest: stmtExecInfo1.Digest, + PlanDigest: "", + ResourceGroupName: stmtExecInfo1.ResourceGroupName, } require.Equal(t, 1, ssMap.summaryMap.Size()) value, ok := ssMap.summaryMap.Get(key) @@ -1521,7 +1531,7 @@ func TestPointGet(t *testing.T) { ssElement := ssbd.history.Back().Value.(*stmtSummaryByDigestElement) require.Equal(t, int64(1), ssElement.execCount) - ssMap.AddStatement(stmtExecInfo1) + ssMap.AddStatement(genStmtSummaryByDigestKey(stmtExecInfo1), stmtExecInfo1) require.Equal(t, int64(2), ssElement.execCount) } @@ -1533,7 +1543,7 @@ func TestAccessPrivilege(t *testing.T) { for i := range loops { stmtExecInfo1.Digest = fmt.Sprintf("digest%d", i) - ssMap.AddStatement(stmtExecInfo1) + ssMap.AddStatement(genStmtSummaryByDigestKey(stmtExecInfo1), stmtExecInfo1) } user := &auth.UserIdentity{Username: "user"} diff --git a/pkg/util/stmtsummary/v2/BUILD.bazel b/pkg/util/stmtsummary/v2/BUILD.bazel index 0d1243c4be01b..e2067b83d9a86 100644 --- a/pkg/util/stmtsummary/v2/BUILD.bazel +++ b/pkg/util/stmtsummary/v2/BUILD.bazel @@ -20,7 +20,6 @@ go_library( "//pkg/types", "//pkg/util", "//pkg/util/execdetails", - "//pkg/util/hack", "//pkg/util/kvcache", "//pkg/util/logutil", "//pkg/util/plancodec", @@ -57,6 +56,7 @@ go_test( "//pkg/types", "//pkg/util", "//pkg/util/set", + "//pkg/util/stmtsummary", "@com_github_stretchr_testify//require", "@org_uber_go_goleak//:goleak", ], diff --git a/pkg/util/stmtsummary/v2/reader_test.go b/pkg/util/stmtsummary/v2/reader_test.go index cf5dded90c7ad..48359cc0b6489 100644 --- a/pkg/util/stmtsummary/v2/reader_test.go +++ b/pkg/util/stmtsummary/v2/reader_test.go @@ -28,6 +28,7 @@ import ( "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util" "github.com/pingcap/tidb/pkg/util/set" + "github.com/pingcap/tidb/pkg/util/stmtsummary" "github.com/stretchr/testify/require" ) @@ -235,16 +236,21 @@ func TestMemReader(t *testing.T) { ss := NewStmtSummary4Test(3) defer ss.Close() - ss.Add(GenerateStmtExecInfo4Test("digest1")) - ss.Add(GenerateStmtExecInfo4Test("digest1")) - ss.Add(GenerateStmtExecInfo4Test("digest2")) - ss.Add(GenerateStmtExecInfo4Test("digest2")) - ss.Add(GenerateStmtExecInfo4Test("digest3")) - ss.Add(GenerateStmtExecInfo4Test("digest3")) - ss.Add(GenerateStmtExecInfo4Test("digest4")) - ss.Add(GenerateStmtExecInfo4Test("digest4")) - ss.Add(GenerateStmtExecInfo4Test("digest5")) - ss.Add(GenerateStmtExecInfo4Test("digest5")) + ssAdd := func(info *stmtsummary.StmtExecInfo) { + k := genStmtSummaryByDigestKey(info) + ss.Add(k, info) + } + + ssAdd(GenerateStmtExecInfo4Test("digest1")) + ssAdd(GenerateStmtExecInfo4Test("digest1")) + ssAdd(GenerateStmtExecInfo4Test("digest2")) + ssAdd(GenerateStmtExecInfo4Test("digest2")) + ssAdd(GenerateStmtExecInfo4Test("digest3")) + ssAdd(GenerateStmtExecInfo4Test("digest3")) + ssAdd(GenerateStmtExecInfo4Test("digest4")) + ssAdd(GenerateStmtExecInfo4Test("digest4")) + ssAdd(GenerateStmtExecInfo4Test("digest5")) + ssAdd(GenerateStmtExecInfo4Test("digest5")) reader := NewMemReader(ss, columns, "", timeLocation, nil, false, nil, nil) rows := reader.Rows() require.Len(t, rows, 4) // 3 rows + 1 other @@ -456,3 +462,13 @@ func readAllRows(t *testing.T, reader *HistoryReader) [][]types.Datum { } return results } + +func genStmtSummaryByDigestKey(info *stmtsummary.StmtExecInfo) *stmtsummary.StmtDigestKey { + return &stmtsummary.StmtDigestKey{ + SchemaName: info.SchemaName, + Digest: info.Digest, + PrevDigest: info.PrevSQLDigest, + PlanDigest: info.PlanDigest, + ResourceGroupName: info.ResourceGroupName, + } +} diff --git a/pkg/util/stmtsummary/v2/stmtsummary.go b/pkg/util/stmtsummary/v2/stmtsummary.go index 267f5cf66088f..3b3071cb7d004 100644 --- a/pkg/util/stmtsummary/v2/stmtsummary.go +++ b/pkg/util/stmtsummary/v2/stmtsummary.go @@ -27,7 +27,6 @@ import ( "github.com/pingcap/tidb/pkg/config" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/types" - "github.com/pingcap/tidb/pkg/util/hack" "github.com/pingcap/tidb/pkg/util/kvcache" "github.com/pingcap/tidb/pkg/util/logutil" "github.com/pingcap/tidb/pkg/util/stmtsummary" @@ -240,18 +239,11 @@ func (s *StmtSummary) SetRefreshInterval(v uint32) error { // of StmtSummary. Before adding, it will check whether the current window has // expired, and if it has expired, the window will be persisted asynchronously // and a new window will be created to replace the current one. -func (s *StmtSummary) Add(info *stmtsummary.StmtExecInfo) { +func (s *StmtSummary) Add(k *stmtsummary.StmtDigestKey, info *stmtsummary.StmtExecInfo) { if s.closed.Load() { return } - k := &stmtKey{ - schemaName: info.SchemaName, - digest: info.Digest, - prevDigest: info.PrevSQLDigest, - planDigest: info.PlanDigest, - resourceGroupName: info.ResourceGroupName, - } k.Hash() // Calculate hash value in advance, to reduce the time holding the window lock. // Add info to the current statistics window. @@ -261,7 +253,7 @@ func (s *StmtSummary) Add(info *stmtsummary.StmtExecInfo) { record = v.(*lockedStmtRecord) } else { record = &lockedStmtRecord{StmtRecord: NewStmtRecord(info)} - s.window.lru.Put(k, record) + s.window.lru.Put(k.Clone(), record) } s.windowLock.Unlock() @@ -431,7 +423,7 @@ func newStmtWindow(begin time.Time, capacity uint) *stmtWindow { r := v.(*lockedStmtRecord) r.Lock() defer r.Unlock() - w.evicted.add(k.(*stmtKey), r.StmtRecord) + w.evicted.add(k.(*stmtsummary.StmtDigestKey), r.StmtRecord) }) return w } @@ -446,36 +438,6 @@ type stmtStorage interface { sync() error } -// stmtKey defines key for stmtElement. -type stmtKey struct { - // Same statements may appear in different schema, but they refer to different tables. - schemaName string - digest string - // The digest of the previous statement. - prevDigest string - // The digest of the plan of this SQL. - planDigest string - // `resourceGroupName` is the resource group's name of this statement is bind to. - resourceGroupName string - // `hash` is the hash value of this object. - hash []byte -} - -// Hash implements SimpleLRUCache.Key. -// Only when current SQL is `commit` do we record `prevSQL`. Otherwise, `prevSQL` is empty. -// `prevSQL` is included in the key To distinguish different transactions. -func (k *stmtKey) Hash() []byte { - if len(k.hash) == 0 { - k.hash = make([]byte, 0, len(k.schemaName)+len(k.digest)+len(k.prevDigest)+len(k.planDigest)+len(k.resourceGroupName)) - k.hash = append(k.hash, hack.Slice(k.digest)...) - k.hash = append(k.hash, hack.Slice(k.schemaName)...) - k.hash = append(k.hash, hack.Slice(k.prevDigest)...) - k.hash = append(k.hash, hack.Slice(k.planDigest)...) - k.hash = append(k.hash, hack.Slice(k.resourceGroupName)...) - } - return k.hash -} - type stmtEvicted struct { sync.Mutex keys map[string]struct{} @@ -494,7 +456,7 @@ func newStmtEvicted() *stmtEvicted { } } -func (e *stmtEvicted) add(key *stmtKey, record *StmtRecord) { +func (e *stmtEvicted) add(key *stmtsummary.StmtDigestKey, record *StmtRecord) { if key == nil || record == nil { return } @@ -533,11 +495,11 @@ func (*mockStmtStorage) sync() error { /* Public proxy functions between v1 and v2 */ // Add wraps GlobalStmtSummary.Add and stmtsummary.StmtSummaryByDigestMap.AddStatement. -func Add(stmtExecInfo *stmtsummary.StmtExecInfo) { +func Add(key *stmtsummary.StmtDigestKey, stmtExecInfo *stmtsummary.StmtExecInfo) { if config.GetGlobalConfig().Instance.StmtSummaryEnablePersistent { - GlobalStmtSummary.Add(stmtExecInfo) + GlobalStmtSummary.Add(key, stmtExecInfo) } else { - stmtsummary.StmtSummaryByDigestMap.AddStatement(stmtExecInfo) + stmtsummary.StmtSummaryByDigestMap.AddStatement(key, stmtExecInfo) } } diff --git a/pkg/util/stmtsummary/v2/stmtsummary_test.go b/pkg/util/stmtsummary/v2/stmtsummary_test.go index cc489195425df..1431a5d0a2355 100644 --- a/pkg/util/stmtsummary/v2/stmtsummary_test.go +++ b/pkg/util/stmtsummary/v2/stmtsummary_test.go @@ -17,21 +17,27 @@ package stmtsummary import ( "testing" + "github.com/pingcap/tidb/pkg/util/stmtsummary" "github.com/stretchr/testify/require" ) func TestStmtWindow(t *testing.T) { ss := NewStmtSummary4Test(5) defer ss.Close() - ss.Add(GenerateStmtExecInfo4Test("digest1")) - ss.Add(GenerateStmtExecInfo4Test("digest1")) - ss.Add(GenerateStmtExecInfo4Test("digest2")) - ss.Add(GenerateStmtExecInfo4Test("digest2")) - ss.Add(GenerateStmtExecInfo4Test("digest3")) - ss.Add(GenerateStmtExecInfo4Test("digest4")) - ss.Add(GenerateStmtExecInfo4Test("digest5")) - ss.Add(GenerateStmtExecInfo4Test("digest6")) - ss.Add(GenerateStmtExecInfo4Test("digest7")) + ssAdd := func(info *stmtsummary.StmtExecInfo) { + k := genStmtSummaryByDigestKey(info) + ss.Add(k, info) + } + + ssAdd(GenerateStmtExecInfo4Test("digest1")) + ssAdd(GenerateStmtExecInfo4Test("digest1")) + ssAdd(GenerateStmtExecInfo4Test("digest2")) + ssAdd(GenerateStmtExecInfo4Test("digest2")) + ssAdd(GenerateStmtExecInfo4Test("digest3")) + ssAdd(GenerateStmtExecInfo4Test("digest4")) + ssAdd(GenerateStmtExecInfo4Test("digest5")) + ssAdd(GenerateStmtExecInfo4Test("digest6")) + ssAdd(GenerateStmtExecInfo4Test("digest7")) require.Equal(t, 5, ss.window.lru.Size()) require.Equal(t, 2, ss.window.evicted.count()) require.Equal(t, int64(4), ss.window.evicted.other.ExecCount) // digest1 digest1 digest2 digest2 @@ -46,18 +52,22 @@ func TestStmtSummary(t *testing.T) { defer ss.Close() w := ss.window - ss.Add(GenerateStmtExecInfo4Test("digest1")) - ss.Add(GenerateStmtExecInfo4Test("digest2")) - ss.Add(GenerateStmtExecInfo4Test("digest3")) - ss.Add(GenerateStmtExecInfo4Test("digest4")) - ss.Add(GenerateStmtExecInfo4Test("digest5")) + ssAdd := func(info *stmtsummary.StmtExecInfo) { + k := genStmtSummaryByDigestKey(info) + ss.Add(k, info) + } + ssAdd(GenerateStmtExecInfo4Test("digest1")) + ssAdd(GenerateStmtExecInfo4Test("digest2")) + ssAdd(GenerateStmtExecInfo4Test("digest3")) + ssAdd(GenerateStmtExecInfo4Test("digest4")) + ssAdd(GenerateStmtExecInfo4Test("digest5")) require.Equal(t, 3, w.lru.Size()) require.Equal(t, 2, w.evicted.count()) ss.rotate(timeNow()) - ss.Add(GenerateStmtExecInfo4Test("digest6")) - ss.Add(GenerateStmtExecInfo4Test("digest7")) + ssAdd(GenerateStmtExecInfo4Test("digest6")) + ssAdd(GenerateStmtExecInfo4Test("digest7")) w = ss.window require.Equal(t, 2, w.lru.Size()) require.Equal(t, 0, w.evicted.count()) @@ -71,21 +81,25 @@ func TestStmtSummaryFlush(t *testing.T) { ss := NewStmtSummary4Test(1000) ss.storage = storage - ss.Add(GenerateStmtExecInfo4Test("digest1")) - ss.Add(GenerateStmtExecInfo4Test("digest2")) - ss.Add(GenerateStmtExecInfo4Test("digest3")) + ssAdd := func(info *stmtsummary.StmtExecInfo) { + k := genStmtSummaryByDigestKey(info) + ss.Add(k, info) + } + ssAdd(GenerateStmtExecInfo4Test("digest1")) + ssAdd(GenerateStmtExecInfo4Test("digest2")) + ssAdd(GenerateStmtExecInfo4Test("digest3")) ss.rotate(timeNow()) - ss.Add(GenerateStmtExecInfo4Test("digest1")) - ss.Add(GenerateStmtExecInfo4Test("digest2")) - ss.Add(GenerateStmtExecInfo4Test("digest3")) + ssAdd(GenerateStmtExecInfo4Test("digest1")) + ssAdd(GenerateStmtExecInfo4Test("digest2")) + ssAdd(GenerateStmtExecInfo4Test("digest3")) ss.rotate(timeNow()) - ss.Add(GenerateStmtExecInfo4Test("digest1")) - ss.Add(GenerateStmtExecInfo4Test("digest2")) - ss.Add(GenerateStmtExecInfo4Test("digest3")) + ssAdd(GenerateStmtExecInfo4Test("digest1")) + ssAdd(GenerateStmtExecInfo4Test("digest2")) + ssAdd(GenerateStmtExecInfo4Test("digest3")) ss.Close()