From 8e99bd712d85bddac7674557946fb9a35a1a6e5e Mon Sep 17 00:00:00 2001 From: mornyx Date: Thu, 23 Dec 2021 01:11:57 +0800 Subject: [PATCH 01/15] Add reporting for sql exec count and sql duration Signed-off-by: mornyx --- go.mod | 3 + go.sum | 2 - util/topsql/reporter/mock/server.go | 14 +- util/topsql/reporter/pubsub.go | 10 +- util/topsql/reporter/reporter.go | 382 ++++++++++++++++++++++---- util/topsql/reporter/reporter_test.go | 363 +++++++++++++++++++++++- util/topsql/reporter/single_target.go | 8 +- util/topsql/stmtstats/aggregator.go | 10 +- util/topsql/topsql.go | 1 + 9 files changed, 712 insertions(+), 81 deletions(-) diff --git a/go.mod b/go.mod index f11eab1cf0456..d85b17cd96f5b 100644 --- a/go.mod +++ b/go.mod @@ -102,3 +102,6 @@ replace github.com/pingcap/tidb/parser => ./parser // fix potential security issue(CVE-2020-26160) introduced by indirect dependency. replace github.com/dgrijalva/jwt-go => github.com/form3tech-oss/jwt-go v3.2.6-0.20210809144907-32ab6a8243d7+incompatible + +// TODO(mornyx): remove +replace github.com/pingcap/tipb => /Users/mornyx/Desktop/repos/tipb diff --git a/go.sum b/go.sum index 2a4c23d9d70bc..eca38cd7d38a5 100644 --- a/go.sum +++ b/go.sum @@ -598,8 +598,6 @@ github.com/pingcap/tidb-dashboard v0.0.0-20211008050453-a25c25809529/go.mod h1:O github.com/pingcap/tidb-dashboard v0.0.0-20211107164327-80363dfbe884/go.mod h1:OCXbZTBTIMRcIt0jFsuCakZP+goYRv6IjawKbwLS2TQ= github.com/pingcap/tidb-tools v5.2.2-0.20211019062242-37a8bef2fa17+incompatible h1:c7+izmker91NkjkZ6FgTlmD4k1A5FLOAq+li6Ki2/GY= github.com/pingcap/tidb-tools v5.2.2-0.20211019062242-37a8bef2fa17+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM= -github.com/pingcap/tipb v0.0.0-20211201080053-bd104bb270ba h1:Tt5W/maVBUbG+wxg2nfc88Cqj/HiWYb0TJQ2Rfi0UOQ= -github.com/pingcap/tipb v0.0.0-20211201080053-bd104bb270ba/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= diff --git a/util/topsql/reporter/mock/server.go b/util/topsql/reporter/mock/server.go index 21737e1e6ca9b..04fb200d7ab84 100644 --- a/util/topsql/reporter/mock/server.go +++ b/util/topsql/reporter/mock/server.go @@ -26,6 +26,8 @@ import ( "github.com/pingcap/tipb/go-tipb" "go.uber.org/zap" "google.golang.org/grpc" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" ) type mockAgentServer struct { @@ -34,7 +36,7 @@ type mockAgentServer struct { grpcServer *grpc.Server sqlMetas map[string]tipb.SQLMeta planMetas map[string]string - records [][]*tipb.CPUTimeRecord + records [][]*tipb.TopSQLRecord hang struct { beginTime atomic.Value // time.Time endTime atomic.Value // time.Time @@ -86,7 +88,11 @@ func (svr *mockAgentServer) mayHang() { } func (svr *mockAgentServer) ReportCPUTimeRecords(stream tipb.TopSQLAgent_ReportCPUTimeRecordsServer) error { - records := make([]*tipb.CPUTimeRecord, 0, 10) + return status.Error(codes.Unimplemented, "unimplemented") +} + +func (svr *mockAgentServer) ReportTopSQLRecords(stream tipb.TopSQLAgent_ReportTopSQLRecordsServer) error { + records := make([]*tipb.TopSQLRecord, 0, 10) for { svr.mayHang() req, err := stream.Recv() @@ -180,10 +186,10 @@ func (svr *mockAgentServer) GetPlanMetaByDigestBlocking(digest []byte, timeout t } } -func (svr *mockAgentServer) GetLatestRecords() []*tipb.CPUTimeRecord { +func (svr *mockAgentServer) GetLatestRecords() []*tipb.TopSQLRecord { svr.Lock() records := svr.records - svr.records = [][]*tipb.CPUTimeRecord{} + svr.records = [][]*tipb.TopSQLRecord{} svr.Unlock() if len(records) == 0 { diff --git a/util/topsql/reporter/pubsub.go b/util/topsql/reporter/pubsub.go index 7d01c077e058f..b39a2a7b8b4fb 100644 --- a/util/topsql/reporter/pubsub.go +++ b/util/topsql/reporter/pubsub.go @@ -146,7 +146,7 @@ func (ds *pubSubDataSink) run() error { } func (ds *pubSubDataSink) doSend(ctx context.Context, data *ReportData) error { - if err := ds.sendCPUTime(ctx, data.CPUTimeRecords); err != nil { + if err := ds.sendTopSQLRecords(ctx, data.DataRecords); err != nil { return err } if err := ds.sendSQLMeta(ctx, data.SQLMetas); err != nil { @@ -155,7 +155,7 @@ func (ds *pubSubDataSink) doSend(ctx context.Context, data *ReportData) error { return ds.sendPlanMeta(ctx, data.PlanMetas) } -func (ds *pubSubDataSink) sendCPUTime(ctx context.Context, records []tipb.CPUTimeRecord) (err error) { +func (ds *pubSubDataSink) sendTopSQLRecords(ctx context.Context, records []tipb.TopSQLRecord) (err error) { if len(records) == 0 { return } @@ -171,11 +171,11 @@ func (ds *pubSubDataSink) sendCPUTime(ctx context.Context, records []tipb.CPUTim } }() - cpuRecord := &tipb.TopSQLSubResponse_Record{} - r := &tipb.TopSQLSubResponse{RespOneof: cpuRecord} + topSQLRecord := &tipb.TopSQLSubResponse_DataRecord{} + r := &tipb.TopSQLSubResponse{RespOneof: topSQLRecord} for i := range records { - cpuRecord.Record = &records[i] + topSQLRecord.DataRecord = &records[i] if err = ds.stream.Send(r); err != nil { return } diff --git a/util/topsql/reporter/reporter.go b/util/topsql/reporter/reporter.go index 879ac0d61e438..0fc1b739a1f6f 100644 --- a/util/topsql/reporter/reporter.go +++ b/util/topsql/reporter/reporter.go @@ -28,6 +28,7 @@ import ( "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/logutil" + "github.com/pingcap/tidb/util/topsql/stmtstats" "github.com/pingcap/tidb/util/topsql/tracecpu" "github.com/pingcap/tipb/go-tipb" "github.com/wangjohn/quickselect" @@ -49,6 +50,7 @@ var _ TopSQLReporter = &RemoteTopSQLReporter{} // TopSQLReporter collects Top SQL metrics. type TopSQLReporter interface { tracecpu.Collector + stmtstats.Collector RegisterSQL(sqlDigest []byte, normalizedSQL string, isInternal bool) RegisterPlan(planDigest []byte, normalizedPlan string) Close() @@ -68,15 +70,173 @@ type cpuData struct { // dataPoints represents the cumulative SQL plan CPU time in current minute window // dataPoints do not guarantee the TimestampList is sorted by timestamp when there is a time jump backward. type dataPoints struct { - SQLDigest []byte - PlanDigest []byte - TimestampList []uint64 - CPUTimeMsList []uint32 - CPUTimeMsTotal uint64 + SQLDigest []byte + PlanDigest []byte + TimestampList []uint64 + CPUTimeMsList []uint32 + CPUTimeMsTotal uint64 + StmtExecCountList []uint64 + StmtKvExecCountList []map[string]uint64 + StmtDurationSumNsList []uint64 + + // tsIndex is used to quickly find the corresponding array index through timestamp. + // + // map: timestamp => index of TimestampList / CPUTimeMsList / StmtExecCountList / ... + tsIndex map[uint64]int +} + +func newDataPoints(sqlDigest, planDigest []byte) *dataPoints { + listCap := int(variable.TopSQLVariable.ReportIntervalSeconds.Load()/variable.TopSQLVariable.PrecisionSeconds.Load() + 1) + if listCap < 1 { + listCap = 1 + } + return &dataPoints{ + SQLDigest: sqlDigest, + PlanDigest: planDigest, + CPUTimeMsList: make([]uint32, 0, listCap), + TimestampList: make([]uint64, 0, listCap), + StmtExecCountList: make([]uint64, 0, listCap), + StmtKvExecCountList: make([]map[string]uint64, 0, listCap), + StmtDurationSumNsList: make([]uint64, 0, listCap), + tsIndex: make(map[uint64]int, listCap), + } +} + +func (d *dataPoints) rebuildTsIndex() { + if len(d.TimestampList) == 0 { + d.tsIndex = map[uint64]int{} + return + } + d.tsIndex = make(map[uint64]int, len(d.TimestampList)) + for index, ts := range d.TimestampList { + d.tsIndex[ts] = index + } +} + +func (d *dataPoints) appendCPUTime(timestamp uint64, cpuTimeMs uint32) { + if index, ok := d.tsIndex[timestamp]; ok { + // For the same timestamp, we have already called appendStmtStatsItem, + // d.TimestampList already exists the corresponding timestamp. And it + // can be determined that the corresponding index of d.StmtXxx has been + // correctly assigned, and the corresponding index of d.CPUTimeMsList + // has been set to 0, so we directly replace it. + // + // let timestamp = 10000, cpuTimeMs = 123 + // + // Before: + // tsIndex: [10000 => 0] + // TimestampList: [10000] + // CPUTimeMsList: [0] + // StmtExecCountList: [999] + // StmtKvExecCountList: [map{"9.9.9.9:9":999}] + // StmtDurationSumNsList: [999] + // + // After: + // tsIndex: [10000 => 0] + // TimestampList: [10000] + // CPUTimeMsList: [123] + // StmtExecCountList: [999] + // StmtKvExecCountList: [map{"9.9.9.9:9":999}] + // StmtDurationSumNsList: [999] + // + d.CPUTimeMsList[index] = cpuTimeMs + } else { + // For this timestamp, we have not appended any data. So append it directly, + // and set the data not related to CPUTimeList to 0. + // + // let timestamp = 10000, cpu_time = 123 + // + // Before: + // tsIndex: [] + // TimestampList: [] + // CPUTimeMsList: [] + // StmtExecCountList: [] + // StmtKvExecCountList: [] + // StmtDurationSumNsList: [] + // + // After: + // tsIndex: [10000 => 0] + // TimestampList: [10000] + // CPUTimeMsList: [123] + // StmtExecCountList: [0] + // StmtKvExecCountList: [map{}] + // StmtDurationSumNsList: [0] + // + d.tsIndex[timestamp] = len(d.TimestampList) + d.TimestampList = append(d.TimestampList, timestamp) + d.CPUTimeMsList = append(d.CPUTimeMsList, cpuTimeMs) + d.StmtExecCountList = append(d.StmtExecCountList, 0) + d.StmtKvExecCountList = append(d.StmtKvExecCountList, map[string]uint64{}) + d.StmtDurationSumNsList = append(d.StmtDurationSumNsList, 0) + } + d.CPUTimeMsTotal += uint64(cpuTimeMs) +} + +func (d *dataPoints) appendStmtStatsItem(timestamp uint64, item *stmtstats.StatementStatsItem) { + if index, ok := d.tsIndex[timestamp]; ok { + // For the same timestamp, we have already called appendCPUTime, + // d.TimestampList already exists the corresponding timestamp. And it + // can be determined that the corresponding index of d.CPUTimeMsList has been + // correctly assigned, and the corresponding index of d.StmtXxx + // has been set to 0 or empty map, so we directly replace it. + // + // let timestamp = 10000, execCount = 123, kvExecCount = map{"1.1.1.1:1": 123}, durationSum = 456 + // + // Before: + // tsIndex: [10000 => 0] + // TimestampList: [10000] + // CPUTimeMsList: [999] + // StmtExecCountList: [0] + // StmtKvExecCountList: [map{}] + // StmtDurationSumNsList: [0] + // + // After: + // tsIndex: [10000 => 0] + // TimestampList: [10000] + // CPUTimeMsList: [999] + // StmtExecCountList: [123] + // StmtKvExecCountList: [map{"1.1.1.1:1": 123}] + // StmtDurationSumNsList: [456] + // + d.StmtExecCountList[index] = item.ExecCount + d.StmtKvExecCountList[index] = item.KvStatsItem.KvExecCount + d.StmtDurationSumNsList[index] = 0 // TODO(mornyx): add duration + } else { + // For this timestamp, we have not appended any data. So append it directly, + // the corresponding index of d.CPUTimeList is preset to 0. + // + // let timestamp = 10000, execCount = 123, kvExecCount = map{"1.1.1.1:1": 123}, durationSum = 456 + // + // Before: + // tsIndex: [] + // TimestampList: [] + // CPUTimeMsList: [] + // StmtExecCountList: [] + // StmtKvExecCountList: [] + // StmtDurationSumNsList: [] + // + // After: + // tsIndex: [10000 => 0] + // TimestampList: [10000] + // CPUTimeMsList: [0] + // StmtExecCountList: [123] + // StmtKvExecCountList: [map{"1.1.1.1:1": 123}] + // StmtDurationSumNsList: [456] + // + d.tsIndex[timestamp] = len(d.TimestampList) + d.TimestampList = append(d.TimestampList, timestamp) + d.CPUTimeMsList = append(d.CPUTimeMsList, 0) + d.StmtExecCountList = append(d.StmtExecCountList, item.ExecCount) + d.StmtKvExecCountList = append(d.StmtKvExecCountList, item.KvStatsItem.KvExecCount) + d.StmtDurationSumNsList = append(d.StmtDurationSumNsList, 0) // TODO(mornyx): add duration + } } func (d *dataPoints) isInvalid() bool { - return len(d.TimestampList) != len(d.CPUTimeMsList) + return !(len(d.TimestampList) == len(d.CPUTimeMsList) && + len(d.TimestampList) == len(d.StmtExecCountList) && + len(d.TimestampList) == len(d.StmtKvExecCountList) && + len(d.TimestampList) == len(d.StmtDurationSumNsList)) } func (d *dataPoints) Len() int { @@ -88,8 +248,24 @@ func (d *dataPoints) Less(i, j int) bool { return d.TimestampList[i] < d.TimestampList[j] } func (d *dataPoints) Swap(i, j int) { + // before swap: + // TimestampList: [10000, 10001, 10002] + // tsIndex: [10000 => 0, 10001 => 1, 10002 => 2] + // + // let i = 0, j = 1 + // after swap tsIndex: + // TimestampList: [10000, 10001, 10002] + // tsIndex: [10000 => 1, 10001 => 0, 10002 => 2] + // + // after swap TimestampList: + // TimestampList: [10001, 10000, 10002] + // tsIndex: [10000 => 1, 10001 => 0, 10002 => 2] + d.tsIndex[d.TimestampList[i]], d.tsIndex[d.TimestampList[j]] = d.tsIndex[d.TimestampList[j]], d.tsIndex[d.TimestampList[i]] d.TimestampList[i], d.TimestampList[j] = d.TimestampList[j], d.TimestampList[i] d.CPUTimeMsList[i], d.CPUTimeMsList[j] = d.CPUTimeMsList[j], d.CPUTimeMsList[i] + d.StmtExecCountList[i], d.StmtExecCountList[j] = d.StmtExecCountList[j], d.StmtExecCountList[i] + d.StmtKvExecCountList[i], d.StmtKvExecCountList[j] = d.StmtKvExecCountList[j], d.StmtKvExecCountList[i] + d.StmtDurationSumNsList[i], d.StmtDurationSumNsList[j] = d.StmtDurationSumNsList[j], d.StmtDurationSumNsList[i] } type dataPointsOrderByCPUTime []*dataPoints @@ -141,6 +317,7 @@ type RemoteTopSQLReporter struct { planMapLength atomic2.Int64 collectCPUDataChan chan cpuData + collectStmtRecordsChan chan []stmtstats.StatementStatsRecord reportCollectedDataChan chan collectedData // calling decodePlan this can take a while, so should not block critical paths @@ -166,6 +343,7 @@ func NewRemoteTopSQLReporter(decodePlan planBinaryDecodeFunc) *RemoteTopSQLRepor dataSinks: make(map[DataSink]struct{}, 10), collectCPUDataChan: make(chan cpuData, 1), + collectStmtRecordsChan: make(chan []stmtstats.StatementStatsRecord, 1), reportCollectedDataChan: make(chan collectedData, 1), decodePlan: decodePlan, } @@ -182,6 +360,7 @@ var ( ignoreExceedSQLCounter = metrics.TopSQLIgnoredCounter.WithLabelValues("ignore_exceed_sql") ignoreExceedPlanCounter = metrics.TopSQLIgnoredCounter.WithLabelValues("ignore_exceed_plan") ignoreCollectChannelFullCounter = metrics.TopSQLIgnoredCounter.WithLabelValues("ignore_collect_channel_full") + ignoreCollectStmtChannelFullCounter = metrics.TopSQLIgnoredCounter.WithLabelValues("ignore_collect_stmt_channel_full") ignoreReportChannelFullCounter = metrics.TopSQLIgnoredCounter.WithLabelValues("ignore_report_channel_full") reportAllDurationSuccHistogram = metrics.TopSQLReportDurationHistogram.WithLabelValues("all", metrics.LblOK) reportAllDurationFailedHistogram = metrics.TopSQLReportDurationHistogram.WithLabelValues("all", metrics.LblError) @@ -291,6 +470,23 @@ func (tsr *RemoteTopSQLReporter) Collect(timestamp uint64, records []tracecpu.SQ } } +// CollectStmtStatsRecords receives stmtstats.StatementStatsRecord for processing. +// WARN: It will drop the records if the processing is not in time. +// This function is thread-safe and efficient. +// +// CollectStmtStatsRecords implements stmtstats.Collector.CollectStmtStatsRecords. +func (tsr *RemoteTopSQLReporter) CollectStmtStatsRecords(records []stmtstats.StatementStatsRecord) { + if len(records) == 0 { + return + } + select { + case tsr.collectStmtRecordsChan <- records: + default: + // ignore if chan blocked + ignoreCollectStmtChannelFullCounter.Inc() + } +} + // Close uses to close and release the reporter resource. func (tsr *RemoteTopSQLReporter) Close() { tsr.cancel() @@ -311,17 +507,19 @@ func addEvictedCPUTime(collectTarget map[string]*dataPoints, timestamp uint64, t } others, ok := collectTarget[keyOthers] if !ok { - others = &dataPoints{} + others = &dataPoints{tsIndex: map[uint64]int{}} collectTarget[keyOthers] = others } - if len(others.TimestampList) == 0 { - others.TimestampList = []uint64{timestamp} - others.CPUTimeMsList = []uint32{totalCPUTimeMs} - } else { - others.TimestampList = append(others.TimestampList, timestamp) - others.CPUTimeMsList = append(others.CPUTimeMsList, totalCPUTimeMs) + others.appendCPUTime(timestamp, totalCPUTimeMs) +} + +func addEvictedStmtStatsItem(collectTarget map[string]*dataPoints, timestamp uint64, item *stmtstats.StatementStatsItem) { + others, ok := collectTarget[keyOthers] + if !ok { + others = &dataPoints{tsIndex: map[uint64]int{}} + collectTarget[keyOthers] = others } - others.CPUTimeMsTotal += uint64(totalCPUTimeMs) + others.appendStmtStatsItem(timestamp, item) } // addEvictedIntoSortedDataPoints adds evicted dataPoints into others. @@ -344,55 +542,101 @@ func addEvictedIntoSortedDataPoints(others *dataPoints, evict *dataPoints) *data others.TimestampList = evict.TimestampList others.CPUTimeMsList = evict.CPUTimeMsList others.CPUTimeMsTotal = evict.CPUTimeMsTotal + others.StmtExecCountList = evict.StmtExecCountList + others.StmtKvExecCountList = evict.StmtKvExecCountList + others.StmtDurationSumNsList = evict.StmtDurationSumNsList + others.tsIndex = evict.tsIndex return others } length := len(others.TimestampList) + len(evict.TimestampList) timestampList := make([]uint64, 0, length) cpuTimeMsList := make([]uint32, 0, length) + stmtExecCountList := make([]uint64, 0, length) + stmtKvExecCountList := make([]map[string]uint64, 0, length) + stmtDurationSumList := make([]uint64, 0, length) i := 0 j := 0 for i < len(others.TimestampList) && j < len(evict.TimestampList) { if others.TimestampList[i] == evict.TimestampList[j] { timestampList = append(timestampList, others.TimestampList[i]) cpuTimeMsList = append(cpuTimeMsList, others.CPUTimeMsList[i]+evict.CPUTimeMsList[j]) + stmtExecCountList = append(stmtExecCountList, others.StmtExecCountList[i]+evict.StmtExecCountList[j]) + stmtKvExecCountList = append(stmtKvExecCountList, mergeKvExecCountMap(others.StmtKvExecCountList[i], evict.StmtKvExecCountList[j])) + stmtDurationSumList = append(stmtDurationSumList, others.StmtDurationSumNsList[i]+evict.StmtDurationSumNsList[j]) i++ j++ } else if others.TimestampList[i] < evict.TimestampList[j] { timestampList = append(timestampList, others.TimestampList[i]) cpuTimeMsList = append(cpuTimeMsList, others.CPUTimeMsList[i]) + stmtExecCountList = append(stmtExecCountList, others.StmtExecCountList[i]) + stmtKvExecCountList = append(stmtKvExecCountList, others.StmtKvExecCountList[i]) + stmtDurationSumList = append(stmtDurationSumList, others.StmtDurationSumNsList[i]) i++ } else { timestampList = append(timestampList, evict.TimestampList[j]) cpuTimeMsList = append(cpuTimeMsList, evict.CPUTimeMsList[j]) + stmtExecCountList = append(stmtExecCountList, evict.StmtExecCountList[j]) + stmtKvExecCountList = append(stmtKvExecCountList, evict.StmtKvExecCountList[j]) + stmtDurationSumList = append(stmtDurationSumList, evict.StmtDurationSumNsList[j]) j++ } } if i < len(others.TimestampList) { timestampList = append(timestampList, others.TimestampList[i:]...) cpuTimeMsList = append(cpuTimeMsList, others.CPUTimeMsList[i:]...) + stmtExecCountList = append(stmtExecCountList, others.StmtExecCountList[i:]...) + for _, l := range others.StmtKvExecCountList[i:] { + stmtKvExecCountList = append(stmtKvExecCountList, l) + } + stmtDurationSumList = append(stmtDurationSumList, others.StmtDurationSumNsList[i:]...) } if j < len(evict.TimestampList) { timestampList = append(timestampList, evict.TimestampList[j:]...) cpuTimeMsList = append(cpuTimeMsList, evict.CPUTimeMsList[j:]...) + stmtExecCountList = append(stmtExecCountList, evict.StmtExecCountList[j:]...) + for _, l := range evict.StmtKvExecCountList[j:] { + stmtKvExecCountList = append(stmtKvExecCountList, l) + } + stmtDurationSumList = append(stmtDurationSumList, evict.StmtDurationSumNsList[j:]...) } others.TimestampList = timestampList others.CPUTimeMsList = cpuTimeMsList + others.StmtExecCountList = stmtExecCountList + others.StmtKvExecCountList = stmtKvExecCountList + others.StmtDurationSumNsList = stmtDurationSumList others.CPUTimeMsTotal += evict.CPUTimeMsTotal + others.rebuildTsIndex() return others } +func mergeKvExecCountMap(a, b map[string]uint64) map[string]uint64 { + r := map[string]uint64{} + for ka, va := range a { + r[ka] = va + } + for kb, vb := range b { + r[kb] += vb + } + return r +} + func (tsr *RemoteTopSQLReporter) collectWorker() { defer util.Recover("top-sql", "collectWorker", nil, false) collectedData := make(map[string]*dataPoints) + evictedDigest := make(map[uint64]map[stmtstats.SQLPlanDigest]struct{}) currentReportInterval := variable.TopSQLVariable.ReportIntervalSeconds.Load() reportTicker := time.NewTicker(time.Second * time.Duration(currentReportInterval)) for { select { case data := <-tsr.collectCPUDataChan: // On receiving data to collect: Write to local data array, and retain records with most CPU time. - tsr.doCollect(collectedData, data.timestamp, data.records) + tsr.doCollect(collectedData, evictedDigest, data.timestamp, data.records) + case data := <-tsr.collectStmtRecordsChan: + tsr.doCollectStmtRecords(collectedData, evictedDigest, data) case <-reportTicker.C: + // We clean up evictedDigest before reporting, to avoid continuous accumulation. + evictedDigest = make(map[uint64]map[stmtstats.SQLPlanDigest]struct{}) tsr.takeDataAndSendToReportChan(&collectedData) // Update `reportTicker` if report interval changed. if newInterval := variable.TopSQLVariable.ReportIntervalSeconds.Load(); newInterval != currentReportInterval { @@ -439,7 +683,10 @@ func getTopNDataPoints(records []*dataPoints) (topN, shouldEvict []*dataPoints) // doCollect collects top N records of each round into collectTarget, and evict the data that is not in top N. // All the evicted record will be summary into the collectedData.others. func (tsr *RemoteTopSQLReporter) doCollect( - collectTarget map[string]*dataPoints, timestamp uint64, records []tracecpu.SQLCPUTimeRecord) { + collectTarget map[string]*dataPoints, + evictedDigest map[uint64]map[stmtstats.SQLPlanDigest]struct{}, + timestamp uint64, + records []tracecpu.SQLCPUTimeRecord) { defer util.Recover("top-sql", "doCollect", nil, false) // Get top N records of each round records. @@ -447,29 +694,15 @@ func (tsr *RemoteTopSQLReporter) doCollect( records, evicted = getTopNRecords(records) keyBuf := bytes.NewBuffer(make([]byte, 0, 64)) - listCapacity := int(variable.TopSQLVariable.ReportIntervalSeconds.Load()/variable.TopSQLVariable.PrecisionSeconds.Load() + 1) - if listCapacity < 1 { - listCapacity = 1 - } // Collect the top N records to collectTarget for each round. for _, record := range records { key := encodeKey(keyBuf, record.SQLDigest, record.PlanDigest) entry, exist := collectTarget[key] if !exist { - entry = &dataPoints{ - SQLDigest: record.SQLDigest, - PlanDigest: record.PlanDigest, - CPUTimeMsList: make([]uint32, 1, listCapacity), - TimestampList: make([]uint64, 1, listCapacity), - } - entry.CPUTimeMsList[0] = record.CPUTimeMs - entry.TimestampList[0] = timestamp - collectTarget[key] = entry - } else { - entry.CPUTimeMsList = append(entry.CPUTimeMsList, record.CPUTimeMs) - entry.TimestampList = append(entry.TimestampList, timestamp) + collectTarget[key] = newDataPoints(record.SQLDigest, record.PlanDigest) + entry = collectTarget[key] } - entry.CPUTimeMsTotal += uint64(record.CPUTimeMs) + entry.appendCPUTime(timestamp, record.CPUTimeMs) } if len(evicted) == 0 { @@ -480,10 +713,51 @@ func (tsr *RemoteTopSQLReporter) doCollect( totalEvictedCPUTime := uint32(0) for _, evict := range evicted { totalEvictedCPUTime += evict.CPUTimeMs + + // Record which digests are evicted under each timestamp, and judge whether + // the corresponding CPUTime is evicted when stmtstats is collected, if it + // has been evicted, then we can ignore it directly. + if _, ok := evictedDigest[timestamp]; !ok { + evictedDigest[timestamp] = map[stmtstats.SQLPlanDigest]struct{}{} + } + evictedDigest[timestamp][stmtstats.SQLPlanDigest{ + SQLDigest: stmtstats.BinaryDigest(evict.SQLDigest), + PlanDigest: stmtstats.BinaryDigest(evict.PlanDigest), + }] = struct{}{} } addEvictedCPUTime(collectTarget, timestamp, totalEvictedCPUTime) } +func (tsr *RemoteTopSQLReporter) doCollectStmtRecords( + collectTarget map[string]*dataPoints, + evictedDigest map[uint64]map[stmtstats.SQLPlanDigest]struct{}, + records []stmtstats.StatementStatsRecord) { + defer util.Recover("top-sql", "doCollectStmtRecords", nil, false) + + keyBuf := bytes.NewBuffer(make([]byte, 0, 64)) + for _, record := range records { + timestamp := uint64(record.Timestamp) + for digest, item := range record.Data { + sqlDigest := []byte(digest.SQLDigest) + planDigest := []byte(digest.PlanDigest) + if digestSet, ok := evictedDigest[timestamp]; ok { + if _, ok := digestSet[digest]; ok { + // This record has been evicted due to low CPUTime. + addEvictedStmtStatsItem(collectTarget, timestamp, item) + continue + } + } + key := encodeKey(keyBuf, sqlDigest, planDigest) + entry, exist := collectTarget[key] + if !exist { + collectTarget[key] = newDataPoints(sqlDigest, planDigest) + entry = collectTarget[key] + } + entry.appendStmtStatsItem(timestamp, item) + } + } +} + // takeDataAndSendToReportChan takes collected data and then send to the report channel for reporting. func (tsr *RemoteTopSQLReporter) takeDataAndSendToReportChan(collectedDataPtr *map[string]*dataPoints) { data := collectedData{ @@ -516,14 +790,14 @@ type collectedData struct { // ReportData contains data that reporter sends to the agent type ReportData struct { - // CPUTimeRecords contains the topN collected records and the `others` record which aggregation all records that is out of Top N. - CPUTimeRecords []tipb.CPUTimeRecord - SQLMetas []tipb.SQLMeta - PlanMetas []tipb.PlanMeta + // DataRecords contains the topN collected records and the `others` record which aggregation all records that is out of Top N. + DataRecords []tipb.TopSQLRecord + SQLMetas []tipb.SQLMeta + PlanMetas []tipb.PlanMeta } func (d *ReportData) hasData() bool { - return len(d.CPUTimeRecords) != 0 || len(d.SQLMetas) != 0 || len(d.PlanMetas) != 0 + return len(d.DataRecords) != 0 || len(d.SQLMetas) != 0 || len(d.PlanMetas) != 0 } // reportWorker sends data to the gRPC endpoint from the `reportCollectedDataChan` one by one. @@ -559,7 +833,14 @@ func getTopNFromCollected(collected collectedData) (records []*dataPoints) { records = make([]*dataPoints, 0, len(collected.records)) for _, v := range collected.records { - records = append(records, v) + if v.CPUTimeMsTotal > 0 { + // For a certain timestamp, when StmtStats is collected, but CPUTime + // has not been collected, we have not yet filled the evictedDigest data, + // so it is possible that a piece of data is evicted in CPUTime, but still + // exists in collectTarget due to StmtStats. So we are here to remove those + // data that have no CPUTime at all. + records = append(records, v) + } } // Evict all records that is out of Top N. @@ -588,17 +869,24 @@ func getTopNFromCollected(collected collectedData) (records []*dataPoints) { // will do heavy jobs in sync.Map.Range and it may block other readers and writers. func (tsr *RemoteTopSQLReporter) buildReportData(records []*dataPoints, sqlMap *sync.Map, planMap *sync.Map) *ReportData { res := &ReportData{ - CPUTimeRecords: make([]tipb.CPUTimeRecord, 0, len(records)), - SQLMetas: make([]tipb.SQLMeta, 0, len(records)), - PlanMetas: make([]tipb.PlanMeta, 0, len(records)), + DataRecords: make([]tipb.TopSQLRecord, 0, len(records)), + SQLMetas: make([]tipb.SQLMeta, 0, len(records)), + PlanMetas: make([]tipb.PlanMeta, 0, len(records)), } for _, record := range records { - res.CPUTimeRecords = append(res.CPUTimeRecords, tipb.CPUTimeRecord{ - RecordListTimestampSec: record.TimestampList, - RecordListCpuTimeMs: record.CPUTimeMsList, - SqlDigest: record.SQLDigest, - PlanDigest: record.PlanDigest, + recordListStmtKvExecCount := make([]*tipb.TopSQLStmtKvExecCount, len(record.StmtKvExecCountList)) + for n, l := range record.StmtKvExecCountList { + recordListStmtKvExecCount[n] = &tipb.TopSQLStmtKvExecCount{ExecCount: l} + } + res.DataRecords = append(res.DataRecords, tipb.TopSQLRecord{ + RecordListTimestampSec: record.TimestampList, + RecordListCpuTimeMs: record.CPUTimeMsList, + RecordListStmtExecCount: record.StmtExecCountList, + RecordListStmtKvExecCount: recordListStmtKvExecCount, + RecordListStmtDurationSumNs: record.StmtDurationSumNsList, + SqlDigest: record.SQLDigest, + PlanDigest: record.PlanDigest, }) } diff --git a/util/topsql/reporter/reporter_test.go b/util/topsql/reporter/reporter_test.go index df251165de02d..0728f2dcc4db0 100644 --- a/util/topsql/reporter/reporter_test.go +++ b/util/topsql/reporter/reporter_test.go @@ -18,14 +18,17 @@ import ( "sort" "strconv" "strings" + "sync" "testing" "time" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/util/topsql/reporter/mock" + "github.com/pingcap/tidb/util/topsql/stmtstats" "github.com/pingcap/tidb/util/topsql/tracecpu" "github.com/pingcap/tipb/go-tipb" + "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" ) @@ -262,7 +265,7 @@ func TestCollectAndTopN(t *testing.T) { sort.Slice(results, func(i, j int) bool { return string(results[i].SqlDigest) < string(results[j].SqlDigest) }) - getTotalCPUTime := func(record *tipb.CPUTimeRecord) int { + getTotalCPUTime := func(record *tipb.TopSQLRecord) int { total := uint32(0) for _, v := range record.RecordListCpuTimeMs { total += v @@ -335,7 +338,7 @@ func TestCollectCapacity(t *testing.T) { variable.TopSQLVariable.MaxStatementCount.Store(5000) collectedData := make(map[string]*dataPoints) - tsr.doCollect(collectedData, 1, genRecord(20000)) + tsr.doCollect(collectedData, map[uint64]map[stmtstats.SQLPlanDigest]struct{}{}, 1, genRecord(20000)) require.Equal(t, 5001, len(collectedData)) require.Equal(t, int64(20000), tsr.sqlMapLength.Load()) require.Equal(t, int64(20000), tsr.planMapLength.Load()) @@ -355,10 +358,13 @@ func TestCollectOthers(t *testing.T) { require.Equal(t, uint64(60), others.CPUTimeMsTotal) // test for time jump backward. - evict := &dataPoints{} + evict := &dataPoints{tsIndex: map[uint64]int{}} evict.TimestampList = []uint64{3, 2, 4} evict.CPUTimeMsList = []uint32{30, 20, 40} evict.CPUTimeMsTotal = 90 + evict.StmtExecCountList = []uint64{0, 0, 0} + evict.StmtKvExecCountList = []map[string]uint64{nil, nil, nil} + evict.StmtDurationSumNsList = []uint64{0, 0, 0} others = addEvictedIntoSortedDataPoints(others, evict) require.Equal(t, uint64(150), others.CPUTimeMsTotal) require.Equal(t, []uint64{1, 2, 3, 4}, others.TimestampList) @@ -376,9 +382,16 @@ func TestDataPoints(t *testing.T) { d = &dataPoints{} d.TimestampList = []uint64{1, 2, 5, 6, 3, 4} d.CPUTimeMsList = []uint32{10, 20, 50, 60, 30, 40} + d.StmtExecCountList = []uint64{11, 12, 13, 14, 15, 16} + d.StmtKvExecCountList = []map[string]uint64{{"": 21}, {"": 22}, {"": 23}, {"": 24}, {"": 25}, {"": 26}} + d.StmtDurationSumNsList = []uint64{31, 32, 33, 34, 35, 36} + d.rebuildTsIndex() sort.Sort(d) require.Equal(t, []uint64{1, 2, 3, 4, 5, 6}, d.TimestampList) require.Equal(t, []uint32{10, 20, 30, 40, 50, 60}, d.CPUTimeMsList) + require.Equal(t, []uint64{11, 12, 15, 16, 13, 14}, d.StmtExecCountList) + require.Equal(t, []map[string]uint64{{"": 21}, {"": 22}, {"": 25}, {"": 26}, {"": 23}, {"": 24}}, d.StmtKvExecCountList) + require.Equal(t, []uint64{31, 32, 35, 36, 33, 34}, d.StmtDurationSumNsList) // test for dataPoints merge. d = &dataPoints{} @@ -387,6 +400,10 @@ func TestDataPoints(t *testing.T) { evict.TimestampList = []uint64{1, 3} evict.CPUTimeMsList = []uint32{10, 30} evict.CPUTimeMsTotal = 40 + evict.StmtExecCountList = []uint64{0, 0} + evict.StmtKvExecCountList = []map[string]uint64{{}, {}} + evict.StmtDurationSumNsList = []uint64{0, 0} + evict.rebuildTsIndex() addEvictedIntoSortedDataPoints(d, evict) require.Equal(t, uint64(40), d.CPUTimeMsTotal) require.Equal(t, []uint64{1, 3}, d.TimestampList) @@ -395,6 +412,10 @@ func TestDataPoints(t *testing.T) { evict.TimestampList = []uint64{1, 2, 3, 4, 5} evict.CPUTimeMsList = []uint32{10, 20, 30, 40, 50} evict.CPUTimeMsTotal = 150 + evict.StmtExecCountList = []uint64{0, 0, 0, 0, 0} + evict.StmtKvExecCountList = []map[string]uint64{{}, {}, {}, {}, {}} + evict.StmtDurationSumNsList = []uint64{0, 0, 0, 0, 0} + evict.rebuildTsIndex() addEvictedIntoSortedDataPoints(d, evict) require.Equal(t, uint64(190), d.CPUTimeMsTotal) require.Equal(t, []uint64{1, 2, 3, 4, 5}, d.TimestampList) @@ -406,6 +427,10 @@ func TestDataPoints(t *testing.T) { evict.TimestampList = []uint64{3, 2} evict.CPUTimeMsList = []uint32{30, 20} evict.CPUTimeMsTotal = 50 + evict.StmtExecCountList = []uint64{0, 0} + evict.StmtKvExecCountList = []map[string]uint64{{}, {}} + evict.StmtDurationSumNsList = []uint64{0, 0} + evict.rebuildTsIndex() addEvictedIntoSortedDataPoints(d, evict) require.Equal(t, uint64(50), d.CPUTimeMsTotal) require.Equal(t, []uint64{2, 3}, d.TimestampList) @@ -487,12 +512,11 @@ func TestMultipleDataSinks(t *testing.T) { for _, ch := range chs { d := <-ch require.NotNil(t, d) - require.Equal(t, []tipb.CPUTimeRecord{{ - SqlDigest: []byte("sqlDigest1"), - PlanDigest: []byte("planDigest1"), - RecordListTimestampSec: []uint64{3}, - RecordListCpuTimeMs: []uint32{2}, - }}, d.CPUTimeRecords) + require.Len(t, d.DataRecords, 1) + require.Equal(t, []byte("sqlDigest1"), d.DataRecords[0].SqlDigest) + require.Equal(t, []byte("planDigest1"), d.DataRecords[0].PlanDigest) + require.Equal(t, []uint64{3}, d.DataRecords[0].RecordListTimestampSec) + require.Equal(t, []uint32{2}, d.DataRecords[0].RecordListCpuTimeMs) require.Equal(t, []tipb.SQLMeta{{ SqlDigest: []byte("sqlDigest1"), @@ -518,12 +542,11 @@ func TestMultipleDataSinks(t *testing.T) { for i := 1; i < 7; i += 2 { d := <-chs[i] require.NotNil(t, d) - require.Equal(t, []tipb.CPUTimeRecord{{ - SqlDigest: []byte("sqlDigest4"), - PlanDigest: []byte("planDigest4"), - RecordListTimestampSec: []uint64{6}, - RecordListCpuTimeMs: []uint32{5}, - }}, d.CPUTimeRecords) + require.Len(t, d.DataRecords, 1) + require.Equal(t, []byte("sqlDigest4"), d.DataRecords[0].SqlDigest) + require.Equal(t, []byte("planDigest4"), d.DataRecords[0].PlanDigest) + require.Equal(t, []uint64{6}, d.DataRecords[0].RecordListTimestampSec) + require.Equal(t, []uint32{5}, d.DataRecords[0].RecordListCpuTimeMs) require.Equal(t, []tipb.SQLMeta{{ SqlDigest: []byte("sqlDigest4"), @@ -563,3 +586,313 @@ func BenchmarkTopSQL_CollectAndEvict(b *testing.B) { populateCache(tsr, begin, end, uint64(i)) } } + +func TestStmtStatsReport(t *testing.T) { + variable.TopSQLVariable.MaxStatementCount.Store(1) + + tsr := NewRemoteTopSQLReporter(mockPlanBinaryDecoderFunc) + tsr.Close() // manual control + + r := tsr.getReportData(collectedData{ + normalizedSQLMap: &sync.Map{}, + normalizedPlanMap: &sync.Map{}, + records: map[string]*dataPoints{ + "S1P1": { + SQLDigest: []byte("S1"), + PlanDigest: []byte("P1"), + TimestampList: []uint64{1, 2, 3, 4}, + CPUTimeMsList: []uint32{11, 12, 13, 14}, + CPUTimeMsTotal: 11 + 12 + 13 + 14, + StmtExecCountList: []uint64{11, 12, 13, 14}, + StmtKvExecCountList: []map[string]uint64{{"": 11}, {"": 12}, {"": 13}, {"": 14}}, + StmtDurationSumNsList: []uint64{11, 12, 13, 14}, + }, + "S2P2": { + SQLDigest: []byte("S2"), + PlanDigest: []byte("P2"), + TimestampList: []uint64{1, 2, 3, 4}, + CPUTimeMsList: []uint32{21, 22, 23, 24}, + CPUTimeMsTotal: 21 + 22 + 23 + 24, + StmtExecCountList: []uint64{21, 22, 23, 24}, + StmtKvExecCountList: []map[string]uint64{{"": 21}, {"": 22}, {"": 23}, {"": 24}}, + StmtDurationSumNsList: []uint64{21, 22, 23, 24}, + }, + keyOthers: { + SQLDigest: []byte(nil), + PlanDigest: []byte(nil), + TimestampList: []uint64{1, 2, 3, 4}, + CPUTimeMsList: []uint32{91, 92, 93, 94}, + CPUTimeMsTotal: 91 + 92 + 93 + 94, + StmtExecCountList: []uint64{91, 92, 93, 94}, + StmtKvExecCountList: []map[string]uint64{{"": 91}, {"": 92}, {"": 93}, {"": 94}}, + StmtDurationSumNsList: []uint64{91, 92, 93, 94}, + }, + }, + }) + assert.True(t, r.hasData()) + assert.Len(t, r.DataRecords, 2) + + s2p2 := r.DataRecords[0] + assert.Equal(t, []byte("S2"), s2p2.SqlDigest) + assert.Equal(t, []byte("P2"), s2p2.PlanDigest) + assert.Equal(t, []uint64{1, 2, 3, 4}, s2p2.RecordListTimestampSec) + assert.Equal(t, []uint32{21, 22, 23, 24}, s2p2.RecordListCpuTimeMs) + assert.Equal(t, []uint64{21, 22, 23, 24}, s2p2.RecordListStmtExecCount) + assert.Equal(t, []uint64{21, 22, 23, 24}, s2p2.RecordListStmtDurationSumNs) + assert.Equal(t, []*tipb.TopSQLStmtKvExecCount{ + {ExecCount: map[string]uint64{"": 21}}, + {ExecCount: map[string]uint64{"": 22}}, + {ExecCount: map[string]uint64{"": 23}}, + {ExecCount: map[string]uint64{"": 24}}, + }, s2p2.RecordListStmtKvExecCount) + + others := r.DataRecords[1] + assert.Equal(t, []byte(nil), others.SqlDigest) + assert.Equal(t, []byte(nil), others.PlanDigest) + assert.Equal(t, []uint64{1, 2, 3, 4}, others.RecordListTimestampSec) + assert.Equal(t, []uint32{91 + 11, 92 + 12, 93 + 13, 94 + 14}, others.RecordListCpuTimeMs) + assert.Equal(t, []uint64{91 + 11, 92 + 12, 93 + 13, 94 + 14}, others.RecordListStmtExecCount) + assert.Equal(t, []uint64{91 + 11, 92 + 12, 93 + 13, 94 + 14}, others.RecordListStmtDurationSumNs) + assert.Equal(t, []*tipb.TopSQLStmtKvExecCount{ + {ExecCount: map[string]uint64{"": 91 + 11}}, + {ExecCount: map[string]uint64{"": 92 + 12}}, + {ExecCount: map[string]uint64{"": 93 + 13}}, + {ExecCount: map[string]uint64{"": 94 + 14}}, + }, others.RecordListStmtKvExecCount) +} + +func TestStmtStatsCollect(t *testing.T) { + variable.TopSQLVariable.MaxStatementCount.Store(1000) + + tsr := NewRemoteTopSQLReporter(mockPlanBinaryDecoderFunc) + tsr.Close() // manual control + + total := map[string]*dataPoints{} + evicted := map[uint64]map[stmtstats.SQLPlanDigest]struct{}{} + // TimestampList: [] + // CPUTimeList: [] + // ExecCountList: [] + // KvExecCountList: [] + + collectCPUTime(tsr, total, evicted, "S1", "P1", 1, 1) + // TimestampList: [1] + // CPUTimeList: [1] + // ExecCountList: [0] + // KvExecCountList: [0] + + collectStmtStats(tsr, total, evicted, "S1", "P1", 1, 2, map[string]uint64{"": 3}) + // TimestampList: [1] + // CPUTimeList: [1] + // ExecCountList: [2] + // KvExecCountList: [3] + + collectCPUTime(tsr, total, evicted, "S1", "P1", 2, 1) + // TimestampList: [1, 2] + // CPUTimeList: [1, 1] + // ExecCountList: [2, 0] + // KvExecCountList: [3, 0] + + collectCPUTime(tsr, total, evicted, "S1", "P1", 3, 1) + // TimestampList: [1, 2, 3] + // CPUTimeList: [1, 1, 1] + // ExecCountList: [2, 0, 0] + // KvExecCountList: [3, 0, 0] + + collectStmtStats(tsr, total, evicted, "S1", "P1", 3, 2, map[string]uint64{"": 3}) + // TimestampList: [1, 2, 3] + // CPUTimeList: [1, 1, 1] + // ExecCountList: [2, 0, 2] + // KvExecCountList: [3, 0, 3] + + collectStmtStats(tsr, total, evicted, "S1", "P1", 2, 2, map[string]uint64{"": 3}) + // TimestampList: [1, 2, 3] + // CPUTimeList: [1, 1, 1] + // ExecCountList: [2, 2, 2] + // KvExecCountList: [3, 3, 3] + + assert.Empty(t, evicted) + data, ok := total["S1P1"] + assert.True(t, ok) + assert.Equal(t, []byte("S1"), data.SQLDigest) + assert.Equal(t, []byte("P1"), data.PlanDigest) + assert.Equal(t, uint64(3), data.CPUTimeMsTotal) + assert.Equal(t, []uint64{1, 2, 3}, data.TimestampList) + assert.Equal(t, []uint32{1, 1, 1}, data.CPUTimeMsList) + assert.Equal(t, []uint64{2, 2, 2}, data.StmtExecCountList) + assert.Equal(t, []map[string]uint64{{"": 3}, {"": 3}, {"": 3}}, data.StmtKvExecCountList) +} + +func TestStmtStatsCollectEvicted(t *testing.T) { + variable.TopSQLVariable.MaxStatementCount.Store(2) + + tsr := NewRemoteTopSQLReporter(mockPlanBinaryDecoderFunc) + tsr.Close() // manual control + + total := map[string]*dataPoints{} + evicted := map[uint64]map[stmtstats.SQLPlanDigest]struct{}{} + + tsr.doCollect(total, evicted, 1, []tracecpu.SQLCPUTimeRecord{ + {SQLDigest: []byte("S1"), PlanDigest: []byte("P1"), CPUTimeMs: 1}, + {SQLDigest: []byte("S2"), PlanDigest: []byte("P2"), CPUTimeMs: 2}, + {SQLDigest: []byte("S3"), PlanDigest: []byte("P3"), CPUTimeMs: 3}, + }) + // S2P2: + // TimestampList: [1] + // CPUTimeList: [2] + // ExecCountList: [0] + // KvExecCountList: [0] + // + // S3P3: + // TimestampList: [1] + // CPUTimeList: [3] + // ExecCountList: [0] + // KvExecCountList: [0] + // + // others: + // TimestampList: [1] + // CPUTimeList: [1] + // ExecCountList: [0] + // KvExecCountList: [0] + // + // evicted: {1: S1P1} + + collectStmtStats(tsr, total, evicted, "S1", "P1", 1, 1, map[string]uint64{"": 1}) + // S2P2: + // TimestampList: [1] + // CPUTimeList: [2] + // ExecCountList: [0] + // KvExecCountList: [0] + // + // S3P3: + // TimestampList: [1] + // CPUTimeList: [3] + // ExecCountList: [0] + // KvExecCountList: [0] + // + // others: + // TimestampList: [1] + // CPUTimeList: [1] + // ExecCountList: [1] + // KvExecCountList: [1] + // + // evicted: {1: S1P1} + + collectStmtStats(tsr, total, evicted, "S2", "P2", 1, 2, map[string]uint64{"": 2}) + // S2P2: + // TimestampList: [1] + // CPUTimeList: [2] + // ExecCountList: [2] + // KvExecCountList: [2] + // + // S3P3: + // TimestampList: [1] + // CPUTimeList: [3] + // ExecCountList: [0] + // KvExecCountList: [0] + // + // others: + // TimestampList: [1] + // CPUTimeList: [1] + // ExecCountList: [1] + // KvExecCountList: [1] + // + // evicted: {1: S1P1} + + collectStmtStats(tsr, total, evicted, "S3", "P3", 1, 3, map[string]uint64{"": 3}) + // S2P2: + // TimestampList: [1] + // CPUTimeList: [2] + // ExecCountList: [2] + // KvExecCountList: [2] + // + // S3P3: + // TimestampList: [1] + // CPUTimeList: [3] + // ExecCountList: [3] + // KvExecCountList: [3] + // + // others: + // TimestampList: [1] + // CPUTimeList: [1] + // ExecCountList: [1] + // KvExecCountList: [1] + // + // evicted: {1: S1P1} + + assert.Len(t, evicted, 1) + m, ok := evicted[1] + assert.True(t, ok) + _, ok = m[stmtstats.SQLPlanDigest{SQLDigest: "S1", PlanDigest: "P1"}] + assert.True(t, ok) + _, ok = total["S1P1"] + assert.False(t, ok) + + s2p2, ok := total["S2P2"] + assert.True(t, ok) + assert.Equal(t, []byte("S2"), s2p2.SQLDigest) + assert.Equal(t, []byte("P2"), s2p2.PlanDigest) + assert.Equal(t, uint64(2), s2p2.CPUTimeMsTotal) + assert.Equal(t, []uint64{1}, s2p2.TimestampList) + assert.Equal(t, []uint32{2}, s2p2.CPUTimeMsList) + assert.Equal(t, []uint64{2}, s2p2.StmtExecCountList) + assert.Equal(t, []map[string]uint64{{"": 2}}, s2p2.StmtKvExecCountList) + + s3p3, ok := total["S3P3"] + assert.True(t, ok) + assert.Equal(t, []byte("S3"), s3p3.SQLDigest) + assert.Equal(t, []byte("P3"), s3p3.PlanDigest) + assert.Equal(t, uint64(3), s3p3.CPUTimeMsTotal) + assert.Equal(t, []uint64{1}, s3p3.TimestampList) + assert.Equal(t, []uint32{3}, s3p3.CPUTimeMsList) + assert.Equal(t, []uint64{3}, s3p3.StmtExecCountList) + assert.Equal(t, []map[string]uint64{{"": 3}}, s3p3.StmtKvExecCountList) + + others, ok := total[keyOthers] + assert.True(t, ok) + assert.Equal(t, []byte(nil), others.SQLDigest) + assert.Equal(t, []byte(nil), others.PlanDigest) + assert.Equal(t, uint64(1), others.CPUTimeMsTotal) + assert.Equal(t, []uint64{1}, others.TimestampList) + assert.Equal(t, []uint32{1}, others.CPUTimeMsList) + assert.Equal(t, []uint64{1}, others.StmtExecCountList) + assert.Equal(t, []map[string]uint64{{"": 1}}, others.StmtKvExecCountList) +} + +func collectCPUTime( + tsr *RemoteTopSQLReporter, + total map[string]*dataPoints, + evicted map[uint64]map[stmtstats.SQLPlanDigest]struct{}, + sqlDigest, planDigest string, + ts uint64, + cpuTime uint32) { + tsr.doCollect(total, evicted, ts, []tracecpu.SQLCPUTimeRecord{{ + SQLDigest: []byte(sqlDigest), + PlanDigest: []byte(planDigest), + CPUTimeMs: cpuTime, + }}) +} + +func collectStmtStats( + tsr *RemoteTopSQLReporter, + total map[string]*dataPoints, + evicted map[uint64]map[stmtstats.SQLPlanDigest]struct{}, + sqlDigest, planDigest string, + ts int64, + execCount uint64, + kvExecCount map[string]uint64) { + tsr.doCollectStmtRecords(total, evicted, []stmtstats.StatementStatsRecord{{ + Timestamp: ts, + Data: stmtstats.StatementStatsMap{ + stmtstats.SQLPlanDigest{ + SQLDigest: stmtstats.BinaryDigest(sqlDigest), + PlanDigest: stmtstats.BinaryDigest(planDigest), + }: &stmtstats.StatementStatsItem{ + ExecCount: execCount, + KvStatsItem: stmtstats.KvStatementStatsItem{ + KvExecCount: kvExecCount, + }, + // TODO(mornyx): add duration + }, + }, + }}) +} diff --git a/util/topsql/reporter/single_target.go b/util/topsql/reporter/single_target.go index 3744702ba26d6..c0af5f59c5bd8 100644 --- a/util/topsql/reporter/single_target.go +++ b/util/topsql/reporter/single_target.go @@ -207,7 +207,7 @@ func (ds *SingleTargetDataSink) doSend(addr string, task sendTask) { }() go func() { defer wg.Done() - errCh <- ds.sendBatchCPUTimeRecord(ctx, task.data.CPUTimeRecords) + errCh <- ds.sendBatchTopSQLRecord(ctx, task.data.DataRecords) }() wg.Wait() close(errCh) @@ -218,8 +218,8 @@ func (ds *SingleTargetDataSink) doSend(addr string, task sendTask) { } } -// sendBatchCPUTimeRecord sends a batch of TopSQL records by stream. -func (ds *SingleTargetDataSink) sendBatchCPUTimeRecord(ctx context.Context, records []tipb.CPUTimeRecord) (err error) { +// sendBatchTopSQLRecord sends a batch of TopSQL records by stream. +func (ds *SingleTargetDataSink) sendBatchTopSQLRecord(ctx context.Context, records []tipb.TopSQLRecord) (err error) { if len(records) == 0 { return nil } @@ -236,7 +236,7 @@ func (ds *SingleTargetDataSink) sendBatchCPUTimeRecord(ctx context.Context, reco }() client := tipb.NewTopSQLAgentClient(ds.conn) - stream, err := client.ReportCPUTimeRecords(ctx) + stream, err := client.ReportTopSQLRecords(ctx) if err != nil { return err } diff --git a/util/topsql/stmtstats/aggregator.go b/util/topsql/stmtstats/aggregator.go index d78ed7b62dafb..db19731b607f9 100644 --- a/util/topsql/stmtstats/aggregator.go +++ b/util/topsql/stmtstats/aggregator.go @@ -81,10 +81,12 @@ func (m *aggregator) aggregate() { r.Data.Merge(stats.Take()) return true }) - m.collectors.Range(func(c, _ interface{}) bool { - c.(Collector).CollectStmtStatsRecords([]StatementStatsRecord{r}) - return true - }) + if len(r.Data) > 0 { + m.collectors.Range(func(c, _ interface{}) bool { + c.(Collector).CollectStmtStatsRecords([]StatementStatsRecord{r}) + return true + }) + } } // register binds StatementStats to aggregator. diff --git a/util/topsql/topsql.go b/util/topsql/topsql.go index ccc1f61eef0a5..cb1199a00c02a 100644 --- a/util/topsql/topsql.go +++ b/util/topsql/topsql.go @@ -53,6 +53,7 @@ func SetupTopSQL() { tracecpu.GlobalSQLCPUProfiler.SetCollector(remoteReporter) tracecpu.GlobalSQLCPUProfiler.Run() + stmtstats.RegisterCollector(remoteReporter) stmtstats.SetupAggregator() } From 8eaada23fd7154e64aa5d1aed3113568422d18aa Mon Sep 17 00:00:00 2001 From: mornyx Date: Thu, 23 Dec 2021 01:15:37 +0800 Subject: [PATCH 02/15] Replace go.mod Signed-off-by: mornyx --- go.mod | 2 +- go.sum | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/go.mod b/go.mod index d85b17cd96f5b..d1feb554525d9 100644 --- a/go.mod +++ b/go.mod @@ -104,4 +104,4 @@ replace github.com/pingcap/tidb/parser => ./parser replace github.com/dgrijalva/jwt-go => github.com/form3tech-oss/jwt-go v3.2.6-0.20210809144907-32ab6a8243d7+incompatible // TODO(mornyx): remove -replace github.com/pingcap/tipb => /Users/mornyx/Desktop/repos/tipb +replace github.com/pingcap/tipb => github.com/mornyx/tipb v0.0.0-20210917081614-311f2369c5f7 diff --git a/go.sum b/go.sum index eca38cd7d38a5..19168d02bba56 100644 --- a/go.sum +++ b/go.sum @@ -511,6 +511,8 @@ github.com/modern-go/reflect2 v1.0.1 h1:9f412s+6RmYXLWZSEzVVgPGK7C2PphHj5RJrvfx9 github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= github.com/montanaflynn/stats v0.5.0 h1:2EkzeTSqBB4V4bJwWrt5gIIrZmpJBcoIRGS2kWLgzmk= github.com/montanaflynn/stats v0.5.0/go.mod h1:wL8QJuTMNUDYhXwkmfOly8iTdp5TEcJFWZD2D7SIkUc= +github.com/mornyx/tipb v0.0.0-20210917081614-311f2369c5f7 h1:eL/037j3+9+yCeq/7zzMm9yOjUKzs5q4TTTz2H/IQ1M= +github.com/mornyx/tipb v0.0.0-20210917081614-311f2369c5f7/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= github.com/moul/http2curl v1.0.0/go.mod h1:8UbvGypXm98wA/IqH45anm5Y2Z6ep6O31QGOAZ3H0fQ= github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/nats-io/nats.go v1.8.1/go.mod h1:BrFz9vVn0fU3AcH9Vn4Kd7W0NpJ651tD5omQ3M8LwxM= From 3381e6ca9b2149a411a727e01b61eb51c6c22b4f Mon Sep 17 00:00:00 2001 From: mornyx Date: Thu, 23 Dec 2021 01:44:35 +0800 Subject: [PATCH 03/15] Upgrade tipb Signed-off-by: mornyx --- go.mod | 2 +- go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/go.mod b/go.mod index d1feb554525d9..5e88cc1f041f0 100644 --- a/go.mod +++ b/go.mod @@ -104,4 +104,4 @@ replace github.com/pingcap/tidb/parser => ./parser replace github.com/dgrijalva/jwt-go => github.com/form3tech-oss/jwt-go v3.2.6-0.20210809144907-32ab6a8243d7+incompatible // TODO(mornyx): remove -replace github.com/pingcap/tipb => github.com/mornyx/tipb v0.0.0-20210917081614-311f2369c5f7 +replace github.com/pingcap/tipb => github.com/mornyx/tipb v0.0.0-20211222170402-84c2de94b2be diff --git a/go.sum b/go.sum index 19168d02bba56..8340a44f3e80d 100644 --- a/go.sum +++ b/go.sum @@ -511,8 +511,8 @@ github.com/modern-go/reflect2 v1.0.1 h1:9f412s+6RmYXLWZSEzVVgPGK7C2PphHj5RJrvfx9 github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= github.com/montanaflynn/stats v0.5.0 h1:2EkzeTSqBB4V4bJwWrt5gIIrZmpJBcoIRGS2kWLgzmk= github.com/montanaflynn/stats v0.5.0/go.mod h1:wL8QJuTMNUDYhXwkmfOly8iTdp5TEcJFWZD2D7SIkUc= -github.com/mornyx/tipb v0.0.0-20210917081614-311f2369c5f7 h1:eL/037j3+9+yCeq/7zzMm9yOjUKzs5q4TTTz2H/IQ1M= -github.com/mornyx/tipb v0.0.0-20210917081614-311f2369c5f7/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= +github.com/mornyx/tipb v0.0.0-20211222170402-84c2de94b2be h1:0CdtbnVAi+bJLkiVpGEj5o6KfpEZXFQEy83Fnzie06w= +github.com/mornyx/tipb v0.0.0-20211222170402-84c2de94b2be/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= github.com/moul/http2curl v1.0.0/go.mod h1:8UbvGypXm98wA/IqH45anm5Y2Z6ep6O31QGOAZ3H0fQ= github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/nats-io/nats.go v1.8.1/go.mod h1:BrFz9vVn0fU3AcH9Vn4Kd7W0NpJ651tD5omQ3M8LwxM= From 81136673679adfa99779de63b36654f351579e3b Mon Sep 17 00:00:00 2001 From: mornyx Date: Sun, 26 Dec 2021 21:26:33 +0800 Subject: [PATCH 04/15] Refactor topsql/reporter Signed-off-by: mornyx --- util/topsql/reporter/datamodel.go | 672 ++++++++++++++++++ util/topsql/reporter/datamodel_test.go | 466 +++++++++++++ util/topsql/reporter/datasink.go | 115 +++- util/topsql/reporter/datasink_test.go | 67 ++ util/topsql/reporter/metrics.go | 36 + util/topsql/reporter/reporter.go | 906 ++++--------------------- util/topsql/reporter/reporter_test.go | 438 +----------- util/topsql/reporter/single_target.go | 6 + util/topsql/topsql_test.go | 6 +- util/topsql/tracecpu/mock/mock.go | 3 + 10 files changed, 1500 insertions(+), 1215 deletions(-) create mode 100644 util/topsql/reporter/datamodel.go create mode 100644 util/topsql/reporter/datamodel_test.go create mode 100644 util/topsql/reporter/datasink_test.go create mode 100644 util/topsql/reporter/metrics.go diff --git a/util/topsql/reporter/datamodel.go b/util/topsql/reporter/datamodel.go new file mode 100644 index 0000000000000..ef150a96f732e --- /dev/null +++ b/util/topsql/reporter/datamodel.go @@ -0,0 +1,672 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package reporter + +import ( + "bytes" + "sort" + "sync" + "sync/atomic" + + "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/util/logutil" + "github.com/pingcap/tidb/util/topsql/stmtstats" + "github.com/pingcap/tidb/util/topsql/tracecpu" + "github.com/pingcap/tipb/go-tipb" + "github.com/wangjohn/quickselect" + atomic2 "go.uber.org/atomic" + "go.uber.org/zap" +) + +// Data naming and relationship in datamodel.go: +// +// tsItem: +// - timestamp +// - cpuTime +// - stmtStats(execCount, durationSum, ...) +// +// tsItems: [ tsItem | tsItem | tsItem | ... ] +// +// record: +// - tsItems: [ tsItem | tsItem | tsItem | ... ] +// - tsIndex: { 1640500000 => 0 | 1640500001 => 1 | 1640500002 => 2 | ... } +// +// records: [ record | record | record | ... ] +// +// collecting: +// - records: { sqlPlanDigest => record | sqlPlanDigest => record | ... } +// - evicted: { sqlPlanDigest | sqlPlanDigest | ... } +// +// cpuRecords: [ tracecpu.SQLCPUTimeRecord | tracecpu.SQLCPUTimeRecord | ... ] +// +// normalizeSQLMap: { sqlDigest => normalizedSQL | sqlDigest => normalizedSQL | ... } +// +// normalizePlanMap: { planDigest => normalizedPlan | planDigest => normalizedPlan | ... } + +// keyOthers is the key to store the aggregation of all records that is out of Top N. +const keyOthers = "" + +// tsItem is a self-contained complete piece of data for a certain timestamp. +type tsItem struct { + timestamp uint64 + cpuTimeMs uint32 + stmtStats stmtstats.StatementStatsItem +} + +func newTsItem() tsItem { + return tsItem{ + stmtStats: stmtstats.StatementStatsItem{ + KvStatsItem: stmtstats.KvStatementStatsItem{ + KvExecCount: map[string]uint64{}, + }, + }, + } +} + +var _ sort.Interface = &tsItems{} + +// tsItems is a sortable list of tsItem, sort by tsItem.timestamp (asc). +type tsItems []tsItem + +func (ts tsItems) Len() int { + return len(ts) +} + +func (ts tsItems) Less(i, j int) bool { + return ts[i].timestamp < ts[j].timestamp +} + +func (ts tsItems) Swap(i, j int) { + ts[i], ts[j] = ts[j], ts[i] +} + +var _ sort.Interface = &record{} + +// record represents the cumulative tsItem in current minute window. +// record do not guarantee the tsItems is sorted by timestamp when there is a time jump backward. +// record is also sortable, and the tsIndex will be updated while sorting the internal tsItems. +type record struct { + sqlDigest []byte + planDigest []byte + totalCPUTimeMs uint64 + tsItems tsItems + + // tsIndex is used to quickly find the corresponding tsItems index through timestamp. + tsIndex map[uint64]int // timestamp => index of tsItems +} + +func newRecord(sqlDigest, planDigest []byte) *record { + listCap := variable.TopSQLVariable.ReportIntervalSeconds.Load()/variable.TopSQLVariable.PrecisionSeconds.Load() + 1 + return &record{ + sqlDigest: sqlDigest, + planDigest: planDigest, + tsItems: make(tsItems, 0, listCap), + tsIndex: make(map[uint64]int, listCap), + } +} + +func (r *record) Len() int { + return r.tsItems.Len() +} + +func (r *record) Less(i, j int) bool { + return r.tsItems.Less(i, j) +} + +func (r *record) Swap(i, j int) { + // before swap: + // timestamps: [10000, 10001, 10002] + // tsIndex: [10000 => 0, 10001 => 1, 10002 => 2] + // + // let i = 0, j = 1 + // after swap tsIndex: + // timestamps: [10000, 10001, 10002] + // tsIndex: [10000 => 1, 10001 => 0, 10002 => 2] + // + // after swap tsItems: + // timestamps: [10001, 10000, 10002] + // tsIndex: [10000 => 1, 10001 => 0, 10002 => 2] + r.tsIndex[r.tsItems[i].timestamp], r.tsIndex[r.tsItems[j].timestamp] = r.tsIndex[r.tsItems[j].timestamp], r.tsIndex[r.tsItems[i].timestamp] + r.tsItems.Swap(i, j) +} + +// appendCPUTime appends a cpuTime under a certain timestamp to record. +// If the timestamp already exists in tsItems, then cpuTime will be replaced. +func (r *record) appendCPUTime(timestamp uint64, cpuTimeMs uint32) { + if index, ok := r.tsIndex[timestamp]; ok { + // For the same timestamp, we have already called appendStmtStatsItem, + // r.tsItems already exists the corresponding timestamp, and the corresponding + // cpuTimeMs has been set to 0, so we directly replace it. + // + // let timestamp = 10000, cpuTimeMs = 123 + // + // Before: + // tsIndex: [10000 => 0] + // tsItems: + // timestamp: [10000] + // cpuTimeMs: [0] + // stmtStats.ExecCount: [?] + // stmtStats.KvExecCount: [map{"?": ?}] + // stmtStats.DurationSum: [?] + // + // After: + // tsIndex: [10000 => 0] + // tsItems: + // timestamp: [10000] + // cpuTimeMs: [123] + // stmtStats.ExecCount: [?] + // stmtStats.KvExecCount: [map{"?": ?}] + // stmtStats.DurationSum: [?] + // + r.tsItems[index].cpuTimeMs = cpuTimeMs + } else { + // For this timestamp, we have not appended any tsItem, so append it directly. + // Other fields in tsItem except cpuTimeMs will be initialized to 0. + // + // let timestamp = 10000, cpu_time = 123 + // + // Before: + // tsIndex: [] + // tsItems: + // timestamp: [] + // cpuTimeMs: [] + // stmtStats.ExecCount: [] + // stmtStats.KvExecCount: [] + // stmtStats.DurationSum: [] + // + // After: + // tsIndex: [10000 => 0] + // tsItems: + // timestamp: [10000] + // cpuTimeMs: [123] + // stmtStats.ExecCount: [0] + // stmtStats.KvExecCount: [map{}] + // stmtStats.DurationSum: [0] + // + newItem := newTsItem() + newItem.timestamp = timestamp + newItem.cpuTimeMs = cpuTimeMs + r.tsIndex[timestamp] = len(r.tsItems) + r.tsItems = append(r.tsItems, newItem) + } + r.totalCPUTimeMs += uint64(cpuTimeMs) +} + +// appendStmtStatsItem appends a stmtstats.StatementStatsItem under a certain timestamp to record. +// If the timestamp already exists in tsItems, then stmtstats.StatementStatsItem will be replaced. +func (r *record) appendStmtStatsItem(timestamp uint64, item stmtstats.StatementStatsItem) { + if index, ok := r.tsIndex[timestamp]; ok { + // For the same timestamp, we have already called appendCPUTime, + // r.tsItems already exists the corresponding timestamp, and the + // corresponding stmtStats has been set to 0, so we directly replace it. + // + // let timestamp = 10000, execCount = 123, kvExecCount = map{"1.1.1.1:1": 123}, durationSum = 456 + // + // Before: + // tsIndex: [10000 => 0] + // tsItems: + // timestamp: [10000] + // cpuTimeMs: [?] + // stmtStats.ExecCount: [0] + // stmtStats.KvExecCount: [map{}] + // stmtStats.DurationSum: [0] + // + // After: + // tsIndex: [10000 => 0] + // tsItems: + // timestamp: [10000] + // cpuTimeMs: [?] + // stmtStats.ExecCount: [123] + // stmtStats.KvExecCount: [map{"1.1.1.1:1": 123}] + // stmtStats.DurationSum: [456] + // + r.tsItems[index].stmtStats = item + } else { + // For this timestamp, we have not appended any tsItem, so append it directly. + // Other fields in tsItem except stmtStats will be initialized to 0. + // + // let timestamp = 10000, execCount = 123, kvExecCount = map{"1.1.1.1:1": 123}, durationSum = 456 + // + // Before: + // tsIndex: [] + // tsItems: + // timestamp: [] + // cpuTimeMs: [] + // stmtStats.ExecCount: [] + // stmtStats.KvExecCount: [] + // stmtStats.DurationSum: [] + // + // After: + // tsIndex: [10000 => 0] + // tsItems: + // timestamp: [10000] + // cpuTimeMs: [0] + // stmtStats.ExecCount: [123] + // stmtStats.KvExecCount: [map{"1.1.1.1:1": 123}] + // stmtStats.DurationSum: [456] + // + newItem := newTsItem() + newItem.timestamp = timestamp + newItem.stmtStats = item + r.tsIndex[timestamp] = len(r.tsItems) + r.tsItems = append(r.tsItems, newItem) + } +} + +// merge other record into r. +// Attention, this function depend on r is sorted, and will sort `other` by timestamp. +func (r *record) merge(other *record) { + if other == nil || len(other.tsItems) == 0 { + return + } + sort.Sort(other) + if len(r.tsItems) == 0 { + r.totalCPUTimeMs = other.totalCPUTimeMs + r.tsItems = other.tsItems + r.tsIndex = other.tsIndex + return + } + length := len(r.tsItems) + len(other.tsItems) + newTsItems := make(tsItems, 0, length) + i, j := 0, 0 + for i < len(r.tsItems) && j < len(other.tsItems) { + if r.tsItems[i].timestamp == other.tsItems[j].timestamp { + newItem := newTsItem() + newItem.timestamp = r.tsItems[i].timestamp + newItem.cpuTimeMs = r.tsItems[i].cpuTimeMs + other.tsItems[j].cpuTimeMs + r.tsItems[i].stmtStats.Merge(&other.tsItems[j].stmtStats) + newItem.stmtStats = r.tsItems[i].stmtStats + newTsItems = append(newTsItems, newItem) + i++ + j++ + } else if r.tsItems[i].timestamp < other.tsItems[j].timestamp { + newItem := newTsItem() + newItem.timestamp = r.tsItems[i].timestamp + newItem.cpuTimeMs = r.tsItems[i].cpuTimeMs + newItem.stmtStats = r.tsItems[i].stmtStats + newTsItems = append(newTsItems, newItem) + i++ + } else { + newItem := newTsItem() + newItem.timestamp = other.tsItems[j].timestamp + newItem.cpuTimeMs = other.tsItems[j].cpuTimeMs + newItem.stmtStats = other.tsItems[j].stmtStats + newTsItems = append(newTsItems, newItem) + j++ + } + } + if i < len(r.tsItems) { + newTsItems = append(newTsItems, r.tsItems[i:]...) + } + if j < len(other.tsItems) { + newTsItems = append(newTsItems, other.tsItems[j:]...) + } + r.tsItems = newTsItems + r.totalCPUTimeMs += other.totalCPUTimeMs + r.rebuildTsIndex() +} + +// rebuildTsIndex rebuilds the entire tsIndex based on tsItems. +func (r *record) rebuildTsIndex() { + if len(r.tsItems) == 0 { + r.tsIndex = map[uint64]int{} + return + } + r.tsIndex = make(map[uint64]int, len(r.tsItems)) + for index, item := range r.tsItems { + r.tsIndex[item.timestamp] = index + } +} + +// toProto converts the record to the corresponding protobuf representation. +func (r *record) toProto() tipb.TopSQLRecord { + capacity := len(r.tsItems) + if capacity == 0 { + return tipb.TopSQLRecord{ + SqlDigest: r.sqlDigest, + PlanDigest: r.planDigest, + } + } + timestampList := make([]uint64, 0, capacity) + cpuTimeList := make([]uint32, 0, capacity) + execCountList := make([]uint64, 0, capacity) + durationSumList := make([]uint64, 0, capacity) + kvExecCountList := make([]*tipb.TopSQLStmtKvExecCount, 0, capacity) + for _, item := range r.tsItems { + timestampList = append(timestampList, item.timestamp) + cpuTimeList = append(cpuTimeList, item.cpuTimeMs) + execCountList = append(execCountList, item.stmtStats.ExecCount) + kvExecCountList = append(kvExecCountList, &tipb.TopSQLStmtKvExecCount{ + ExecCount: item.stmtStats.KvStatsItem.KvExecCount, + }) + // TODO: append duration + } + return tipb.TopSQLRecord{ + SqlDigest: r.sqlDigest, + PlanDigest: r.planDigest, + RecordListTimestampSec: timestampList, + RecordListCpuTimeMs: cpuTimeList, + RecordListStmtExecCount: execCountList, + RecordListStmtKvExecCount: kvExecCountList, + RecordListStmtDurationSumNs: durationSumList, + // Add more indicators here. + } +} + +var _ sort.Interface = &records{} + +// records is a sortable list of record, sort by record.totalCPUTimeMs (desc). +type records []record + +func (rs records) Len() int { + return len(rs) +} + +func (rs records) Less(i, j int) bool { + // Order by totalCPUTimeMs **DESC**. + return rs[i].totalCPUTimeMs > rs[j].totalCPUTimeMs +} + +func (rs records) Swap(i, j int) { + rs[i], rs[j] = rs[j], rs[i] +} + +// topN returns the largest n records (by record.totalCPUTimeMs), other +// records are returned as evicted. +func (rs records) topN(n int) (top, evicted records) { + if len(rs) <= n { + return rs, nil + } + if err := quickselect.QuickSelect(rs, n); err != nil { + return rs, nil + } + return rs[:n], rs[n:] +} + +// toProto converts the records to the corresponding protobuf representation. +func (rs records) toProto() []tipb.TopSQLRecord { + pb := make([]tipb.TopSQLRecord, 0, len(rs)) + for _, r := range rs { + pb = append(pb, r.toProto()) + } + return pb +} + +// collecting includes the collection of data being collected by the reporter. +type collecting struct { + records map[string]*record // sqlPlanDigest => record + evicted map[uint64]map[string]struct{} // { sqlPlanDigest } + keyBuf *bytes.Buffer +} + +func newCollecting() *collecting { + return &collecting{ + records: map[string]*record{}, + evicted: map[uint64]map[string]struct{}{}, + keyBuf: bytes.NewBuffer(make([]byte, 0, 64)), + } +} + +// getOrCreateRecord gets the record corresponding to sqlDigest + planDigest, if it +// does not exist, it will be created. +func (c *collecting) getOrCreateRecord(sqlDigest, planDigest []byte) *record { + key := encodeKey(c.keyBuf, sqlDigest, planDigest) + r, ok := c.records[key] + if !ok { + r = newRecord(sqlDigest, planDigest) + c.records[key] = r + } + return r +} + +// markAsEvicted marks sqlDigest + planDigest under a certain timestamp as "evicted". +// Later, we can determine whether a certain sqlDigest + planDigest within a certain +// timestamp has been evicted. +func (c *collecting) markAsEvicted(timestamp uint64, sqlDigest, planDigest []byte) { + if _, ok := c.evicted[timestamp]; !ok { + c.evicted[timestamp] = map[string]struct{}{} + } + c.evicted[timestamp][encodeKey(c.keyBuf, sqlDigest, planDigest)] = struct{}{} +} + +// hasEvicted determines whether a certain sqlDigest + planDigest has been evicted +// in a certain timestamp. +func (c *collecting) hasEvicted(timestamp uint64, sqlDigest, planDigest []byte) bool { + if digestSet, ok := c.evicted[timestamp]; ok { + if _, ok := digestSet[encodeKey(c.keyBuf, sqlDigest, planDigest)]; ok { + return true + } + } + return false +} + +// appendOthersCPUTime appends totalCPUTimeMs to a special record named "others". +func (c *collecting) appendOthersCPUTime(timestamp uint64, totalCPUTimeMs uint32) { + if totalCPUTimeMs == 0 { + return + } + others, ok := c.records[keyOthers] + if !ok { + others = newRecord(nil, nil) + c.records[keyOthers] = others + } + others.appendCPUTime(timestamp, totalCPUTimeMs) +} + +// appendOthersCPUTime appends stmtstats.StatementStatsItem to a special record named "others". +func (c *collecting) appendOthersStmtStatsItem(timestamp uint64, item stmtstats.StatementStatsItem) { + others, ok := c.records[keyOthers] + if !ok { + others = newRecord(nil, nil) + c.records[keyOthers] = others + } + others.appendStmtStatsItem(timestamp, item) +} + +// topN returns the largest N records, other records will be packed and appended to the end. +func (c *collecting) topN(n int) records { + others := c.records[keyOthers] + delete(c.records, keyOthers) + rs := make(records, 0, len(c.records)) + for _, v := range c.records { + rs = append(rs, *v) + } + // Fetch TopN records. + var evicted records + rs, evicted = rs.topN(n) + if others != nil { + // Sort the records by timestamp to fix the affect of time jump backward. + sort.Sort(others) + } else { + others = newRecord(nil, nil) + } + for _, e := range evicted { + others.merge(&e) + } + if others.totalCPUTimeMs > 0 { + // append others which summarize all evicted item's cpu-time. + rs = append(rs, *others) + } + return rs +} + +// take away all data inside collecting, put them in the returned new collecting. +func (c *collecting) take() *collecting { + r := &collecting{ + records: c.records, + evicted: c.evicted, + keyBuf: bytes.NewBuffer(make([]byte, 0, 64)), + } + c.records = map[string]*record{} + c.evicted = map[uint64]map[string]struct{}{} + return r +} + +// cpuRecords is a sortable list of tracecpu.SQLCPUTimeRecord, sort by CPUTimeMs (desc). +type cpuRecords []tracecpu.SQLCPUTimeRecord + +func (rs cpuRecords) Len() int { + return len(rs) +} + +func (rs cpuRecords) Less(i, j int) bool { + // Order by CPUTimeMs **DESC**. + return rs[i].CPUTimeMs > rs[j].CPUTimeMs +} + +func (rs cpuRecords) Swap(i, j int) { + rs[i], rs[j] = rs[j], rs[i] +} + +// topN returns the largest n cpuRecords (by CPUTimeMs), other cpuRecords are returned as evicted. +func (rs cpuRecords) topN(n int) (top, evicted cpuRecords) { + if len(rs) <= n { + return rs, nil + } + if err := quickselect.QuickSelect(rs, n); err != nil { + return rs, nil + } + return rs[:n], rs[n:] +} + +// sqlMeta is the SQL meta which contains the normalized SQL string and a bool +// field which uses to distinguish internal SQL. +type sqlMeta struct { + normalizedSQL string + isInternal bool +} + +// normalizedSQLMap is a wrapped map used to register normalizedSQL. +type normalizedSQLMap struct { + data atomic.Value // *sync.Map + length atomic2.Int64 +} + +func newNormalizedSQLMap() *normalizedSQLMap { + r := &normalizedSQLMap{} + r.data.Store(&sync.Map{}) + return r +} + +// register saves the relationship between sqlDigest and normalizedSQL. +// If the internal map size exceeds the limit, the relationship will be discarded. +func (m *normalizedSQLMap) register(sqlDigest []byte, normalizedSQL string, isInternal bool) { + if m.length.Load() >= variable.TopSQLVariable.MaxCollect.Load() { + ignoreExceedSQLCounter.Inc() + return + } + data := m.data.Load().(*sync.Map) + _, loaded := data.LoadOrStore(string(sqlDigest), sqlMeta{ + normalizedSQL: normalizedSQL, + isInternal: isInternal, + }) + if !loaded { + m.length.Add(1) + } +} + +// take away all data inside normalizedSQLMap, put them in the returned new normalizedSQLMap. +func (m *normalizedSQLMap) take() *normalizedSQLMap { + data := m.data.Load().(*sync.Map) + length := m.length.Load() + r := &normalizedSQLMap{} + r.data.Store(data) + r.length.Store(length) + m.data.Store(&sync.Map{}) + m.length.Store(0) + return r +} + +// toProto converts the normalizedSQLMap to the corresponding protobuf representation. +func (m *normalizedSQLMap) toProto() []tipb.SQLMeta { + metas := make([]tipb.SQLMeta, 0, m.length.Load()) + m.data.Load().(*sync.Map).Range(func(k, v interface{}) bool { + meta := v.(sqlMeta) + metas = append(metas, tipb.SQLMeta{ + SqlDigest: []byte(k.(string)), + NormalizedSql: meta.normalizedSQL, + IsInternalSql: meta.isInternal, + }) + return true + }) + return metas +} + +// planBinaryDecodeFunc is used to decode the value when converting +// normalizedPlanMap to protobuf representation. +type planBinaryDecodeFunc func(string) (string, error) + +// normalizedSQLMap is a wrapped map used to register normalizedPlan. +type normalizedPlanMap struct { + data atomic.Value // *sync.Map + length atomic2.Int64 +} + +func newNormalizedPlanMap() *normalizedPlanMap { + r := &normalizedPlanMap{} + r.data.Store(&sync.Map{}) + return r +} + +// register saves the relationship between planDigest and normalizedPlan. +// If the internal map size exceeds the limit, the relationship will be discarded. +func (m *normalizedPlanMap) register(planDigest []byte, normalizedPlan string) { + if m.length.Load() >= variable.TopSQLVariable.MaxCollect.Load() { + ignoreExceedPlanCounter.Inc() + return + } + data := m.data.Load().(*sync.Map) + _, loaded := data.LoadOrStore(string(planDigest), normalizedPlan) + if !loaded { + m.length.Add(1) + } +} + +// take away all data inside normalizedPlanMap, put them in the returned new normalizedPlanMap. +func (m *normalizedPlanMap) take() *normalizedPlanMap { + data := m.data.Load().(*sync.Map) + length := m.length.Load() + r := &normalizedPlanMap{} + r.data.Store(data) + r.length.Store(length) + m.data.Store(&sync.Map{}) + m.length.Store(0) + return r +} + +// toProto converts the normalizedPlanMap to the corresponding protobuf representation. +func (m *normalizedPlanMap) toProto(decodePlan planBinaryDecodeFunc) []tipb.PlanMeta { + metas := make([]tipb.PlanMeta, 0, m.length.Load()) + m.data.Load().(*sync.Map).Range(func(k, v interface{}) bool { + planDecoded, errDecode := decodePlan(v.(string)) + if errDecode != nil { + logutil.BgLogger().Warn("[top-sql] decode plan failed", zap.Error(errDecode)) + return true + } + metas = append(metas, tipb.PlanMeta{ + PlanDigest: []byte(k.(string)), + NormalizedPlan: planDecoded, + }) + return true + }) + return metas +} + +func encodeKey(buf *bytes.Buffer, sqlDigest, planDigest []byte) string { + buf.Reset() + buf.Write(sqlDigest) + buf.Write(planDigest) + return buf.String() +} diff --git a/util/topsql/reporter/datamodel_test.go b/util/topsql/reporter/datamodel_test.go new file mode 100644 index 0000000000000..3a0ba7b54fcac --- /dev/null +++ b/util/topsql/reporter/datamodel_test.go @@ -0,0 +1,466 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package reporter + +import ( + "bytes" + "sort" + "sync" + "testing" + + "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/util/topsql/stmtstats" + "github.com/pingcap/tipb/go-tipb" + "github.com/stretchr/testify/assert" +) + +func Test_tsItems_Sort(t *testing.T) { + items := tsItems{ + {timestamp: 2}, + {timestamp: 3}, + {timestamp: 1}, + } + sort.Sort(items) + assert.Equal(t, uint64(1), items[0].timestamp) + assert.Equal(t, uint64(2), items[1].timestamp) + assert.Equal(t, uint64(3), items[2].timestamp) +} + +func Test_record_Sort(t *testing.T) { + r := record{ + tsItems: tsItems{ + {timestamp: 2}, + {timestamp: 3}, + {timestamp: 1}, + }, + tsIndex: map[uint64]int{ + 2: 0, + 3: 1, + 1: 2, + }, + } + sort.Sort(&r) + assert.Equal(t, uint64(1), r.tsItems[0].timestamp) + assert.Equal(t, uint64(2), r.tsItems[1].timestamp) + assert.Equal(t, uint64(3), r.tsItems[2].timestamp) + assert.Equal(t, 0, r.tsIndex[1]) + assert.Equal(t, 1, r.tsIndex[2]) + assert.Equal(t, 2, r.tsIndex[3]) +} + +func Test_record_append(t *testing.T) { + r := newRecord(nil, nil) + // TimestampList: [] + // CPUTimeList: [] + // ExecCountList: [] + + r.appendCPUTime(1, 1) + // TimestampList: [1] + // CPUTimeList: [1] + // ExecCountList: [0] + + r.appendStmtStatsItem(1, stmtstats.StatementStatsItem{ExecCount: 1}) + // TimestampList: [1] + // CPUTimeList: [1] + // ExecCountList: [1] + + r.appendCPUTime(2, 1) + // TimestampList: [1, 2] + // CPUTimeList: [1, 1] + // ExecCountList: [1, 0] + + r.appendCPUTime(3, 1) + // TimestampList: [1, 2, 3] + // CPUTimeList: [1, 1, 1] + // ExecCountList: [1, 0, 0] + + r.appendStmtStatsItem(3, stmtstats.StatementStatsItem{ExecCount: 1}) + // TimestampList: [1, 2, 3] + // CPUTimeList: [1, 1, 1] + // ExecCountList: [1, 0, 1] + + r.appendStmtStatsItem(2, stmtstats.StatementStatsItem{ExecCount: 1}) + // TimestampList: [1, 2, 3] + // CPUTimeList: [1, 1, 1] + // ExecCountList: [1, 1, 1] + + assert.Len(t, r.tsItems, 3) + assert.Len(t, r.tsIndex, 3) + assert.Equal(t, uint64(3), r.totalCPUTimeMs) + assert.Equal(t, uint64(1), r.tsItems[0].timestamp) + assert.Equal(t, uint64(2), r.tsItems[1].timestamp) + assert.Equal(t, uint64(3), r.tsItems[2].timestamp) + assert.Equal(t, uint32(1), r.tsItems[0].cpuTimeMs) + assert.Equal(t, uint32(1), r.tsItems[1].cpuTimeMs) + assert.Equal(t, uint32(1), r.tsItems[2].cpuTimeMs) + assert.Equal(t, uint64(1), r.tsItems[0].stmtStats.ExecCount) + assert.Equal(t, uint64(1), r.tsItems[1].stmtStats.ExecCount) + assert.Equal(t, uint64(1), r.tsItems[2].stmtStats.ExecCount) +} + +func Test_record_merge(t *testing.T) { + r1 := record{ + totalCPUTimeMs: 1 + 2 + 3, + tsItems: tsItems{ + {timestamp: 1, cpuTimeMs: 1, stmtStats: *stmtstats.NewStatementStatsItem()}, + {timestamp: 2, cpuTimeMs: 2, stmtStats: *stmtstats.NewStatementStatsItem()}, + {timestamp: 3, cpuTimeMs: 3, stmtStats: *stmtstats.NewStatementStatsItem()}, + }, + } + r1.rebuildTsIndex() + r2 := record{ + totalCPUTimeMs: 6 + 5 + 4, + tsItems: tsItems{ + {timestamp: 6, cpuTimeMs: 6, stmtStats: *stmtstats.NewStatementStatsItem()}, + {timestamp: 5, cpuTimeMs: 5, stmtStats: *stmtstats.NewStatementStatsItem()}, + {timestamp: 4, cpuTimeMs: 4, stmtStats: *stmtstats.NewStatementStatsItem()}, + }, + } + r2.rebuildTsIndex() + r1.merge(&r2) + assert.Equal(t, uint64(4), r2.tsItems[0].timestamp) + assert.Equal(t, uint64(5), r2.tsItems[1].timestamp) + assert.Equal(t, uint64(6), r2.tsItems[2].timestamp) + assert.Len(t, r1.tsItems, 6) + assert.Len(t, r1.tsIndex, 6) + assert.Equal(t, uint64(1), r1.tsItems[0].timestamp) + assert.Equal(t, uint64(2), r1.tsItems[1].timestamp) + assert.Equal(t, uint64(3), r1.tsItems[2].timestamp) + assert.Equal(t, uint64(4), r1.tsItems[3].timestamp) + assert.Equal(t, uint64(5), r1.tsItems[4].timestamp) + assert.Equal(t, uint64(6), r1.tsItems[5].timestamp) + assert.Equal(t, uint64(1+2+3+4+5+6), r1.totalCPUTimeMs) +} + +func Test_record_rebuildTsIndex(t *testing.T) { + r := record{tsIndex: map[uint64]int{1: 1}} + r.rebuildTsIndex() + assert.Empty(t, r.tsIndex) + r.tsItems = tsItems{ + {timestamp: 1, cpuTimeMs: 1}, + {timestamp: 2, cpuTimeMs: 2}, + {timestamp: 3, cpuTimeMs: 3}, + } + r.rebuildTsIndex() + assert.Len(t, r.tsIndex, 3) + assert.Equal(t, 0, r.tsIndex[1]) + assert.Equal(t, 1, r.tsIndex[2]) + assert.Equal(t, 2, r.tsIndex[3]) +} + +func Test_record_toProto(t *testing.T) { + r := record{ + sqlDigest: []byte("SQL-1"), + planDigest: []byte("PLAN-1"), + totalCPUTimeMs: 123, + tsItems: tsItems{{ + timestamp: 1, + cpuTimeMs: 1, + stmtStats: stmtstats.StatementStatsItem{ + ExecCount: 1, + KvStatsItem: stmtstats.KvStatementStatsItem{KvExecCount: map[string]uint64{"": 1}}, + }, + }, { + timestamp: 2, + cpuTimeMs: 2, + stmtStats: stmtstats.StatementStatsItem{ + ExecCount: 2, + KvStatsItem: stmtstats.KvStatementStatsItem{KvExecCount: map[string]uint64{"": 2}}, + }, + }, { + timestamp: 3, + cpuTimeMs: 3, + stmtStats: stmtstats.StatementStatsItem{ + ExecCount: 3, + KvStatsItem: stmtstats.KvStatementStatsItem{KvExecCount: map[string]uint64{"": 3}}, + }, + }}, + } + pb := r.toProto() + assert.Equal(t, []byte("SQL-1"), pb.SqlDigest) + assert.Equal(t, []byte("PLAN-1"), pb.PlanDigest) + assert.Equal(t, []uint64{1, 2, 3}, pb.RecordListTimestampSec) + assert.Equal(t, []uint32{1, 2, 3}, pb.RecordListCpuTimeMs) + assert.Equal(t, []uint64{1, 2, 3}, pb.RecordListStmtExecCount) + assert.Equal(t, []*tipb.TopSQLStmtKvExecCount{ + {ExecCount: map[string]uint64{"": 1}}, + {ExecCount: map[string]uint64{"": 2}}, + {ExecCount: map[string]uint64{"": 3}}, + }, pb.RecordListStmtKvExecCount) +} + +func Test_records_Sort(t *testing.T) { + rs := records{ + {totalCPUTimeMs: 1}, + {totalCPUTimeMs: 3}, + {totalCPUTimeMs: 2}, + } + sort.Sort(rs) + assert.Equal(t, uint64(3), rs[0].totalCPUTimeMs) + assert.Equal(t, uint64(2), rs[1].totalCPUTimeMs) + assert.Equal(t, uint64(1), rs[2].totalCPUTimeMs) +} + +func Test_records_topN(t *testing.T) { + rs := records{ + {totalCPUTimeMs: 1}, + {totalCPUTimeMs: 3}, + {totalCPUTimeMs: 2}, + } + top, evicted := rs.topN(2) + assert.Len(t, top, 2) + assert.Len(t, evicted, 1) + assert.Equal(t, uint64(3), top[0].totalCPUTimeMs) + assert.Equal(t, uint64(2), top[1].totalCPUTimeMs) + assert.Equal(t, uint64(1), evicted[0].totalCPUTimeMs) +} + +func Test_records_toProto(t *testing.T) { + rs := records{{}, {}} + pb := rs.toProto() + assert.Len(t, pb, 2) +} + +func Test_collecting_getOrCreateRecord(t *testing.T) { + c := newCollecting() + r1 := c.getOrCreateRecord([]byte("SQL-1"), []byte("PLAN-1")) + assert.NotNil(t, r1) + r2 := c.getOrCreateRecord([]byte("SQL-1"), []byte("PLAN-1")) + assert.Equal(t, r1, r2) +} + +func Test_collecting_markAsEvicted_hasEvicted(t *testing.T) { + c := newCollecting() + c.markAsEvicted(1, []byte("SQL-1"), []byte("PLAN-1")) + assert.True(t, c.hasEvicted(1, []byte("SQL-1"), []byte("PLAN-1"))) + assert.False(t, c.hasEvicted(1, []byte("SQL-2"), []byte("PLAN-2"))) + assert.False(t, c.hasEvicted(2, []byte("SQL-1"), []byte("PLAN-1"))) +} + +func Test_collecting_appendOthers(t *testing.T) { + c := newCollecting() + c.appendOthersCPUTime(1, 1) + c.appendOthersCPUTime(2, 2) + c.appendOthersStmtStatsItem(1, stmtstats.StatementStatsItem{ExecCount: 1}) + c.appendOthersStmtStatsItem(2, stmtstats.StatementStatsItem{ExecCount: 2}) + r := c.records[keyOthers] + assert.Len(t, r.tsItems, 2) + assert.Len(t, r.tsIndex, 2) + assert.Equal(t, uint64(1), r.tsItems[0].timestamp) + assert.Equal(t, uint64(2), r.tsItems[1].timestamp) + assert.Equal(t, uint32(1), r.tsItems[0].cpuTimeMs) + assert.Equal(t, uint32(2), r.tsItems[1].cpuTimeMs) + assert.Equal(t, uint64(1), r.tsItems[0].stmtStats.ExecCount) + assert.Equal(t, uint64(2), r.tsItems[1].stmtStats.ExecCount) +} + +func Test_collecting_topN(t *testing.T) { + c := newCollecting() + c.getOrCreateRecord([]byte("SQL-1"), []byte("PLAN-1")).appendCPUTime(1, 1) + c.getOrCreateRecord([]byte("SQL-2"), []byte("PLAN-2")).appendCPUTime(1, 2) + c.getOrCreateRecord([]byte("SQL-3"), []byte("PLAN-3")).appendCPUTime(1, 3) + rs := c.topN(1) + assert.Len(t, rs, 2) + assert.Equal(t, []byte("SQL-3"), rs[0].sqlDigest) + assert.Equal(t, []byte("PLAN-3"), rs[0].planDigest) + assert.Equal(t, uint64(3), rs[0].totalCPUTimeMs) + assert.Len(t, rs[0].tsItems, 1) + assert.Equal(t, uint32(3), rs[0].tsItems[0].cpuTimeMs) + assert.Equal(t, uint64(3), rs[1].totalCPUTimeMs) // 1 + 2 = 3 +} + +func Test_collecting_take(t *testing.T) { + c1 := newCollecting() + c1.getOrCreateRecord([]byte("SQL-1"), []byte("PLAN-1")).appendCPUTime(1, 1) + c2 := c1.take() + assert.Empty(t, c1.records) + assert.Len(t, c2.records, 1) + assert.NotEqual(t, c1.keyBuf, c2.keyBuf) +} + +func Test_cpuRecords_Sort(t *testing.T) { + rs := cpuRecords{ + {CPUTimeMs: 1}, + {CPUTimeMs: 3}, + {CPUTimeMs: 2}, + } + sort.Sort(rs) + assert.Equal(t, uint32(3), rs[0].CPUTimeMs) + assert.Equal(t, uint32(2), rs[1].CPUTimeMs) + assert.Equal(t, uint32(1), rs[2].CPUTimeMs) +} + +func Test_cpuRecords_topN(t *testing.T) { + rs := cpuRecords{ + {CPUTimeMs: 1}, + {CPUTimeMs: 3}, + {CPUTimeMs: 2}, + } + top, evicted := rs.topN(2) + assert.Len(t, top, 2) + assert.Len(t, evicted, 1) + assert.Equal(t, uint32(3), top[0].CPUTimeMs) + assert.Equal(t, uint32(2), top[1].CPUTimeMs) + assert.Equal(t, uint32(1), evicted[0].CPUTimeMs) +} + +func Test_normalizedSQLMap_register(t *testing.T) { + variable.TopSQLVariable.MaxCollect.Store(2) + m := newNormalizedSQLMap() + m.register([]byte("SQL-1"), "SQL-1", true) + m.register([]byte("SQL-2"), "SQL-2", false) + m.register([]byte("SQL-3"), "SQL-3", true) + assert.Equal(t, int64(2), m.length.Load()) + v, ok := m.data.Load().(*sync.Map).Load("SQL-1") + meta := v.(sqlMeta) + assert.True(t, ok) + assert.Equal(t, "SQL-1", meta.normalizedSQL) + assert.True(t, meta.isInternal) + v, ok = m.data.Load().(*sync.Map).Load("SQL-2") + meta = v.(sqlMeta) + assert.True(t, ok) + assert.Equal(t, "SQL-2", meta.normalizedSQL) + assert.False(t, meta.isInternal) + _, ok = m.data.Load().(*sync.Map).Load("SQL-3") + assert.False(t, ok) +} + +func Test_normalizedSQLMap_take(t *testing.T) { + variable.TopSQLVariable.MaxCollect.Store(999) + m1 := newNormalizedSQLMap() + m1.register([]byte("SQL-1"), "SQL-1", true) + m1.register([]byte("SQL-2"), "SQL-2", false) + m1.register([]byte("SQL-3"), "SQL-3", true) + m2 := m1.take() + assert.Equal(t, int64(0), m1.length.Load()) + assert.Equal(t, int64(3), m2.length.Load()) + data1 := m1.data.Load().(*sync.Map) + _, ok := data1.Load("SQL-1") + assert.False(t, ok) + _, ok = data1.Load("SQL-2") + assert.False(t, ok) + _, ok = data1.Load("SQL-3") + assert.False(t, ok) + data2 := m2.data.Load().(*sync.Map) + _, ok = data2.Load("SQL-1") + assert.True(t, ok) + _, ok = data2.Load("SQL-2") + assert.True(t, ok) + _, ok = data2.Load("SQL-3") + assert.True(t, ok) +} + +func Test_normalizedSQLMap_toProto(t *testing.T) { + variable.TopSQLVariable.MaxCollect.Store(999) + m := newNormalizedSQLMap() + m.register([]byte("SQL-1"), "SQL-1", true) + m.register([]byte("SQL-2"), "SQL-2", false) + m.register([]byte("SQL-3"), "SQL-3", true) + pb := m.toProto() + assert.Len(t, pb, 3) + hash := map[string]tipb.SQLMeta{} + for _, meta := range pb { + hash[meta.NormalizedSql] = meta + } + assert.Equal(t, tipb.SQLMeta{ + SqlDigest: []byte("SQL-1"), + NormalizedSql: "SQL-1", + IsInternalSql: true, + }, hash["SQL-1"]) + assert.Equal(t, tipb.SQLMeta{ + SqlDigest: []byte("SQL-2"), + NormalizedSql: "SQL-2", + IsInternalSql: false, + }, hash["SQL-2"]) + assert.Equal(t, tipb.SQLMeta{ + SqlDigest: []byte("SQL-3"), + NormalizedSql: "SQL-3", + IsInternalSql: true, + }, hash["SQL-3"]) +} + +func Test_normalizedPlanMap_register(t *testing.T) { + variable.TopSQLVariable.MaxCollect.Store(2) + m := newNormalizedPlanMap() + m.register([]byte("PLAN-1"), "PLAN-1") + m.register([]byte("PLAN-2"), "PLAN-2") + m.register([]byte("PLAN-3"), "PLAN-3") + assert.Equal(t, int64(2), m.length.Load()) + v, ok := m.data.Load().(*sync.Map).Load("PLAN-1") + assert.True(t, ok) + assert.Equal(t, "PLAN-1", v.(string)) + v, ok = m.data.Load().(*sync.Map).Load("PLAN-2") + assert.True(t, ok) + assert.Equal(t, "PLAN-2", v.(string)) + _, ok = m.data.Load().(*sync.Map).Load("PLAN-3") + assert.False(t, ok) +} + +func Test_normalizedPlanMap_take(t *testing.T) { + variable.TopSQLVariable.MaxCollect.Store(999) + m1 := newNormalizedPlanMap() + m1.register([]byte("PLAN-1"), "PLAN-1") + m1.register([]byte("PLAN-2"), "PLAN-2") + m1.register([]byte("PLAN-3"), "PLAN-3") + m2 := m1.take() + assert.Equal(t, int64(0), m1.length.Load()) + assert.Equal(t, int64(3), m2.length.Load()) + data1 := m1.data.Load().(*sync.Map) + _, ok := data1.Load("PLAN-1") + assert.False(t, ok) + _, ok = data1.Load("PLAN-2") + assert.False(t, ok) + _, ok = data1.Load("PLAN-3") + assert.False(t, ok) + data2 := m2.data.Load().(*sync.Map) + _, ok = data2.Load("PLAN-1") + assert.True(t, ok) + _, ok = data2.Load("PLAN-2") + assert.True(t, ok) + _, ok = data2.Load("PLAN-3") + assert.True(t, ok) +} + +func Test_normalizedPlanMap_toProto(t *testing.T) { + variable.TopSQLVariable.MaxCollect.Store(999) + m := newNormalizedPlanMap() + m.register([]byte("PLAN-1"), "PLAN-1") + m.register([]byte("PLAN-2"), "PLAN-2") + m.register([]byte("PLAN-3"), "PLAN-3") + pb := m.toProto(func(s string) (string, error) { return s, nil }) + assert.Len(t, pb, 3) + hash := map[string]tipb.PlanMeta{} + for _, meta := range pb { + hash[meta.NormalizedPlan] = meta + } + assert.Equal(t, tipb.PlanMeta{ + PlanDigest: []byte("PLAN-1"), + NormalizedPlan: "PLAN-1", + }, hash["PLAN-1"]) + assert.Equal(t, tipb.PlanMeta{ + PlanDigest: []byte("PLAN-2"), + NormalizedPlan: "PLAN-2", + }, hash["PLAN-2"]) + assert.Equal(t, tipb.PlanMeta{ + PlanDigest: []byte("PLAN-3"), + NormalizedPlan: "PLAN-3", + }, hash["PLAN-3"]) +} + +func Test_encodeKey(t *testing.T) { + buf := bytes.NewBuffer(make([]byte, 0, 64)) + key := encodeKey(buf, []byte("S"), []byte("P")) + assert.Equal(t, "SP", key) +} diff --git a/util/topsql/reporter/datasink.go b/util/topsql/reporter/datasink.go index 2196be54a0f93..809798619f1c3 100644 --- a/util/topsql/reporter/datasink.go +++ b/util/topsql/reporter/datasink.go @@ -14,7 +14,17 @@ package reporter -import "time" +import ( + "context" + "sync" + "time" + + "github.com/pingcap/errors" + "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/util/logutil" + "github.com/pingcap/tipb/go-tipb" + "go.uber.org/zap" +) // DataSink collects and sends data to a target. type DataSink interface { @@ -26,3 +36,106 @@ type DataSink interface { // OnReporterClosing notifies DataSink that the reporter is closing. OnReporterClosing() } + +// DataSinkRegisterer is for registering DataSink +type DataSinkRegisterer interface { + Register(dataSink DataSink) error + Deregister(dataSink DataSink) +} + +// ReportData contains data that reporter sends to the agent. +type ReportData struct { + // DataRecords contains the topN records []tipb.TopSQLRecord and the `others` + // record which aggregation all []tipb.TopSQLRecord that is out of Top N. + DataRecords []tipb.TopSQLRecord + SQLMetas []tipb.SQLMeta + PlanMetas []tipb.PlanMeta +} + +func (d *ReportData) hasData() bool { + return len(d.DataRecords) != 0 || len(d.SQLMetas) != 0 || len(d.PlanMetas) != 0 +} + +var _ DataSink = &DefaultDataSinkRegisterer{} +var _ DataSinkRegisterer = &DefaultDataSinkRegisterer{} + +// DefaultDataSinkRegisterer implements DataSinkRegisterer. +type DefaultDataSinkRegisterer struct { + sync.Mutex + ctx context.Context + dataSinks map[DataSink]struct{} +} + +// NewDefaultDataSinkRegisterer creates a new DefaultDataSinkRegisterer which implements DataSinkRegisterer. +func NewDefaultDataSinkRegisterer(ctx context.Context) DefaultDataSinkRegisterer { + return DefaultDataSinkRegisterer{ + ctx: ctx, + dataSinks: make(map[DataSink]struct{}, 10), + } +} + +// Register implements DataSinkRegisterer. +func (r *DefaultDataSinkRegisterer) Register(dataSink DataSink) error { + r.Lock() + defer r.Unlock() + + select { + case <-r.ctx.Done(): + return errors.New("DefaultDataSinkRegisterer closed") + default: + if len(r.dataSinks) >= 10 { + return errors.New("too many datasinks") + } + r.dataSinks[dataSink] = struct{}{} + if len(r.dataSinks) > 0 { + variable.TopSQLVariable.Enable.Store(true) + } + return nil + } +} + +// Deregister implements DataSinkRegisterer. +func (r *DefaultDataSinkRegisterer) Deregister(dataSink DataSink) { + r.Lock() + defer r.Unlock() + + select { + case <-r.ctx.Done(): + default: + delete(r.dataSinks, dataSink) + if len(r.dataSinks) == 0 { + variable.TopSQLVariable.Enable.Store(false) + } + } +} + +// TrySend implements DataSink. +// +// TrySend sends ReportData to all internal registered DataSinks. +func (r *DefaultDataSinkRegisterer) TrySend(data *ReportData, deadline time.Time) error { + r.Lock() + dataSinks := make([]DataSink, 0, len(r.dataSinks)) + for ds := range r.dataSinks { + dataSinks = append(dataSinks, ds) + } + r.Unlock() + for _, ds := range dataSinks { + if err := ds.TrySend(data, deadline); err != nil { + logutil.BgLogger().Warn("[top-sql] failed to send data to datasink", zap.Error(err)) + } + } + return nil +} + +// OnReporterClosing implements DataSink. +// +// OnReporterClosing calls the OnReporterClosing method of all internally registered DataSinks. +func (r *DefaultDataSinkRegisterer) OnReporterClosing() { + var m map[DataSink]struct{} + r.Lock() + m, r.dataSinks = r.dataSinks, make(map[DataSink]struct{}) + r.Unlock() + for d := range m { + d.OnReporterClosing() + } +} diff --git a/util/topsql/reporter/datasink_test.go b/util/topsql/reporter/datasink_test.go new file mode 100644 index 0000000000000..b57d36515eb3d --- /dev/null +++ b/util/topsql/reporter/datasink_test.go @@ -0,0 +1,67 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package reporter + +import ( + "context" + "testing" + "time" + + "github.com/stretchr/testify/assert" +) + +func TestDefaultDataSinkRegisterer(t *testing.T) { + var err error + r := NewDefaultDataSinkRegisterer(context.Background()) + m1 := newMockDataSink2() + m2 := newMockDataSink2() + err = r.Register(m1) + assert.NoError(t, err) + err = r.Register(m2) + assert.NoError(t, err) + err = r.TrySend(&ReportData{}, time.Now().Add(time.Hour)) + assert.NoError(t, err) + assert.Len(t, m1.data, 1) + assert.Len(t, m2.data, 1) + err = r.TrySend(&ReportData{}, time.Now().Add(time.Hour)) + assert.NoError(t, err) + assert.Len(t, m1.data, 2) + assert.Len(t, m2.data, 2) + assert.False(t, m1.closed) + assert.False(t, m2.closed) + r.OnReporterClosing() + assert.True(t, m1.closed) + assert.True(t, m2.closed) +} + +type mockDataSink2 struct { + data []*ReportData + closed bool +} + +func newMockDataSink2() *mockDataSink2 { + return &mockDataSink2{ + data: []*ReportData{}, + } +} + +func (m *mockDataSink2) TrySend(data *ReportData, deadline time.Time) error { + m.data = append(m.data, data) + return nil +} + +func (m *mockDataSink2) OnReporterClosing() { + m.closed = true +} diff --git a/util/topsql/reporter/metrics.go b/util/topsql/reporter/metrics.go new file mode 100644 index 0000000000000..0314956a6c795 --- /dev/null +++ b/util/topsql/reporter/metrics.go @@ -0,0 +1,36 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package reporter + +import "github.com/pingcap/tidb/metrics" + +var ( + ignoreExceedSQLCounter = metrics.TopSQLIgnoredCounter.WithLabelValues("ignore_exceed_sql") + ignoreExceedPlanCounter = metrics.TopSQLIgnoredCounter.WithLabelValues("ignore_exceed_plan") + ignoreCollectChannelFullCounter = metrics.TopSQLIgnoredCounter.WithLabelValues("ignore_collect_channel_full") + ignoreCollectStmtChannelFullCounter = metrics.TopSQLIgnoredCounter.WithLabelValues("ignore_collect_stmt_channel_full") + ignoreReportChannelFullCounter = metrics.TopSQLIgnoredCounter.WithLabelValues("ignore_report_channel_full") + reportAllDurationSuccHistogram = metrics.TopSQLReportDurationHistogram.WithLabelValues("all", metrics.LblOK) + reportAllDurationFailedHistogram = metrics.TopSQLReportDurationHistogram.WithLabelValues("all", metrics.LblError) + reportRecordDurationSuccHistogram = metrics.TopSQLReportDurationHistogram.WithLabelValues("record", metrics.LblOK) + reportRecordDurationFailedHistogram = metrics.TopSQLReportDurationHistogram.WithLabelValues("record", metrics.LblError) + reportSQLDurationSuccHistogram = metrics.TopSQLReportDurationHistogram.WithLabelValues("sql", metrics.LblOK) + reportSQLDurationFailedHistogram = metrics.TopSQLReportDurationHistogram.WithLabelValues("sql", metrics.LblError) + reportPlanDurationSuccHistogram = metrics.TopSQLReportDurationHistogram.WithLabelValues("plan", metrics.LblOK) + reportPlanDurationFailedHistogram = metrics.TopSQLReportDurationHistogram.WithLabelValues("plan", metrics.LblError) + topSQLReportRecordCounterHistogram = metrics.TopSQLReportDataHistogram.WithLabelValues("record") + topSQLReportSQLCountHistogram = metrics.TopSQLReportDataHistogram.WithLabelValues("sql") + topSQLReportPlanCountHistogram = metrics.TopSQLReportDataHistogram.WithLabelValues("plan") +) diff --git a/util/topsql/reporter/reporter.go b/util/topsql/reporter/reporter.go index 0fc1b739a1f6f..9391a232c47bf 100644 --- a/util/topsql/reporter/reporter.go +++ b/util/topsql/reporter/reporter.go @@ -15,629 +15,149 @@ package reporter import ( - "bytes" "context" - "errors" - "sort" - "sync" - "sync/atomic" "time" "github.com/pingcap/failpoint" - "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/util" - "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/topsql/stmtstats" "github.com/pingcap/tidb/util/topsql/tracecpu" - "github.com/pingcap/tipb/go-tipb" - "github.com/wangjohn/quickselect" - atomic2 "go.uber.org/atomic" - "go.uber.org/zap" ) -const ( - dialTimeout = 5 * time.Second - reportTimeout = 40 * time.Second - grpcInitialWindowSize = 1 << 30 - grpcInitialConnWindowSize = 1 << 30 - // keyOthers is the key to store the aggregation of all records that is out of Top N. - keyOthers = "" -) - -var _ TopSQLReporter = &RemoteTopSQLReporter{} +const reportTimeout = 40 * time.Second // TopSQLReporter collects Top SQL metrics. type TopSQLReporter interface { tracecpu.Collector stmtstats.Collector - RegisterSQL(sqlDigest []byte, normalizedSQL string, isInternal bool) - RegisterPlan(planDigest []byte, normalizedPlan string) - Close() -} - -// DataSinkRegisterer is for registering DataSink -type DataSinkRegisterer interface { - Register(dataSink DataSink) error - Deregister(dataSink DataSink) -} - -type cpuData struct { - timestamp uint64 - records []tracecpu.SQLCPUTimeRecord -} - -// dataPoints represents the cumulative SQL plan CPU time in current minute window -// dataPoints do not guarantee the TimestampList is sorted by timestamp when there is a time jump backward. -type dataPoints struct { - SQLDigest []byte - PlanDigest []byte - TimestampList []uint64 - CPUTimeMsList []uint32 - CPUTimeMsTotal uint64 - StmtExecCountList []uint64 - StmtKvExecCountList []map[string]uint64 - StmtDurationSumNsList []uint64 - - // tsIndex is used to quickly find the corresponding array index through timestamp. - // - // map: timestamp => index of TimestampList / CPUTimeMsList / StmtExecCountList / ... - tsIndex map[uint64]int -} - -func newDataPoints(sqlDigest, planDigest []byte) *dataPoints { - listCap := int(variable.TopSQLVariable.ReportIntervalSeconds.Load()/variable.TopSQLVariable.PrecisionSeconds.Load() + 1) - if listCap < 1 { - listCap = 1 - } - return &dataPoints{ - SQLDigest: sqlDigest, - PlanDigest: planDigest, - CPUTimeMsList: make([]uint32, 0, listCap), - TimestampList: make([]uint64, 0, listCap), - StmtExecCountList: make([]uint64, 0, listCap), - StmtKvExecCountList: make([]map[string]uint64, 0, listCap), - StmtDurationSumNsList: make([]uint64, 0, listCap), - tsIndex: make(map[uint64]int, listCap), - } -} - -func (d *dataPoints) rebuildTsIndex() { - if len(d.TimestampList) == 0 { - d.tsIndex = map[uint64]int{} - return - } - d.tsIndex = make(map[uint64]int, len(d.TimestampList)) - for index, ts := range d.TimestampList { - d.tsIndex[ts] = index - } -} - -func (d *dataPoints) appendCPUTime(timestamp uint64, cpuTimeMs uint32) { - if index, ok := d.tsIndex[timestamp]; ok { - // For the same timestamp, we have already called appendStmtStatsItem, - // d.TimestampList already exists the corresponding timestamp. And it - // can be determined that the corresponding index of d.StmtXxx has been - // correctly assigned, and the corresponding index of d.CPUTimeMsList - // has been set to 0, so we directly replace it. - // - // let timestamp = 10000, cpuTimeMs = 123 - // - // Before: - // tsIndex: [10000 => 0] - // TimestampList: [10000] - // CPUTimeMsList: [0] - // StmtExecCountList: [999] - // StmtKvExecCountList: [map{"9.9.9.9:9":999}] - // StmtDurationSumNsList: [999] - // - // After: - // tsIndex: [10000 => 0] - // TimestampList: [10000] - // CPUTimeMsList: [123] - // StmtExecCountList: [999] - // StmtKvExecCountList: [map{"9.9.9.9:9":999}] - // StmtDurationSumNsList: [999] - // - d.CPUTimeMsList[index] = cpuTimeMs - } else { - // For this timestamp, we have not appended any data. So append it directly, - // and set the data not related to CPUTimeList to 0. - // - // let timestamp = 10000, cpu_time = 123 - // - // Before: - // tsIndex: [] - // TimestampList: [] - // CPUTimeMsList: [] - // StmtExecCountList: [] - // StmtKvExecCountList: [] - // StmtDurationSumNsList: [] - // - // After: - // tsIndex: [10000 => 0] - // TimestampList: [10000] - // CPUTimeMsList: [123] - // StmtExecCountList: [0] - // StmtKvExecCountList: [map{}] - // StmtDurationSumNsList: [0] - // - d.tsIndex[timestamp] = len(d.TimestampList) - d.TimestampList = append(d.TimestampList, timestamp) - d.CPUTimeMsList = append(d.CPUTimeMsList, cpuTimeMs) - d.StmtExecCountList = append(d.StmtExecCountList, 0) - d.StmtKvExecCountList = append(d.StmtKvExecCountList, map[string]uint64{}) - d.StmtDurationSumNsList = append(d.StmtDurationSumNsList, 0) - } - d.CPUTimeMsTotal += uint64(cpuTimeMs) -} - -func (d *dataPoints) appendStmtStatsItem(timestamp uint64, item *stmtstats.StatementStatsItem) { - if index, ok := d.tsIndex[timestamp]; ok { - // For the same timestamp, we have already called appendCPUTime, - // d.TimestampList already exists the corresponding timestamp. And it - // can be determined that the corresponding index of d.CPUTimeMsList has been - // correctly assigned, and the corresponding index of d.StmtXxx - // has been set to 0 or empty map, so we directly replace it. - // - // let timestamp = 10000, execCount = 123, kvExecCount = map{"1.1.1.1:1": 123}, durationSum = 456 - // - // Before: - // tsIndex: [10000 => 0] - // TimestampList: [10000] - // CPUTimeMsList: [999] - // StmtExecCountList: [0] - // StmtKvExecCountList: [map{}] - // StmtDurationSumNsList: [0] - // - // After: - // tsIndex: [10000 => 0] - // TimestampList: [10000] - // CPUTimeMsList: [999] - // StmtExecCountList: [123] - // StmtKvExecCountList: [map{"1.1.1.1:1": 123}] - // StmtDurationSumNsList: [456] - // - d.StmtExecCountList[index] = item.ExecCount - d.StmtKvExecCountList[index] = item.KvStatsItem.KvExecCount - d.StmtDurationSumNsList[index] = 0 // TODO(mornyx): add duration - } else { - // For this timestamp, we have not appended any data. So append it directly, - // the corresponding index of d.CPUTimeList is preset to 0. - // - // let timestamp = 10000, execCount = 123, kvExecCount = map{"1.1.1.1:1": 123}, durationSum = 456 - // - // Before: - // tsIndex: [] - // TimestampList: [] - // CPUTimeMsList: [] - // StmtExecCountList: [] - // StmtKvExecCountList: [] - // StmtDurationSumNsList: [] - // - // After: - // tsIndex: [10000 => 0] - // TimestampList: [10000] - // CPUTimeMsList: [0] - // StmtExecCountList: [123] - // StmtKvExecCountList: [map{"1.1.1.1:1": 123}] - // StmtDurationSumNsList: [456] - // - d.tsIndex[timestamp] = len(d.TimestampList) - d.TimestampList = append(d.TimestampList, timestamp) - d.CPUTimeMsList = append(d.CPUTimeMsList, 0) - d.StmtExecCountList = append(d.StmtExecCountList, item.ExecCount) - d.StmtKvExecCountList = append(d.StmtKvExecCountList, item.KvStatsItem.KvExecCount) - d.StmtDurationSumNsList = append(d.StmtDurationSumNsList, 0) // TODO(mornyx): add duration - } -} -func (d *dataPoints) isInvalid() bool { - return !(len(d.TimestampList) == len(d.CPUTimeMsList) && - len(d.TimestampList) == len(d.StmtExecCountList) && - len(d.TimestampList) == len(d.StmtKvExecCountList) && - len(d.TimestampList) == len(d.StmtDurationSumNsList)) -} - -func (d *dataPoints) Len() int { - return len(d.TimestampList) -} - -func (d *dataPoints) Less(i, j int) bool { - // sort by timestamp - return d.TimestampList[i] < d.TimestampList[j] -} -func (d *dataPoints) Swap(i, j int) { - // before swap: - // TimestampList: [10000, 10001, 10002] - // tsIndex: [10000 => 0, 10001 => 1, 10002 => 2] - // - // let i = 0, j = 1 - // after swap tsIndex: - // TimestampList: [10000, 10001, 10002] - // tsIndex: [10000 => 1, 10001 => 0, 10002 => 2] + // RegisterSQL registers a normalizedSQL with SQLDigest. // - // after swap TimestampList: - // TimestampList: [10001, 10000, 10002] - // tsIndex: [10000 => 1, 10001 => 0, 10002 => 2] - d.tsIndex[d.TimestampList[i]], d.tsIndex[d.TimestampList[j]] = d.tsIndex[d.TimestampList[j]], d.tsIndex[d.TimestampList[i]] - d.TimestampList[i], d.TimestampList[j] = d.TimestampList[j], d.TimestampList[i] - d.CPUTimeMsList[i], d.CPUTimeMsList[j] = d.CPUTimeMsList[j], d.CPUTimeMsList[i] - d.StmtExecCountList[i], d.StmtExecCountList[j] = d.StmtExecCountList[j], d.StmtExecCountList[i] - d.StmtKvExecCountList[i], d.StmtKvExecCountList[j] = d.StmtKvExecCountList[j], d.StmtKvExecCountList[i] - d.StmtDurationSumNsList[i], d.StmtDurationSumNsList[j] = d.StmtDurationSumNsList[j], d.StmtDurationSumNsList[i] -} - -type dataPointsOrderByCPUTime []*dataPoints - -func (t dataPointsOrderByCPUTime) Len() int { - return len(t) -} - -func (t dataPointsOrderByCPUTime) Less(i, j int) bool { - // We need find the kth largest value, so here should use > - return t[i].CPUTimeMsTotal > t[j].CPUTimeMsTotal -} -func (t dataPointsOrderByCPUTime) Swap(i, j int) { - t[i], t[j] = t[j], t[i] -} - -type sqlCPUTimeRecordSlice []tracecpu.SQLCPUTimeRecord + // Note that the normalized SQL string can be of >1M long. + // This function should be thread-safe, which means concurrently calling it + // in several goroutines should be fine. It should also return immediately, + // and do any CPU-intensive job asynchronously. + RegisterSQL(sqlDigest []byte, normalizedSQL string, isInternal bool) -func (t sqlCPUTimeRecordSlice) Len() int { - return len(t) -} + // RegisterPlan like RegisterSQL, but for normalized plan strings. + RegisterPlan(planDigest []byte, normalizedPlan string) -func (t sqlCPUTimeRecordSlice) Less(i, j int) bool { - // We need find the kth largest value, so here should use > - return t[i].CPUTimeMs > t[j].CPUTimeMs -} -func (t sqlCPUTimeRecordSlice) Swap(i, j int) { - t[i], t[j] = t[j], t[i] + // Close uses to close and release the reporter resource. + Close() } -type planBinaryDecodeFunc func(string) (string, error) +var _ TopSQLReporter = &RemoteTopSQLReporter{} +var _ DataSinkRegisterer = &RemoteTopSQLReporter{} -// RemoteTopSQLReporter implements a TopSQL reporter that sends data to a remote agent -// This should be called periodically to collect TopSQL resource usage metrics +// RemoteTopSQLReporter implements TopSQLReporter that sends data to a remote agent. +// This should be called periodically to collect TopSQL resource usage metrics. type RemoteTopSQLReporter struct { + DefaultDataSinkRegisterer + ctx context.Context cancel context.CancelFunc - dataSinkMu sync.Mutex - dataSinks map[DataSink]struct{} - - // normalizedSQLMap is an map, whose keys are SQL digest strings and values are SQLMeta. - normalizedSQLMap atomic.Value // sync.Map - sqlMapLength atomic2.Int64 - - // normalizedPlanMap is an map, whose keys are plan digest strings and values are normalized plans **in binary**. - // The normalized plans in binary can be decoded to string using the `planBinaryDecoder`. - normalizedPlanMap atomic.Value // sync.Map - planMapLength atomic2.Int64 - collectCPUDataChan chan cpuData collectStmtRecordsChan chan []stmtstats.StatementStatsRecord reportCollectedDataChan chan collectedData - // calling decodePlan this can take a while, so should not block critical paths - decodePlan planBinaryDecodeFunc -} + collecting *collecting + normalizedSQLMap *normalizedSQLMap + normalizedPlanMap *normalizedPlanMap -// SQLMeta is the SQL meta which contains the normalized SQL string and a bool field which uses to distinguish internal SQL. -type SQLMeta struct { - normalizedSQL string - isInternal bool + // calling decodePlan this can take a while, so should not block critical paths. + decodePlan planBinaryDecodeFunc } -// NewRemoteTopSQLReporter creates a new TopSQL reporter +// NewRemoteTopSQLReporter creates a new RemoteTopSQLReporter. // -// planBinaryDecoder is a decoding function which will be called asynchronously to decode the plan binary to string -// MaxStatementsNum is the maximum SQL and plan number, which will restrict the memory usage of the internal LFU cache +// decodePlan is a decoding function which will be called asynchronously to decode the plan binary to string. func NewRemoteTopSQLReporter(decodePlan planBinaryDecodeFunc) *RemoteTopSQLReporter { ctx, cancel := context.WithCancel(context.Background()) tsr := &RemoteTopSQLReporter{ - ctx: ctx, - cancel: cancel, - - dataSinks: make(map[DataSink]struct{}, 10), - - collectCPUDataChan: make(chan cpuData, 1), - collectStmtRecordsChan: make(chan []stmtstats.StatementStatsRecord, 1), - reportCollectedDataChan: make(chan collectedData, 1), - decodePlan: decodePlan, + DefaultDataSinkRegisterer: NewDefaultDataSinkRegisterer(ctx), + ctx: ctx, + cancel: cancel, + collectCPUDataChan: make(chan cpuData, 1), + collectStmtRecordsChan: make(chan []stmtstats.StatementStatsRecord, 1), + reportCollectedDataChan: make(chan collectedData, 1), + collecting: newCollecting(), + normalizedSQLMap: newNormalizedSQLMap(), + normalizedPlanMap: newNormalizedPlanMap(), + decodePlan: decodePlan, } - tsr.normalizedSQLMap.Store(&sync.Map{}) - tsr.normalizedPlanMap.Store(&sync.Map{}) - go tsr.collectWorker() go tsr.reportWorker() - return tsr } -var ( - ignoreExceedSQLCounter = metrics.TopSQLIgnoredCounter.WithLabelValues("ignore_exceed_sql") - ignoreExceedPlanCounter = metrics.TopSQLIgnoredCounter.WithLabelValues("ignore_exceed_plan") - ignoreCollectChannelFullCounter = metrics.TopSQLIgnoredCounter.WithLabelValues("ignore_collect_channel_full") - ignoreCollectStmtChannelFullCounter = metrics.TopSQLIgnoredCounter.WithLabelValues("ignore_collect_stmt_channel_full") - ignoreReportChannelFullCounter = metrics.TopSQLIgnoredCounter.WithLabelValues("ignore_report_channel_full") - reportAllDurationSuccHistogram = metrics.TopSQLReportDurationHistogram.WithLabelValues("all", metrics.LblOK) - reportAllDurationFailedHistogram = metrics.TopSQLReportDurationHistogram.WithLabelValues("all", metrics.LblError) - reportRecordDurationSuccHistogram = metrics.TopSQLReportDurationHistogram.WithLabelValues("record", metrics.LblOK) - reportRecordDurationFailedHistogram = metrics.TopSQLReportDurationHistogram.WithLabelValues("record", metrics.LblError) - reportSQLDurationSuccHistogram = metrics.TopSQLReportDurationHistogram.WithLabelValues("sql", metrics.LblOK) - reportSQLDurationFailedHistogram = metrics.TopSQLReportDurationHistogram.WithLabelValues("sql", metrics.LblError) - reportPlanDurationSuccHistogram = metrics.TopSQLReportDurationHistogram.WithLabelValues("plan", metrics.LblOK) - reportPlanDurationFailedHistogram = metrics.TopSQLReportDurationHistogram.WithLabelValues("plan", metrics.LblError) - topSQLReportRecordCounterHistogram = metrics.TopSQLReportDataHistogram.WithLabelValues("record") - topSQLReportSQLCountHistogram = metrics.TopSQLReportDataHistogram.WithLabelValues("sql") - topSQLReportPlanCountHistogram = metrics.TopSQLReportDataHistogram.WithLabelValues("plan") -) - -// RegisterSQL registers a normalized SQL string to a SQL digest. -// This function is thread-safe and efficient. +// Collect implements tracecpu.Collector. // -// Note that the normalized SQL string can be of >1M long. -// This function should be thread-safe, which means parallelly calling it in several goroutines should be fine. -// It should also return immediately, and do any CPU-intensive job asynchronously. -func (tsr *RemoteTopSQLReporter) RegisterSQL(sqlDigest []byte, normalizedSQL string, isInternal bool) { - if tsr.sqlMapLength.Load() >= variable.TopSQLVariable.MaxCollect.Load() { - ignoreExceedSQLCounter.Inc() - return - } - m := tsr.normalizedSQLMap.Load().(*sync.Map) - key := string(sqlDigest) - _, loaded := m.LoadOrStore(key, SQLMeta{ - normalizedSQL: normalizedSQL, - isInternal: isInternal, - }) - if !loaded { - tsr.sqlMapLength.Add(1) - } -} - -// RegisterPlan is like RegisterSQL, but for normalized plan strings. -// This function is thread-safe and efficient. -func (tsr *RemoteTopSQLReporter) RegisterPlan(planDigest []byte, normalizedBinaryPlan string) { - if tsr.planMapLength.Load() >= variable.TopSQLVariable.MaxCollect.Load() { - ignoreExceedPlanCounter.Inc() - return - } - m := tsr.normalizedPlanMap.Load().(*sync.Map) - key := string(planDigest) - _, loaded := m.LoadOrStore(key, normalizedBinaryPlan) - if !loaded { - tsr.planMapLength.Add(1) - } -} - -var _ DataSinkRegisterer = &RemoteTopSQLReporter{} - -// Register implements DataSinkRegisterer interface. -func (tsr *RemoteTopSQLReporter) Register(dataSink DataSink) error { - tsr.dataSinkMu.Lock() - defer tsr.dataSinkMu.Unlock() - - select { - case <-tsr.ctx.Done(): - return errors.New("reporter is closed") - default: - if len(tsr.dataSinks) >= 10 { - return errors.New("too many datasinks") - } - - tsr.dataSinks[dataSink] = struct{}{} - - if len(tsr.dataSinks) > 0 { - variable.TopSQLVariable.Enable.Store(true) - } - - return nil - } -} - -// Deregister implements DataSinkRegisterer interface. -func (tsr *RemoteTopSQLReporter) Deregister(dataSink DataSink) { - tsr.dataSinkMu.Lock() - defer tsr.dataSinkMu.Unlock() - - select { - case <-tsr.ctx.Done(): - default: - delete(tsr.dataSinks, dataSink) - - if len(tsr.dataSinks) == 0 { - variable.TopSQLVariable.Enable.Store(false) - } - } -} - -// Collect receives CPU time records for processing. WARN: It will drop the records if the processing is not in time. +// WARN: It will drop the DataRecords if the processing is not in time. // This function is thread-safe and efficient. func (tsr *RemoteTopSQLReporter) Collect(timestamp uint64, records []tracecpu.SQLCPUTimeRecord) { if len(records) == 0 { return } select { - case tsr.collectCPUDataChan <- cpuData{ - timestamp: timestamp, - records: records, - }: + case tsr.collectCPUDataChan <- cpuData{timestamp: timestamp, records: records}: default: // ignore if chan blocked ignoreCollectChannelFullCounter.Inc() } } -// CollectStmtStatsRecords receives stmtstats.StatementStatsRecord for processing. -// WARN: It will drop the records if the processing is not in time. -// This function is thread-safe and efficient. +// CollectStmtStatsRecords implements stmtstats.Collector. // -// CollectStmtStatsRecords implements stmtstats.Collector.CollectStmtStatsRecords. -func (tsr *RemoteTopSQLReporter) CollectStmtStatsRecords(records []stmtstats.StatementStatsRecord) { - if len(records) == 0 { +// WARN: It will drop the DataRecords if the processing is not in time. +// This function is thread-safe and efficient. +func (tsr *RemoteTopSQLReporter) CollectStmtStatsRecords(rs []stmtstats.StatementStatsRecord) { + if len(rs) == 0 { return } select { - case tsr.collectStmtRecordsChan <- records: + case tsr.collectStmtRecordsChan <- rs: default: // ignore if chan blocked ignoreCollectStmtChannelFullCounter.Inc() } } -// Close uses to close and release the reporter resource. -func (tsr *RemoteTopSQLReporter) Close() { - tsr.cancel() - - var m map[DataSink]struct{} - tsr.dataSinkMu.Lock() - m, tsr.dataSinks = tsr.dataSinks, make(map[DataSink]struct{}) - tsr.dataSinkMu.Unlock() - - for d := range m { - d.OnReporterClosing() - } -} - -func addEvictedCPUTime(collectTarget map[string]*dataPoints, timestamp uint64, totalCPUTimeMs uint32) { - if totalCPUTimeMs == 0 { - return - } - others, ok := collectTarget[keyOthers] - if !ok { - others = &dataPoints{tsIndex: map[uint64]int{}} - collectTarget[keyOthers] = others - } - others.appendCPUTime(timestamp, totalCPUTimeMs) -} - -func addEvictedStmtStatsItem(collectTarget map[string]*dataPoints, timestamp uint64, item *stmtstats.StatementStatsItem) { - others, ok := collectTarget[keyOthers] - if !ok { - others = &dataPoints{tsIndex: map[uint64]int{}} - collectTarget[keyOthers] = others - } - others.appendStmtStatsItem(timestamp, item) +// RegisterSQL implements TopSQLReporter. +// +// This function is thread-safe and efficient. +func (tsr *RemoteTopSQLReporter) RegisterSQL(sqlDigest []byte, normalizedSQL string, isInternal bool) { + tsr.normalizedSQLMap.register(sqlDigest, normalizedSQL, isInternal) } -// addEvictedIntoSortedDataPoints adds evicted dataPoints into others. -// Attention, this function depend on others dataPoints is sorted, and this function will modify evicted dataPoints -// to make sure it is sorted by timestamp. -func addEvictedIntoSortedDataPoints(others *dataPoints, evict *dataPoints) *dataPoints { - if others == nil { - others = &dataPoints{} - } - if evict == nil || len(evict.TimestampList) == 0 { - return others - } - if evict.isInvalid() { - logutil.BgLogger().Warn("[top-sql] data points is invalid, it should never happen", zap.Any("self", others), zap.Any("evict", evict)) - return others - } - // Sort the dataPoints by timestamp to fix the affect of time jump backward. - sort.Sort(evict) - if len(others.TimestampList) == 0 { - others.TimestampList = evict.TimestampList - others.CPUTimeMsList = evict.CPUTimeMsList - others.CPUTimeMsTotal = evict.CPUTimeMsTotal - others.StmtExecCountList = evict.StmtExecCountList - others.StmtKvExecCountList = evict.StmtKvExecCountList - others.StmtDurationSumNsList = evict.StmtDurationSumNsList - others.tsIndex = evict.tsIndex - return others - } - length := len(others.TimestampList) + len(evict.TimestampList) - timestampList := make([]uint64, 0, length) - cpuTimeMsList := make([]uint32, 0, length) - stmtExecCountList := make([]uint64, 0, length) - stmtKvExecCountList := make([]map[string]uint64, 0, length) - stmtDurationSumList := make([]uint64, 0, length) - i := 0 - j := 0 - for i < len(others.TimestampList) && j < len(evict.TimestampList) { - if others.TimestampList[i] == evict.TimestampList[j] { - timestampList = append(timestampList, others.TimestampList[i]) - cpuTimeMsList = append(cpuTimeMsList, others.CPUTimeMsList[i]+evict.CPUTimeMsList[j]) - stmtExecCountList = append(stmtExecCountList, others.StmtExecCountList[i]+evict.StmtExecCountList[j]) - stmtKvExecCountList = append(stmtKvExecCountList, mergeKvExecCountMap(others.StmtKvExecCountList[i], evict.StmtKvExecCountList[j])) - stmtDurationSumList = append(stmtDurationSumList, others.StmtDurationSumNsList[i]+evict.StmtDurationSumNsList[j]) - i++ - j++ - } else if others.TimestampList[i] < evict.TimestampList[j] { - timestampList = append(timestampList, others.TimestampList[i]) - cpuTimeMsList = append(cpuTimeMsList, others.CPUTimeMsList[i]) - stmtExecCountList = append(stmtExecCountList, others.StmtExecCountList[i]) - stmtKvExecCountList = append(stmtKvExecCountList, others.StmtKvExecCountList[i]) - stmtDurationSumList = append(stmtDurationSumList, others.StmtDurationSumNsList[i]) - i++ - } else { - timestampList = append(timestampList, evict.TimestampList[j]) - cpuTimeMsList = append(cpuTimeMsList, evict.CPUTimeMsList[j]) - stmtExecCountList = append(stmtExecCountList, evict.StmtExecCountList[j]) - stmtKvExecCountList = append(stmtKvExecCountList, evict.StmtKvExecCountList[j]) - stmtDurationSumList = append(stmtDurationSumList, evict.StmtDurationSumNsList[j]) - j++ - } - } - if i < len(others.TimestampList) { - timestampList = append(timestampList, others.TimestampList[i:]...) - cpuTimeMsList = append(cpuTimeMsList, others.CPUTimeMsList[i:]...) - stmtExecCountList = append(stmtExecCountList, others.StmtExecCountList[i:]...) - for _, l := range others.StmtKvExecCountList[i:] { - stmtKvExecCountList = append(stmtKvExecCountList, l) - } - stmtDurationSumList = append(stmtDurationSumList, others.StmtDurationSumNsList[i:]...) - } - if j < len(evict.TimestampList) { - timestampList = append(timestampList, evict.TimestampList[j:]...) - cpuTimeMsList = append(cpuTimeMsList, evict.CPUTimeMsList[j:]...) - stmtExecCountList = append(stmtExecCountList, evict.StmtExecCountList[j:]...) - for _, l := range evict.StmtKvExecCountList[j:] { - stmtKvExecCountList = append(stmtKvExecCountList, l) - } - stmtDurationSumList = append(stmtDurationSumList, evict.StmtDurationSumNsList[j:]...) - } - others.TimestampList = timestampList - others.CPUTimeMsList = cpuTimeMsList - others.StmtExecCountList = stmtExecCountList - others.StmtKvExecCountList = stmtKvExecCountList - others.StmtDurationSumNsList = stmtDurationSumList - others.CPUTimeMsTotal += evict.CPUTimeMsTotal - others.rebuildTsIndex() - return others +// RegisterPlan implements TopSQLReporter. +// +// This function is thread-safe and efficient. +func (tsr *RemoteTopSQLReporter) RegisterPlan(planDigest []byte, normalizedPlan string) { + tsr.normalizedPlanMap.register(planDigest, normalizedPlan) } -func mergeKvExecCountMap(a, b map[string]uint64) map[string]uint64 { - r := map[string]uint64{} - for ka, va := range a { - r[ka] = va - } - for kb, vb := range b { - r[kb] += vb - } - return r +// Close implements TopSQLReporter. +func (tsr *RemoteTopSQLReporter) Close() { + tsr.cancel() + tsr.OnReporterClosing() } +// collectWorker consumes and collects data from tracecpu.Collector/stmtstats.Collector. func (tsr *RemoteTopSQLReporter) collectWorker() { defer util.Recover("top-sql", "collectWorker", nil, false) - collectedData := make(map[string]*dataPoints) - evictedDigest := make(map[uint64]map[stmtstats.SQLPlanDigest]struct{}) currentReportInterval := variable.TopSQLVariable.ReportIntervalSeconds.Load() reportTicker := time.NewTicker(time.Second * time.Duration(currentReportInterval)) for { select { case data := <-tsr.collectCPUDataChan: - // On receiving data to collect: Write to local data array, and retain records with most CPU time. - tsr.doCollect(collectedData, evictedDigest, data.timestamp, data.records) - case data := <-tsr.collectStmtRecordsChan: - tsr.doCollectStmtRecords(collectedData, evictedDigest, data) + tsr.doCollect(data.timestamp, data.records) + case rs := <-tsr.collectStmtRecordsChan: + tsr.doCollectStmtRecords(rs) case <-reportTicker.C: - // We clean up evictedDigest before reporting, to avoid continuous accumulation. - evictedDigest = make(map[uint64]map[stmtstats.SQLPlanDigest]struct{}) - tsr.takeDataAndSendToReportChan(&collectedData) + tsr.takeDataAndSendToReportChan() // Update `reportTicker` if report interval changed. if newInterval := variable.TopSQLVariable.ReportIntervalSeconds.Load(); newInterval != currentReportInterval { currentReportInterval = newInterval @@ -649,157 +169,67 @@ func (tsr *RemoteTopSQLReporter) collectWorker() { } } -func encodeKey(buf *bytes.Buffer, sqlDigest, planDigest []byte) string { - buf.Reset() - buf.Write(sqlDigest) - buf.Write(planDigest) - return buf.String() -} - -func getTopNRecords(records []tracecpu.SQLCPUTimeRecord) (topN, shouldEvict []tracecpu.SQLCPUTimeRecord) { - maxStmt := int(variable.TopSQLVariable.MaxStatementCount.Load()) - if len(records) <= maxStmt { - return records, nil - } - if err := quickselect.QuickSelect(sqlCPUTimeRecordSlice(records), maxStmt); err != nil { - // skip eviction - return records, nil - } - return records[:maxStmt], records[maxStmt:] -} - -func getTopNDataPoints(records []*dataPoints) (topN, shouldEvict []*dataPoints) { - maxStmt := int(variable.TopSQLVariable.MaxStatementCount.Load()) - if len(records) <= maxStmt { - return records, nil - } - if err := quickselect.QuickSelect(dataPointsOrderByCPUTime(records), maxStmt); err != nil { - // skip eviction - return records, nil - } - return records[:maxStmt], records[maxStmt:] -} - -// doCollect collects top N records of each round into collectTarget, and evict the data that is not in top N. -// All the evicted record will be summary into the collectedData.others. -func (tsr *RemoteTopSQLReporter) doCollect( - collectTarget map[string]*dataPoints, - evictedDigest map[uint64]map[stmtstats.SQLPlanDigest]struct{}, - timestamp uint64, - records []tracecpu.SQLCPUTimeRecord) { +// doCollect collects top N cpuRecords of each round into tsr.collecting, and evict the +// data that is not in top N. All the evicted cpuRecords will be summary into the others. +func (tsr *RemoteTopSQLReporter) doCollect(timestamp uint64, rs cpuRecords) { defer util.Recover("top-sql", "doCollect", nil, false) - // Get top N records of each round records. - var evicted []tracecpu.SQLCPUTimeRecord - records, evicted = getTopNRecords(records) - - keyBuf := bytes.NewBuffer(make([]byte, 0, 64)) - // Collect the top N records to collectTarget for each round. - for _, record := range records { - key := encodeKey(keyBuf, record.SQLDigest, record.PlanDigest) - entry, exist := collectTarget[key] - if !exist { - collectTarget[key] = newDataPoints(record.SQLDigest, record.PlanDigest) - entry = collectTarget[key] - } - entry.appendCPUTime(timestamp, record.CPUTimeMs) + // Get top N cpuRecords of each round cpuRecords. Collect the top N to tsr.collecting + // for each round. SQL meta will not be evicted, since the evicted SQL can be appeared + // on other components (TiKV) TopN DataRecords. + var evicted cpuRecords + rs, evicted = rs.topN(int(variable.TopSQLVariable.MaxStatementCount.Load())) + for _, r := range rs { + tsr.collecting.getOrCreateRecord(r.SQLDigest, r.PlanDigest).appendCPUTime(timestamp, r.CPUTimeMs) } - if len(evicted) == 0 { return } - // Merge non Top N data as "others" (keyed by in the `keyOthers`) in `collectTarget`. - // SQL meta will not be evicted, since the evicted SQL can be appear on Other components (TiKV) TopN records. totalEvictedCPUTime := uint32(0) - for _, evict := range evicted { - totalEvictedCPUTime += evict.CPUTimeMs - - // Record which digests are evicted under each timestamp, and judge whether - // the corresponding CPUTime is evicted when stmtstats is collected, if it - // has been evicted, then we can ignore it directly. - if _, ok := evictedDigest[timestamp]; !ok { - evictedDigest[timestamp] = map[stmtstats.SQLPlanDigest]struct{}{} - } - evictedDigest[timestamp][stmtstats.SQLPlanDigest{ - SQLDigest: stmtstats.BinaryDigest(evict.SQLDigest), - PlanDigest: stmtstats.BinaryDigest(evict.PlanDigest), - }] = struct{}{} + for _, e := range evicted { + totalEvictedCPUTime += e.CPUTimeMs + // Mark which digests are evicted under each timestamp. + // We will determine whether the corresponding CPUTime has been evicted + // when collecting stmtstats. If so, then we can ignore it directly. + tsr.collecting.markAsEvicted(timestamp, e.SQLDigest, e.PlanDigest) } - addEvictedCPUTime(collectTarget, timestamp, totalEvictedCPUTime) + tsr.collecting.appendOthersCPUTime(timestamp, totalEvictedCPUTime) } -func (tsr *RemoteTopSQLReporter) doCollectStmtRecords( - collectTarget map[string]*dataPoints, - evictedDigest map[uint64]map[stmtstats.SQLPlanDigest]struct{}, - records []stmtstats.StatementStatsRecord) { +// doCollectStmtRecords collects []stmtstats.StatementStatsRecord into tsr.collecting. +// All the evicted items will be summary into the others. +func (tsr *RemoteTopSQLReporter) doCollectStmtRecords(rs []stmtstats.StatementStatsRecord) { defer util.Recover("top-sql", "doCollectStmtRecords", nil, false) - keyBuf := bytes.NewBuffer(make([]byte, 0, 64)) - for _, record := range records { - timestamp := uint64(record.Timestamp) - for digest, item := range record.Data { - sqlDigest := []byte(digest.SQLDigest) - planDigest := []byte(digest.PlanDigest) - if digestSet, ok := evictedDigest[timestamp]; ok { - if _, ok := digestSet[digest]; ok { - // This record has been evicted due to low CPUTime. - addEvictedStmtStatsItem(collectTarget, timestamp, item) - continue - } + for _, r := range rs { + timestamp := uint64(r.Timestamp) + for digest, item := range r.Data { + sqlDigest, planDigest := []byte(digest.SQLDigest), []byte(digest.PlanDigest) + if tsr.collecting.hasEvicted(timestamp, sqlDigest, planDigest) { + // This timestamp+sql+plan has been evicted due to low CPUTime. + tsr.collecting.appendOthersStmtStatsItem(timestamp, *item) + continue } - key := encodeKey(keyBuf, sqlDigest, planDigest) - entry, exist := collectTarget[key] - if !exist { - collectTarget[key] = newDataPoints(sqlDigest, planDigest) - entry = collectTarget[key] - } - entry.appendStmtStatsItem(timestamp, item) + tsr.collecting.getOrCreateRecord(sqlDigest, planDigest).appendStmtStatsItem(timestamp, *item) } } } -// takeDataAndSendToReportChan takes collected data and then send to the report channel for reporting. -func (tsr *RemoteTopSQLReporter) takeDataAndSendToReportChan(collectedDataPtr *map[string]*dataPoints) { - data := collectedData{ - records: *collectedDataPtr, - normalizedSQLMap: tsr.normalizedSQLMap.Load().(*sync.Map), - normalizedPlanMap: tsr.normalizedPlanMap.Load().(*sync.Map), - } - - // Reset data for next report. - *collectedDataPtr = make(map[string]*dataPoints) - tsr.normalizedSQLMap.Store(&sync.Map{}) - tsr.normalizedPlanMap.Store(&sync.Map{}) - tsr.sqlMapLength.Store(0) - tsr.planMapLength.Store(0) - +// takeDataAndSendToReportChan takes records data and then send to the report channel for reporting. +func (tsr *RemoteTopSQLReporter) takeDataAndSendToReportChan() { // Send to report channel. When channel is full, data will be dropped. select { - case tsr.reportCollectedDataChan <- data: + case tsr.reportCollectedDataChan <- collectedData{ + collected: tsr.collecting.take(), + normalizedSQLMap: tsr.normalizedSQLMap.take(), + normalizedPlanMap: tsr.normalizedPlanMap.take(), + }: default: // ignore if chan blocked ignoreReportChannelFullCounter.Inc() } } -type collectedData struct { - records map[string]*dataPoints - normalizedSQLMap *sync.Map - normalizedPlanMap *sync.Map -} - -// ReportData contains data that reporter sends to the agent -type ReportData struct { - // DataRecords contains the topN collected records and the `others` record which aggregation all records that is out of Top N. - DataRecords []tipb.TopSQLRecord - SQLMetas []tipb.SQLMeta - PlanMetas []tipb.PlanMeta -} - -func (d *ReportData) hasData() bool { - return len(d.DataRecords) != 0 || len(d.SQLMetas) != 0 || len(d.PlanMetas) != 0 -} - // reportWorker sends data to the gRPC endpoint from the `reportCollectedDataChan` one by one. func (tsr *RemoteTopSQLReporter) reportWorker() { defer util.Recover("top-sql", "reportWorker", nil, false) @@ -807,122 +237,32 @@ func (tsr *RemoteTopSQLReporter) reportWorker() { for { select { case data := <-tsr.reportCollectedDataChan: - // When `reportCollectedDataChan` receives something, there could be ongoing `RegisterSQL` and `RegisterPlan` running, - // who writes to the data structure that `data` contains. So we wait for a little while to ensure that - // these writes are finished. + // When `reportCollectedDataChan` receives something, there could be ongoing + // `RegisterSQL` and `RegisterPlan` running, who writes to the data structure + // that `data` contains. So we wait for a little while to ensure that writes + // are finished. time.Sleep(time.Millisecond * 100) - report := tsr.getReportData(data) - tsr.doReport(report) + // Get top N records from records. + rs := data.collected.topN(int(variable.TopSQLVariable.MaxStatementCount.Load())) + // Convert to protobuf data and do report. + tsr.doReport(&ReportData{ + DataRecords: rs.toProto(), + SQLMetas: data.normalizedSQLMap.toProto(), + PlanMetas: data.normalizedPlanMap.toProto(tsr.decodePlan), + }) case <-tsr.ctx.Done(): return } } } -// getReportData gets ReportData from the collectedData. -// This function will calculate the topN collected records and the `others` record which aggregation all records that is out of Top N. -func (tsr *RemoteTopSQLReporter) getReportData(collected collectedData) *ReportData { - records := getTopNFromCollected(collected) - return tsr.buildReportData(records, collected.normalizedSQLMap, collected.normalizedPlanMap) -} - -func getTopNFromCollected(collected collectedData) (records []*dataPoints) { - // Fetch TopN dataPoints. - others := collected.records[keyOthers] - delete(collected.records, keyOthers) - - records = make([]*dataPoints, 0, len(collected.records)) - for _, v := range collected.records { - if v.CPUTimeMsTotal > 0 { - // For a certain timestamp, when StmtStats is collected, but CPUTime - // has not been collected, we have not yet filled the evictedDigest data, - // so it is possible that a piece of data is evicted in CPUTime, but still - // exists in collectTarget due to StmtStats. So we are here to remove those - // data that have no CPUTime at all. - records = append(records, v) - } - } - - // Evict all records that is out of Top N. - var evicted []*dataPoints - records, evicted = getTopNDataPoints(records) - if others != nil { - // Sort the dataPoints by timestamp to fix the affect of time jump backward. - sort.Sort(others) - } - for _, evict := range evicted { - // SQL meta will not be evicted, since the evicted SQL can be appeared on Other components (TiKV) TopN records. - others = addEvictedIntoSortedDataPoints(others, evict) - } - - // append others which summarize all evicted item's cpu-time. - if others != nil && others.CPUTimeMsTotal > 0 { - records = append(records, others) - } - - return -} - -// buildReportData convert record data in dataPoints slice and meta data in sync.Map to ReportData. -// -// Attention, caller should guarantee no more reader or writer access `sqlMap` and `planMap`, because buildReportData -// will do heavy jobs in sync.Map.Range and it may block other readers and writers. -func (tsr *RemoteTopSQLReporter) buildReportData(records []*dataPoints, sqlMap *sync.Map, planMap *sync.Map) *ReportData { - res := &ReportData{ - DataRecords: make([]tipb.TopSQLRecord, 0, len(records)), - SQLMetas: make([]tipb.SQLMeta, 0, len(records)), - PlanMetas: make([]tipb.PlanMeta, 0, len(records)), - } - - for _, record := range records { - recordListStmtKvExecCount := make([]*tipb.TopSQLStmtKvExecCount, len(record.StmtKvExecCountList)) - for n, l := range record.StmtKvExecCountList { - recordListStmtKvExecCount[n] = &tipb.TopSQLStmtKvExecCount{ExecCount: l} - } - res.DataRecords = append(res.DataRecords, tipb.TopSQLRecord{ - RecordListTimestampSec: record.TimestampList, - RecordListCpuTimeMs: record.CPUTimeMsList, - RecordListStmtExecCount: record.StmtExecCountList, - RecordListStmtKvExecCount: recordListStmtKvExecCount, - RecordListStmtDurationSumNs: record.StmtDurationSumNsList, - SqlDigest: record.SQLDigest, - PlanDigest: record.PlanDigest, - }) - } - - sqlMap.Range(func(key, value interface{}) bool { - meta := value.(SQLMeta) - res.SQLMetas = append(res.SQLMetas, tipb.SQLMeta{ - SqlDigest: []byte(key.(string)), - NormalizedSql: meta.normalizedSQL, - IsInternalSql: meta.isInternal, - }) - return true - }) - - planMap.Range(func(key, value interface{}) bool { - planDecoded, errDecode := tsr.decodePlan(value.(string)) - if errDecode != nil { - logutil.BgLogger().Warn("[top-sql] decode plan failed", zap.Error(errDecode)) - return true - } - res.PlanMetas = append(res.PlanMetas, tipb.PlanMeta{ - PlanDigest: []byte(key.(string)), - NormalizedPlan: planDecoded, - }) - return true - }) - - return res -} - +// doReport sends ReportData to DataSinks. func (tsr *RemoteTopSQLReporter) doReport(data *ReportData) { defer util.Recover("top-sql", "doReport", nil, false) if !data.hasData() { return } - timeout := reportTimeout failpoint.Inject("resetTimeoutForTest", func(val failpoint.Value) { if val.(bool) { @@ -932,18 +272,18 @@ func (tsr *RemoteTopSQLReporter) doReport(data *ReportData) { } } }) - deadline := time.Now().Add(timeout) + _ = tsr.TrySend(data, time.Now().Add(timeout)) +} - tsr.dataSinkMu.Lock() - dataSinks := make([]DataSink, 0, len(tsr.dataSinks)) - for ds := range tsr.dataSinks { - dataSinks = append(dataSinks, ds) - } - tsr.dataSinkMu.Unlock() +// collectedData is used for transmission in the channel. +type collectedData struct { + collected *collecting + normalizedSQLMap *normalizedSQLMap + normalizedPlanMap *normalizedPlanMap +} - for _, ds := range dataSinks { - if err := ds.TrySend(data, deadline); err != nil { - logutil.BgLogger().Warn("[top-sql] failed to send data to datasink", zap.Error(err)) - } - } +// cpuData is used for transmission in the channel. +type cpuData struct { + timestamp uint64 + records []tracecpu.SQLCPUTimeRecord } diff --git a/util/topsql/reporter/reporter_test.go b/util/topsql/reporter/reporter_test.go index 0728f2dcc4db0..d9c57c3d1f122 100644 --- a/util/topsql/reporter/reporter_test.go +++ b/util/topsql/reporter/reporter_test.go @@ -18,17 +18,14 @@ import ( "sort" "strconv" "strings" - "sync" "testing" "time" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/util/topsql/reporter/mock" - "github.com/pingcap/tidb/util/topsql/stmtstats" "github.com/pingcap/tidb/util/topsql/tracecpu" "github.com/pingcap/tipb/go-tipb" - "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" ) @@ -321,131 +318,26 @@ func TestCollectCapacity(t *testing.T) { variable.TopSQLVariable.MaxCollect.Store(10000) registerSQL(5000) - require.Equal(t, int64(5000), tsr.sqlMapLength.Load()) + require.Equal(t, int64(5000), tsr.normalizedSQLMap.length.Load()) registerPlan(1000) - require.Equal(t, int64(1000), tsr.planMapLength.Load()) + require.Equal(t, int64(1000), tsr.normalizedPlanMap.length.Load()) registerSQL(20000) - require.Equal(t, int64(10000), tsr.sqlMapLength.Load()) + require.Equal(t, int64(10000), tsr.normalizedSQLMap.length.Load()) registerPlan(20000) - require.Equal(t, int64(10000), tsr.planMapLength.Load()) + require.Equal(t, int64(10000), tsr.normalizedPlanMap.length.Load()) variable.TopSQLVariable.MaxCollect.Store(20000) registerSQL(50000) - require.Equal(t, int64(20000), tsr.sqlMapLength.Load()) + require.Equal(t, int64(20000), tsr.normalizedSQLMap.length.Load()) registerPlan(50000) - require.Equal(t, int64(20000), tsr.planMapLength.Load()) + require.Equal(t, int64(20000), tsr.normalizedPlanMap.length.Load()) variable.TopSQLVariable.MaxStatementCount.Store(5000) - collectedData := make(map[string]*dataPoints) - tsr.doCollect(collectedData, map[uint64]map[stmtstats.SQLPlanDigest]struct{}{}, 1, genRecord(20000)) - require.Equal(t, 5001, len(collectedData)) - require.Equal(t, int64(20000), tsr.sqlMapLength.Load()) - require.Equal(t, int64(20000), tsr.planMapLength.Load()) -} - -func TestCollectOthers(t *testing.T) { - collectTarget := make(map[string]*dataPoints) - addEvictedCPUTime(collectTarget, 1, 10) - addEvictedCPUTime(collectTarget, 2, 20) - addEvictedCPUTime(collectTarget, 3, 30) - others := collectTarget[keyOthers] - require.Equal(t, uint64(60), others.CPUTimeMsTotal) - require.Equal(t, []uint64{1, 2, 3}, others.TimestampList) - require.Equal(t, []uint32{10, 20, 30}, others.CPUTimeMsList) - - others = addEvictedIntoSortedDataPoints(nil, others) - require.Equal(t, uint64(60), others.CPUTimeMsTotal) - - // test for time jump backward. - evict := &dataPoints{tsIndex: map[uint64]int{}} - evict.TimestampList = []uint64{3, 2, 4} - evict.CPUTimeMsList = []uint32{30, 20, 40} - evict.CPUTimeMsTotal = 90 - evict.StmtExecCountList = []uint64{0, 0, 0} - evict.StmtKvExecCountList = []map[string]uint64{nil, nil, nil} - evict.StmtDurationSumNsList = []uint64{0, 0, 0} - others = addEvictedIntoSortedDataPoints(others, evict) - require.Equal(t, uint64(150), others.CPUTimeMsTotal) - require.Equal(t, []uint64{1, 2, 3, 4}, others.TimestampList) - require.Equal(t, []uint32{10, 40, 60, 40}, others.CPUTimeMsList) -} - -func TestDataPoints(t *testing.T) { - // test for dataPoints invalid. - d := &dataPoints{} - d.TimestampList = []uint64{1} - d.CPUTimeMsList = []uint32{10, 30} - require.True(t, d.isInvalid()) - - // test for dataPoints sort. - d = &dataPoints{} - d.TimestampList = []uint64{1, 2, 5, 6, 3, 4} - d.CPUTimeMsList = []uint32{10, 20, 50, 60, 30, 40} - d.StmtExecCountList = []uint64{11, 12, 13, 14, 15, 16} - d.StmtKvExecCountList = []map[string]uint64{{"": 21}, {"": 22}, {"": 23}, {"": 24}, {"": 25}, {"": 26}} - d.StmtDurationSumNsList = []uint64{31, 32, 33, 34, 35, 36} - d.rebuildTsIndex() - sort.Sort(d) - require.Equal(t, []uint64{1, 2, 3, 4, 5, 6}, d.TimestampList) - require.Equal(t, []uint32{10, 20, 30, 40, 50, 60}, d.CPUTimeMsList) - require.Equal(t, []uint64{11, 12, 15, 16, 13, 14}, d.StmtExecCountList) - require.Equal(t, []map[string]uint64{{"": 21}, {"": 22}, {"": 25}, {"": 26}, {"": 23}, {"": 24}}, d.StmtKvExecCountList) - require.Equal(t, []uint64{31, 32, 35, 36, 33, 34}, d.StmtDurationSumNsList) - - // test for dataPoints merge. - d = &dataPoints{} - evict := &dataPoints{} - addEvictedIntoSortedDataPoints(d, evict) - evict.TimestampList = []uint64{1, 3} - evict.CPUTimeMsList = []uint32{10, 30} - evict.CPUTimeMsTotal = 40 - evict.StmtExecCountList = []uint64{0, 0} - evict.StmtKvExecCountList = []map[string]uint64{{}, {}} - evict.StmtDurationSumNsList = []uint64{0, 0} - evict.rebuildTsIndex() - addEvictedIntoSortedDataPoints(d, evict) - require.Equal(t, uint64(40), d.CPUTimeMsTotal) - require.Equal(t, []uint64{1, 3}, d.TimestampList) - require.Equal(t, []uint32{10, 30}, d.CPUTimeMsList) - - evict.TimestampList = []uint64{1, 2, 3, 4, 5} - evict.CPUTimeMsList = []uint32{10, 20, 30, 40, 50} - evict.CPUTimeMsTotal = 150 - evict.StmtExecCountList = []uint64{0, 0, 0, 0, 0} - evict.StmtKvExecCountList = []map[string]uint64{{}, {}, {}, {}, {}} - evict.StmtDurationSumNsList = []uint64{0, 0, 0, 0, 0} - evict.rebuildTsIndex() - addEvictedIntoSortedDataPoints(d, evict) - require.Equal(t, uint64(190), d.CPUTimeMsTotal) - require.Equal(t, []uint64{1, 2, 3, 4, 5}, d.TimestampList) - require.Equal(t, []uint32{20, 20, 60, 40, 50}, d.CPUTimeMsList) - - // test for time jump backward. - d = &dataPoints{} - evict = &dataPoints{} - evict.TimestampList = []uint64{3, 2} - evict.CPUTimeMsList = []uint32{30, 20} - evict.CPUTimeMsTotal = 50 - evict.StmtExecCountList = []uint64{0, 0} - evict.StmtKvExecCountList = []map[string]uint64{{}, {}} - evict.StmtDurationSumNsList = []uint64{0, 0} - evict.rebuildTsIndex() - addEvictedIntoSortedDataPoints(d, evict) - require.Equal(t, uint64(50), d.CPUTimeMsTotal) - require.Equal(t, []uint64{2, 3}, d.TimestampList) - require.Equal(t, []uint32{20, 30}, d.CPUTimeMsList) - - // test for merge invalid dataPoints - d = &dataPoints{} - evict = &dataPoints{} - evict.TimestampList = []uint64{1} - evict.CPUTimeMsList = []uint32{10, 30} - require.True(t, evict.isInvalid()) - addEvictedIntoSortedDataPoints(d, evict) - require.False(t, d.isInvalid()) - require.Nil(t, d.CPUTimeMsList) - require.Nil(t, d.TimestampList) + tsr.doCollect(1, genRecord(20000)) + require.Equal(t, 5001, len(tsr.collecting.records)) + require.Equal(t, int64(20000), tsr.normalizedSQLMap.length.Load()) + require.Equal(t, int64(20000), tsr.normalizedPlanMap.length.Load()) } func TestCollectInternal(t *testing.T) { @@ -586,313 +478,3 @@ func BenchmarkTopSQL_CollectAndEvict(b *testing.B) { populateCache(tsr, begin, end, uint64(i)) } } - -func TestStmtStatsReport(t *testing.T) { - variable.TopSQLVariable.MaxStatementCount.Store(1) - - tsr := NewRemoteTopSQLReporter(mockPlanBinaryDecoderFunc) - tsr.Close() // manual control - - r := tsr.getReportData(collectedData{ - normalizedSQLMap: &sync.Map{}, - normalizedPlanMap: &sync.Map{}, - records: map[string]*dataPoints{ - "S1P1": { - SQLDigest: []byte("S1"), - PlanDigest: []byte("P1"), - TimestampList: []uint64{1, 2, 3, 4}, - CPUTimeMsList: []uint32{11, 12, 13, 14}, - CPUTimeMsTotal: 11 + 12 + 13 + 14, - StmtExecCountList: []uint64{11, 12, 13, 14}, - StmtKvExecCountList: []map[string]uint64{{"": 11}, {"": 12}, {"": 13}, {"": 14}}, - StmtDurationSumNsList: []uint64{11, 12, 13, 14}, - }, - "S2P2": { - SQLDigest: []byte("S2"), - PlanDigest: []byte("P2"), - TimestampList: []uint64{1, 2, 3, 4}, - CPUTimeMsList: []uint32{21, 22, 23, 24}, - CPUTimeMsTotal: 21 + 22 + 23 + 24, - StmtExecCountList: []uint64{21, 22, 23, 24}, - StmtKvExecCountList: []map[string]uint64{{"": 21}, {"": 22}, {"": 23}, {"": 24}}, - StmtDurationSumNsList: []uint64{21, 22, 23, 24}, - }, - keyOthers: { - SQLDigest: []byte(nil), - PlanDigest: []byte(nil), - TimestampList: []uint64{1, 2, 3, 4}, - CPUTimeMsList: []uint32{91, 92, 93, 94}, - CPUTimeMsTotal: 91 + 92 + 93 + 94, - StmtExecCountList: []uint64{91, 92, 93, 94}, - StmtKvExecCountList: []map[string]uint64{{"": 91}, {"": 92}, {"": 93}, {"": 94}}, - StmtDurationSumNsList: []uint64{91, 92, 93, 94}, - }, - }, - }) - assert.True(t, r.hasData()) - assert.Len(t, r.DataRecords, 2) - - s2p2 := r.DataRecords[0] - assert.Equal(t, []byte("S2"), s2p2.SqlDigest) - assert.Equal(t, []byte("P2"), s2p2.PlanDigest) - assert.Equal(t, []uint64{1, 2, 3, 4}, s2p2.RecordListTimestampSec) - assert.Equal(t, []uint32{21, 22, 23, 24}, s2p2.RecordListCpuTimeMs) - assert.Equal(t, []uint64{21, 22, 23, 24}, s2p2.RecordListStmtExecCount) - assert.Equal(t, []uint64{21, 22, 23, 24}, s2p2.RecordListStmtDurationSumNs) - assert.Equal(t, []*tipb.TopSQLStmtKvExecCount{ - {ExecCount: map[string]uint64{"": 21}}, - {ExecCount: map[string]uint64{"": 22}}, - {ExecCount: map[string]uint64{"": 23}}, - {ExecCount: map[string]uint64{"": 24}}, - }, s2p2.RecordListStmtKvExecCount) - - others := r.DataRecords[1] - assert.Equal(t, []byte(nil), others.SqlDigest) - assert.Equal(t, []byte(nil), others.PlanDigest) - assert.Equal(t, []uint64{1, 2, 3, 4}, others.RecordListTimestampSec) - assert.Equal(t, []uint32{91 + 11, 92 + 12, 93 + 13, 94 + 14}, others.RecordListCpuTimeMs) - assert.Equal(t, []uint64{91 + 11, 92 + 12, 93 + 13, 94 + 14}, others.RecordListStmtExecCount) - assert.Equal(t, []uint64{91 + 11, 92 + 12, 93 + 13, 94 + 14}, others.RecordListStmtDurationSumNs) - assert.Equal(t, []*tipb.TopSQLStmtKvExecCount{ - {ExecCount: map[string]uint64{"": 91 + 11}}, - {ExecCount: map[string]uint64{"": 92 + 12}}, - {ExecCount: map[string]uint64{"": 93 + 13}}, - {ExecCount: map[string]uint64{"": 94 + 14}}, - }, others.RecordListStmtKvExecCount) -} - -func TestStmtStatsCollect(t *testing.T) { - variable.TopSQLVariable.MaxStatementCount.Store(1000) - - tsr := NewRemoteTopSQLReporter(mockPlanBinaryDecoderFunc) - tsr.Close() // manual control - - total := map[string]*dataPoints{} - evicted := map[uint64]map[stmtstats.SQLPlanDigest]struct{}{} - // TimestampList: [] - // CPUTimeList: [] - // ExecCountList: [] - // KvExecCountList: [] - - collectCPUTime(tsr, total, evicted, "S1", "P1", 1, 1) - // TimestampList: [1] - // CPUTimeList: [1] - // ExecCountList: [0] - // KvExecCountList: [0] - - collectStmtStats(tsr, total, evicted, "S1", "P1", 1, 2, map[string]uint64{"": 3}) - // TimestampList: [1] - // CPUTimeList: [1] - // ExecCountList: [2] - // KvExecCountList: [3] - - collectCPUTime(tsr, total, evicted, "S1", "P1", 2, 1) - // TimestampList: [1, 2] - // CPUTimeList: [1, 1] - // ExecCountList: [2, 0] - // KvExecCountList: [3, 0] - - collectCPUTime(tsr, total, evicted, "S1", "P1", 3, 1) - // TimestampList: [1, 2, 3] - // CPUTimeList: [1, 1, 1] - // ExecCountList: [2, 0, 0] - // KvExecCountList: [3, 0, 0] - - collectStmtStats(tsr, total, evicted, "S1", "P1", 3, 2, map[string]uint64{"": 3}) - // TimestampList: [1, 2, 3] - // CPUTimeList: [1, 1, 1] - // ExecCountList: [2, 0, 2] - // KvExecCountList: [3, 0, 3] - - collectStmtStats(tsr, total, evicted, "S1", "P1", 2, 2, map[string]uint64{"": 3}) - // TimestampList: [1, 2, 3] - // CPUTimeList: [1, 1, 1] - // ExecCountList: [2, 2, 2] - // KvExecCountList: [3, 3, 3] - - assert.Empty(t, evicted) - data, ok := total["S1P1"] - assert.True(t, ok) - assert.Equal(t, []byte("S1"), data.SQLDigest) - assert.Equal(t, []byte("P1"), data.PlanDigest) - assert.Equal(t, uint64(3), data.CPUTimeMsTotal) - assert.Equal(t, []uint64{1, 2, 3}, data.TimestampList) - assert.Equal(t, []uint32{1, 1, 1}, data.CPUTimeMsList) - assert.Equal(t, []uint64{2, 2, 2}, data.StmtExecCountList) - assert.Equal(t, []map[string]uint64{{"": 3}, {"": 3}, {"": 3}}, data.StmtKvExecCountList) -} - -func TestStmtStatsCollectEvicted(t *testing.T) { - variable.TopSQLVariable.MaxStatementCount.Store(2) - - tsr := NewRemoteTopSQLReporter(mockPlanBinaryDecoderFunc) - tsr.Close() // manual control - - total := map[string]*dataPoints{} - evicted := map[uint64]map[stmtstats.SQLPlanDigest]struct{}{} - - tsr.doCollect(total, evicted, 1, []tracecpu.SQLCPUTimeRecord{ - {SQLDigest: []byte("S1"), PlanDigest: []byte("P1"), CPUTimeMs: 1}, - {SQLDigest: []byte("S2"), PlanDigest: []byte("P2"), CPUTimeMs: 2}, - {SQLDigest: []byte("S3"), PlanDigest: []byte("P3"), CPUTimeMs: 3}, - }) - // S2P2: - // TimestampList: [1] - // CPUTimeList: [2] - // ExecCountList: [0] - // KvExecCountList: [0] - // - // S3P3: - // TimestampList: [1] - // CPUTimeList: [3] - // ExecCountList: [0] - // KvExecCountList: [0] - // - // others: - // TimestampList: [1] - // CPUTimeList: [1] - // ExecCountList: [0] - // KvExecCountList: [0] - // - // evicted: {1: S1P1} - - collectStmtStats(tsr, total, evicted, "S1", "P1", 1, 1, map[string]uint64{"": 1}) - // S2P2: - // TimestampList: [1] - // CPUTimeList: [2] - // ExecCountList: [0] - // KvExecCountList: [0] - // - // S3P3: - // TimestampList: [1] - // CPUTimeList: [3] - // ExecCountList: [0] - // KvExecCountList: [0] - // - // others: - // TimestampList: [1] - // CPUTimeList: [1] - // ExecCountList: [1] - // KvExecCountList: [1] - // - // evicted: {1: S1P1} - - collectStmtStats(tsr, total, evicted, "S2", "P2", 1, 2, map[string]uint64{"": 2}) - // S2P2: - // TimestampList: [1] - // CPUTimeList: [2] - // ExecCountList: [2] - // KvExecCountList: [2] - // - // S3P3: - // TimestampList: [1] - // CPUTimeList: [3] - // ExecCountList: [0] - // KvExecCountList: [0] - // - // others: - // TimestampList: [1] - // CPUTimeList: [1] - // ExecCountList: [1] - // KvExecCountList: [1] - // - // evicted: {1: S1P1} - - collectStmtStats(tsr, total, evicted, "S3", "P3", 1, 3, map[string]uint64{"": 3}) - // S2P2: - // TimestampList: [1] - // CPUTimeList: [2] - // ExecCountList: [2] - // KvExecCountList: [2] - // - // S3P3: - // TimestampList: [1] - // CPUTimeList: [3] - // ExecCountList: [3] - // KvExecCountList: [3] - // - // others: - // TimestampList: [1] - // CPUTimeList: [1] - // ExecCountList: [1] - // KvExecCountList: [1] - // - // evicted: {1: S1P1} - - assert.Len(t, evicted, 1) - m, ok := evicted[1] - assert.True(t, ok) - _, ok = m[stmtstats.SQLPlanDigest{SQLDigest: "S1", PlanDigest: "P1"}] - assert.True(t, ok) - _, ok = total["S1P1"] - assert.False(t, ok) - - s2p2, ok := total["S2P2"] - assert.True(t, ok) - assert.Equal(t, []byte("S2"), s2p2.SQLDigest) - assert.Equal(t, []byte("P2"), s2p2.PlanDigest) - assert.Equal(t, uint64(2), s2p2.CPUTimeMsTotal) - assert.Equal(t, []uint64{1}, s2p2.TimestampList) - assert.Equal(t, []uint32{2}, s2p2.CPUTimeMsList) - assert.Equal(t, []uint64{2}, s2p2.StmtExecCountList) - assert.Equal(t, []map[string]uint64{{"": 2}}, s2p2.StmtKvExecCountList) - - s3p3, ok := total["S3P3"] - assert.True(t, ok) - assert.Equal(t, []byte("S3"), s3p3.SQLDigest) - assert.Equal(t, []byte("P3"), s3p3.PlanDigest) - assert.Equal(t, uint64(3), s3p3.CPUTimeMsTotal) - assert.Equal(t, []uint64{1}, s3p3.TimestampList) - assert.Equal(t, []uint32{3}, s3p3.CPUTimeMsList) - assert.Equal(t, []uint64{3}, s3p3.StmtExecCountList) - assert.Equal(t, []map[string]uint64{{"": 3}}, s3p3.StmtKvExecCountList) - - others, ok := total[keyOthers] - assert.True(t, ok) - assert.Equal(t, []byte(nil), others.SQLDigest) - assert.Equal(t, []byte(nil), others.PlanDigest) - assert.Equal(t, uint64(1), others.CPUTimeMsTotal) - assert.Equal(t, []uint64{1}, others.TimestampList) - assert.Equal(t, []uint32{1}, others.CPUTimeMsList) - assert.Equal(t, []uint64{1}, others.StmtExecCountList) - assert.Equal(t, []map[string]uint64{{"": 1}}, others.StmtKvExecCountList) -} - -func collectCPUTime( - tsr *RemoteTopSQLReporter, - total map[string]*dataPoints, - evicted map[uint64]map[stmtstats.SQLPlanDigest]struct{}, - sqlDigest, planDigest string, - ts uint64, - cpuTime uint32) { - tsr.doCollect(total, evicted, ts, []tracecpu.SQLCPUTimeRecord{{ - SQLDigest: []byte(sqlDigest), - PlanDigest: []byte(planDigest), - CPUTimeMs: cpuTime, - }}) -} - -func collectStmtStats( - tsr *RemoteTopSQLReporter, - total map[string]*dataPoints, - evicted map[uint64]map[stmtstats.SQLPlanDigest]struct{}, - sqlDigest, planDigest string, - ts int64, - execCount uint64, - kvExecCount map[string]uint64) { - tsr.doCollectStmtRecords(total, evicted, []stmtstats.StatementStatsRecord{{ - Timestamp: ts, - Data: stmtstats.StatementStatsMap{ - stmtstats.SQLPlanDigest{ - SQLDigest: stmtstats.BinaryDigest(sqlDigest), - PlanDigest: stmtstats.BinaryDigest(planDigest), - }: &stmtstats.StatementStatsItem{ - ExecCount: execCount, - KvStatsItem: stmtstats.KvStatementStatsItem{ - KvExecCount: kvExecCount, - }, - // TODO(mornyx): add duration - }, - }, - }}) -} diff --git a/util/topsql/reporter/single_target.go b/util/topsql/reporter/single_target.go index c0af5f59c5bd8..05d0791dfe069 100644 --- a/util/topsql/reporter/single_target.go +++ b/util/topsql/reporter/single_target.go @@ -30,6 +30,12 @@ import ( "google.golang.org/grpc/backoff" ) +const ( + dialTimeout = 5 * time.Second + grpcInitialWindowSize = 1 << 30 + grpcInitialConnWindowSize = 1 << 30 +) + // SingleTargetDataSink reports data to grpc servers. type SingleTargetDataSink struct { ctx context.Context diff --git a/util/topsql/topsql_test.go b/util/topsql/topsql_test.go index 462c6ffb70aa2..7ca09be4205af 100644 --- a/util/topsql/topsql_test.go +++ b/util/topsql/topsql_test.go @@ -279,7 +279,7 @@ func TestTopSQLPubSub(t *testing.T) { sqlMetas := make(map[string]*tipb.SQLMeta) planMetas := make(map[string]string) - records := make(map[string]*tipb.CPUTimeRecord) + records := make(map[string]*tipb.TopSQLRecord) for { r, err := stream.Recv() @@ -287,8 +287,8 @@ func TestTopSQLPubSub(t *testing.T) { break } - if r.GetRecord() != nil { - rec := r.GetRecord() + if r.GetDataRecord() != nil { + rec := r.GetDataRecord() if _, ok := records[string(rec.SqlDigest)]; !ok { records[string(rec.SqlDigest)] = rec } else { diff --git a/util/topsql/tracecpu/mock/mock.go b/util/topsql/tracecpu/mock/mock.go index 62125d202f67c..0a33b8a6bcf68 100644 --- a/util/topsql/tracecpu/mock/mock.go +++ b/util/topsql/tracecpu/mock/mock.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/util/hack" "github.com/pingcap/tidb/util/logutil" + "github.com/pingcap/tidb/util/topsql/stmtstats" "github.com/pingcap/tidb/util/topsql/tracecpu" "go.uber.org/atomic" "go.uber.org/zap" @@ -73,6 +74,8 @@ func (c *TopSQLCollector) Collect(ts uint64, stats []tracecpu.SQLCPUTimeRecord) } } +func (c *TopSQLCollector) CollectStmtStatsRecords(rs []stmtstats.StatementStatsRecord) {} + // GetSQLStatsBySQLWithRetry uses for testing. func (c *TopSQLCollector) GetSQLStatsBySQLWithRetry(sql string, planIsNotNull bool) []*tracecpu.SQLCPUTimeRecord { after := time.After(time.Second * 10) From 435b3da8bb05db4ac09c49b40306969ba5b49901 Mon Sep 17 00:00:00 2001 From: mornyx Date: Sun, 26 Dec 2021 21:35:25 +0800 Subject: [PATCH 05/15] Add comment Signed-off-by: mornyx --- util/topsql/tracecpu/mock/mock.go | 1 + 1 file changed, 1 insertion(+) diff --git a/util/topsql/tracecpu/mock/mock.go b/util/topsql/tracecpu/mock/mock.go index 0a33b8a6bcf68..ff229783674b8 100644 --- a/util/topsql/tracecpu/mock/mock.go +++ b/util/topsql/tracecpu/mock/mock.go @@ -74,6 +74,7 @@ func (c *TopSQLCollector) Collect(ts uint64, stats []tracecpu.SQLCPUTimeRecord) } } +// CollectStmtStatsRecords implements stmtstats.Collector. func (c *TopSQLCollector) CollectStmtStatsRecords(rs []stmtstats.StatementStatsRecord) {} // GetSQLStatsBySQLWithRetry uses for testing. From ec2e155be2796df3e6d5f0b3de4cb869356beb3f Mon Sep 17 00:00:00 2001 From: mornyx Date: Mon, 27 Dec 2021 10:37:42 +0800 Subject: [PATCH 06/15] Fix implicit memory aliasing in for loop Signed-off-by: mornyx --- util/topsql/reporter/datamodel.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/util/topsql/reporter/datamodel.go b/util/topsql/reporter/datamodel.go index ef150a96f732e..239008f55afa7 100644 --- a/util/topsql/reporter/datamodel.go +++ b/util/topsql/reporter/datamodel.go @@ -492,7 +492,8 @@ func (c *collecting) topN(n int) records { } else { others = newRecord(nil, nil) } - for _, e := range evicted { + for _, evict := range evicted { + e := evict // Avoid implicit memory aliasing in for loop. others.merge(&e) } if others.totalCPUTimeMs > 0 { From abf988cdeb808842b23e27c5e6201639f9ec2bca Mon Sep 17 00:00:00 2001 From: mornyx Date: Mon, 27 Dec 2021 12:23:46 +0800 Subject: [PATCH 07/15] Upgrade tipb, use TopSQLRecordItem list Signed-off-by: mornyx --- go.mod | 2 +- go.sum | 4 +- util/topsql/reporter/datamodel.go | 58 +++++++++++++------------- util/topsql/reporter/datamodel_test.go | 54 +++++++++++------------- util/topsql/reporter/reporter_test.go | 44 ++++++++++--------- util/topsql/topsql_test.go | 15 ++++--- 6 files changed, 87 insertions(+), 90 deletions(-) diff --git a/go.mod b/go.mod index 5e88cc1f041f0..866b5a834c64a 100644 --- a/go.mod +++ b/go.mod @@ -104,4 +104,4 @@ replace github.com/pingcap/tidb/parser => ./parser replace github.com/dgrijalva/jwt-go => github.com/form3tech-oss/jwt-go v3.2.6-0.20210809144907-32ab6a8243d7+incompatible // TODO(mornyx): remove -replace github.com/pingcap/tipb => github.com/mornyx/tipb v0.0.0-20211222170402-84c2de94b2be +replace github.com/pingcap/tipb => github.com/mornyx/tipb v0.0.0-20211227042052-974e18b70381 diff --git a/go.sum b/go.sum index 8340a44f3e80d..8e710468dcf51 100644 --- a/go.sum +++ b/go.sum @@ -511,8 +511,8 @@ github.com/modern-go/reflect2 v1.0.1 h1:9f412s+6RmYXLWZSEzVVgPGK7C2PphHj5RJrvfx9 github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= github.com/montanaflynn/stats v0.5.0 h1:2EkzeTSqBB4V4bJwWrt5gIIrZmpJBcoIRGS2kWLgzmk= github.com/montanaflynn/stats v0.5.0/go.mod h1:wL8QJuTMNUDYhXwkmfOly8iTdp5TEcJFWZD2D7SIkUc= -github.com/mornyx/tipb v0.0.0-20211222170402-84c2de94b2be h1:0CdtbnVAi+bJLkiVpGEj5o6KfpEZXFQEy83Fnzie06w= -github.com/mornyx/tipb v0.0.0-20211222170402-84c2de94b2be/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= +github.com/mornyx/tipb v0.0.0-20211227042052-974e18b70381 h1:OlTIzfq5zFkfl33s0LUTr9wiutG5ZQNBd0ixlbnKVVU= +github.com/mornyx/tipb v0.0.0-20211227042052-974e18b70381/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= github.com/moul/http2curl v1.0.0/go.mod h1:8UbvGypXm98wA/IqH45anm5Y2Z6ep6O31QGOAZ3H0fQ= github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/nats-io/nats.go v1.8.1/go.mod h1:BrFz9vVn0fU3AcH9Vn4Kd7W0NpJ651tD5omQ3M8LwxM= diff --git a/util/topsql/reporter/datamodel.go b/util/topsql/reporter/datamodel.go index 239008f55afa7..07dcc71625f94 100644 --- a/util/topsql/reporter/datamodel.go +++ b/util/topsql/reporter/datamodel.go @@ -75,6 +75,19 @@ func newTsItem() tsItem { } } +// toProto converts the tsItem to the corresponding protobuf representation. +func (i *tsItem) toProto() *tipb.TopSQLRecordItem { + return &tipb.TopSQLRecordItem{ + TimestampSec: i.timestamp, + CpuTimeMs: i.cpuTimeMs, + StmtExecCount: i.stmtStats.ExecCount, + StmtKvExecCount: i.stmtStats.KvStatsItem.KvExecCount, + // TODO: add duration + // StmtDurationSumNs: ti.stmtStats., + // Convert more indicators here. + } +} + var _ sort.Interface = &tsItems{} // tsItems is a sortable list of tsItem, sort by tsItem.timestamp (asc). @@ -92,6 +105,19 @@ func (ts tsItems) Swap(i, j int) { ts[i], ts[j] = ts[j], ts[i] } +// toProto converts the tsItems to the corresponding protobuf representation. +func (ts tsItems) toProto() []*tipb.TopSQLRecordItem { + capacity := len(ts) + if capacity == 0 { + return nil + } + items := make([]*tipb.TopSQLRecordItem, 0, capacity) + for _, i := range ts { + items = append(items, i.toProto()) + } + return items +} + var _ sort.Interface = &record{} // record represents the cumulative tsItem in current minute window. @@ -332,36 +358,10 @@ func (r *record) rebuildTsIndex() { // toProto converts the record to the corresponding protobuf representation. func (r *record) toProto() tipb.TopSQLRecord { - capacity := len(r.tsItems) - if capacity == 0 { - return tipb.TopSQLRecord{ - SqlDigest: r.sqlDigest, - PlanDigest: r.planDigest, - } - } - timestampList := make([]uint64, 0, capacity) - cpuTimeList := make([]uint32, 0, capacity) - execCountList := make([]uint64, 0, capacity) - durationSumList := make([]uint64, 0, capacity) - kvExecCountList := make([]*tipb.TopSQLStmtKvExecCount, 0, capacity) - for _, item := range r.tsItems { - timestampList = append(timestampList, item.timestamp) - cpuTimeList = append(cpuTimeList, item.cpuTimeMs) - execCountList = append(execCountList, item.stmtStats.ExecCount) - kvExecCountList = append(kvExecCountList, &tipb.TopSQLStmtKvExecCount{ - ExecCount: item.stmtStats.KvStatsItem.KvExecCount, - }) - // TODO: append duration - } return tipb.TopSQLRecord{ - SqlDigest: r.sqlDigest, - PlanDigest: r.planDigest, - RecordListTimestampSec: timestampList, - RecordListCpuTimeMs: cpuTimeList, - RecordListStmtExecCount: execCountList, - RecordListStmtKvExecCount: kvExecCountList, - RecordListStmtDurationSumNs: durationSumList, - // Add more indicators here. + SqlDigest: r.sqlDigest, + PlanDigest: r.planDigest, + Items: r.tsItems.toProto(), } } diff --git a/util/topsql/reporter/datamodel_test.go b/util/topsql/reporter/datamodel_test.go index 3a0ba7b54fcac..da5c7ac769ad9 100644 --- a/util/topsql/reporter/datamodel_test.go +++ b/util/topsql/reporter/datamodel_test.go @@ -26,6 +26,22 @@ import ( "github.com/stretchr/testify/assert" ) +func Test_tsItem_toProto(t *testing.T) { + item := &tsItem{ + timestamp: 1, + cpuTimeMs: 2, + stmtStats: stmtstats.StatementStatsItem{ + ExecCount: 3, + KvStatsItem: stmtstats.KvStatementStatsItem{KvExecCount: map[string]uint64{"": 4}}, + }, + } + pb := item.toProto() + assert.Equal(t, uint64(1), pb.TimestampSec) + assert.Equal(t, uint32(2), pb.CpuTimeMs) + assert.Equal(t, uint64(3), pb.StmtExecCount) + assert.Equal(t, uint64(4), pb.StmtKvExecCount[""]) +} + func Test_tsItems_Sort(t *testing.T) { items := tsItems{ {timestamp: 2}, @@ -38,6 +54,12 @@ func Test_tsItems_Sort(t *testing.T) { assert.Equal(t, uint64(3), items[2].timestamp) } +func Test_tsItems_toProto(t *testing.T) { + items := &tsItems{{}, {}, {}} + pb := items.toProto() + assert.Len(t, pb, 3) +} + func Test_record_Sort(t *testing.T) { r := record{ tsItems: tsItems{ @@ -165,40 +187,12 @@ func Test_record_toProto(t *testing.T) { sqlDigest: []byte("SQL-1"), planDigest: []byte("PLAN-1"), totalCPUTimeMs: 123, - tsItems: tsItems{{ - timestamp: 1, - cpuTimeMs: 1, - stmtStats: stmtstats.StatementStatsItem{ - ExecCount: 1, - KvStatsItem: stmtstats.KvStatementStatsItem{KvExecCount: map[string]uint64{"": 1}}, - }, - }, { - timestamp: 2, - cpuTimeMs: 2, - stmtStats: stmtstats.StatementStatsItem{ - ExecCount: 2, - KvStatsItem: stmtstats.KvStatementStatsItem{KvExecCount: map[string]uint64{"": 2}}, - }, - }, { - timestamp: 3, - cpuTimeMs: 3, - stmtStats: stmtstats.StatementStatsItem{ - ExecCount: 3, - KvStatsItem: stmtstats.KvStatementStatsItem{KvExecCount: map[string]uint64{"": 3}}, - }, - }}, + tsItems: tsItems{{}, {}, {}}, } pb := r.toProto() assert.Equal(t, []byte("SQL-1"), pb.SqlDigest) assert.Equal(t, []byte("PLAN-1"), pb.PlanDigest) - assert.Equal(t, []uint64{1, 2, 3}, pb.RecordListTimestampSec) - assert.Equal(t, []uint32{1, 2, 3}, pb.RecordListCpuTimeMs) - assert.Equal(t, []uint64{1, 2, 3}, pb.RecordListStmtExecCount) - assert.Equal(t, []*tipb.TopSQLStmtKvExecCount{ - {ExecCount: map[string]uint64{"": 1}}, - {ExecCount: map[string]uint64{"": 2}}, - {ExecCount: map[string]uint64{"": 3}}, - }, pb.RecordListStmtKvExecCount) + assert.Len(t, pb.Items, 3) } func Test_records_Sort(t *testing.T) { diff --git a/util/topsql/reporter/reporter_test.go b/util/topsql/reporter/reporter_test.go index d9c57c3d1f122..df612fc6c5ba0 100644 --- a/util/topsql/reporter/reporter_test.go +++ b/util/topsql/reporter/reporter_test.go @@ -126,13 +126,10 @@ func TestCollectAndSendBatch(t *testing.T) { require.NoError(t, err) id = n } - require.Len(t, req.RecordListCpuTimeMs, 1) - for i := range req.RecordListCpuTimeMs { - require.Equal(t, uint32(id), req.RecordListCpuTimeMs[i]) - } - require.Len(t, req.RecordListTimestampSec, 1) - for i := range req.RecordListTimestampSec { - require.Equal(t, uint64(1), req.RecordListTimestampSec[i]) + require.Len(t, req.Items, 1) + for i := range req.Items { + require.Equal(t, uint64(1), req.Items[i].TimestampSec) + require.Equal(t, uint32(id), req.Items[i].CpuTimeMs) } sqlMeta, exist := agentServer.GetSQLMetaByDigestBlocking(req.SqlDigest, time.Second) require.True(t, exist) @@ -168,19 +165,18 @@ func TestCollectAndEvicted(t *testing.T) { require.NoError(t, err) id = n } - require.Len(t, req.RecordListTimestampSec, 1) - require.Equal(t, uint64(2), req.RecordListTimestampSec[0]) - require.Len(t, req.RecordListCpuTimeMs, 1) + require.Len(t, req.Items, 1) + require.Equal(t, uint64(2), req.Items[0].TimestampSec) if id == 0 { // test for others require.Nil(t, req.SqlDigest) require.Nil(t, req.PlanDigest) // 12502500 is the sum of all evicted item's cpu time. 1 + 2 + 3 + ... + 5000 = (1 + 5000) * 2500 = 12502500 - require.Equal(t, 12502500, int(req.RecordListCpuTimeMs[0])) + require.Equal(t, 12502500, int(req.Items[0].CpuTimeMs)) continue } require.Greater(t, id, maxSQLNum) - require.Equal(t, uint32(id), req.RecordListCpuTimeMs[0]) + require.Equal(t, uint32(id), req.Items[0].CpuTimeMs) sqlMeta, exist := agentServer.GetSQLMetaByDigestBlocking(req.SqlDigest, time.Second) require.True(t, exist) require.Equal(t, "sqlNormalized"+strconv.Itoa(id), sqlMeta.NormalizedSql) @@ -264,15 +260,21 @@ func TestCollectAndTopN(t *testing.T) { }) getTotalCPUTime := func(record *tipb.TopSQLRecord) int { total := uint32(0) - for _, v := range record.RecordListCpuTimeMs { - total += v + for _, i := range record.Items { + total += i.CpuTimeMs } return int(total) } require.Nil(t, results[0].SqlDigest) require.Equal(t, 5, getTotalCPUTime(results[0])) - require.Equal(t, []uint64{0, 1, 3, 4}, results[0].RecordListTimestampSec) - require.Equal(t, []uint32{1, 2, 1, 1}, results[0].RecordListCpuTimeMs) + require.Equal(t, uint64(0), results[0].Items[0].TimestampSec) + require.Equal(t, uint64(1), results[0].Items[1].TimestampSec) + require.Equal(t, uint64(3), results[0].Items[2].TimestampSec) + require.Equal(t, uint64(4), results[0].Items[3].TimestampSec) + require.Equal(t, uint32(1), results[0].Items[0].CpuTimeMs) + require.Equal(t, uint32(2), results[0].Items[1].CpuTimeMs) + require.Equal(t, uint32(1), results[0].Items[2].CpuTimeMs) + require.Equal(t, uint32(1), results[0].Items[3].CpuTimeMs) require.Equal(t, []byte("sqlDigest1"), results[1].SqlDigest) require.Equal(t, 5, getTotalCPUTime(results[1])) require.Equal(t, []byte("sqlDigest3"), results[2].SqlDigest) @@ -407,8 +409,9 @@ func TestMultipleDataSinks(t *testing.T) { require.Len(t, d.DataRecords, 1) require.Equal(t, []byte("sqlDigest1"), d.DataRecords[0].SqlDigest) require.Equal(t, []byte("planDigest1"), d.DataRecords[0].PlanDigest) - require.Equal(t, []uint64{3}, d.DataRecords[0].RecordListTimestampSec) - require.Equal(t, []uint32{2}, d.DataRecords[0].RecordListCpuTimeMs) + require.Len(t, d.DataRecords[0].Items, 1) + require.Equal(t, uint64(3), d.DataRecords[0].Items[0].TimestampSec) + require.Equal(t, uint32(2), d.DataRecords[0].Items[0].CpuTimeMs) require.Equal(t, []tipb.SQLMeta{{ SqlDigest: []byte("sqlDigest1"), @@ -437,8 +440,9 @@ func TestMultipleDataSinks(t *testing.T) { require.Len(t, d.DataRecords, 1) require.Equal(t, []byte("sqlDigest4"), d.DataRecords[0].SqlDigest) require.Equal(t, []byte("planDigest4"), d.DataRecords[0].PlanDigest) - require.Equal(t, []uint64{6}, d.DataRecords[0].RecordListTimestampSec) - require.Equal(t, []uint32{5}, d.DataRecords[0].RecordListCpuTimeMs) + require.Len(t, d.DataRecords[0].Items, 1) + require.Equal(t, uint64(6), d.DataRecords[0].Items[0].TimestampSec) + require.Equal(t, uint32(5), d.DataRecords[0].Items[0].CpuTimeMs) require.Equal(t, []tipb.SQLMeta{{ SqlDigest: []byte("sqlDigest4"), diff --git a/util/topsql/topsql_test.go b/util/topsql/topsql_test.go index 7ca09be4205af..82e9f0d282ec8 100644 --- a/util/topsql/topsql_test.go +++ b/util/topsql/topsql_test.go @@ -162,8 +162,8 @@ func TestTopSQLReporter(t *testing.T) { records := server.GetLatestRecords() checkSQLPlanMap := map[string]struct{}{} for _, req := range records { - require.Greater(t, len(req.RecordListCpuTimeMs), 0) - require.Greater(t, req.RecordListCpuTimeMs[0], uint32(0)) + require.Greater(t, len(req.Items), 0) + require.Greater(t, req.Items[0].CpuTimeMs, uint32(0)) sqlMeta, exist := server.GetSQLMetaByDigestBlocking(req.SqlDigest, time.Second) require.True(t, exist) expectedNormalizedSQL, exist := sqlMap[string(req.SqlDigest)] @@ -292,12 +292,11 @@ func TestTopSQLPubSub(t *testing.T) { if _, ok := records[string(rec.SqlDigest)]; !ok { records[string(rec.SqlDigest)] = rec } else { - cpu := records[string(rec.SqlDigest)] + record := records[string(rec.SqlDigest)] if rec.PlanDigest != nil { - cpu.PlanDigest = rec.PlanDigest + record.PlanDigest = rec.PlanDigest } - cpu.RecordListTimestampSec = append(cpu.RecordListTimestampSec, rec.RecordListTimestampSec...) - cpu.RecordListCpuTimeMs = append(cpu.RecordListCpuTimeMs, rec.RecordListCpuTimeMs...) + record.Items = append(record.Items, rec.Items...) } } else if r.GetSqlMeta() != nil { sql := r.GetSqlMeta() @@ -315,8 +314,8 @@ func TestTopSQLPubSub(t *testing.T) { checkSQLPlanMap := map[string]struct{}{} for i := range records { record := records[i] - require.Greater(t, len(record.RecordListCpuTimeMs), 0) - require.Greater(t, record.RecordListCpuTimeMs[0], uint32(0)) + require.Greater(t, len(record.Items), 0) + require.Greater(t, record.Items[0].CpuTimeMs, uint32(0)) sqlMeta, exist := sqlMetas[string(record.SqlDigest)] require.True(t, exist) expectedNormalizedSQL, exist := digest2sql[string(record.SqlDigest)] From e0804997ca51f397e3be62ac32aedff35963dd68 Mon Sep 17 00:00:00 2001 From: mornyx Date: Mon, 27 Dec 2021 23:55:28 +0800 Subject: [PATCH 08/15] Get timestamp in reporter instead of pass; Upgrade tipb Signed-off-by: mornyx --- go.mod | 11 +-- go.sum | 16 ++-- util/topsql/reporter/datamodel.go | 2 +- util/topsql/reporter/mock/server.go | 6 -- util/topsql/reporter/pubsub.go | 4 +- util/topsql/reporter/reporter.go | 80 +++++++++++-------- util/topsql/reporter/reporter_test.go | 8 +- util/topsql/stmtstats/aggregator.go | 23 ++---- util/topsql/stmtstats/aggregator_test.go | 20 ++--- .../stmtstats/stmtstatstest/stmtstats_test.go | 14 ++-- util/topsql/topsql_test.go | 4 +- util/topsql/tracecpu/mock/mock.go | 6 +- util/topsql/tracecpu/profile.go | 10 +-- 13 files changed, 99 insertions(+), 105 deletions(-) diff --git a/go.mod b/go.mod index 866b5a834c64a..a7fa9de1959c3 100644 --- a/go.mod +++ b/go.mod @@ -53,7 +53,7 @@ require ( github.com/pingcap/sysutil v0.0.0-20211208032423-041a72e5860d github.com/pingcap/tidb-tools v5.2.2-0.20211019062242-37a8bef2fa17+incompatible github.com/pingcap/tidb/parser v0.0.0-20211011031125-9b13dc409c5e - github.com/pingcap/tipb v0.0.0-20211201080053-bd104bb270ba + github.com/pingcap/tipb v0.0.0-20211227115224-a06a85f9d2a5 github.com/prometheus/client_golang v1.5.1 github.com/prometheus/client_model v0.2.0 github.com/prometheus/common v0.9.1 @@ -79,12 +79,12 @@ require ( go.uber.org/goleak v1.1.12 go.uber.org/multierr v1.7.0 go.uber.org/zap v1.19.1 - golang.org/x/net v0.0.0-20210503060351-7fd8e65b6420 + golang.org/x/net v0.0.0-20211015210444-4f30a5c0130f golang.org/x/oauth2 v0.0.0-20210805134026-6f1e6394065a golang.org/x/sync v0.0.0-20210220032951-036812b2e83c - golang.org/x/sys v0.0.0-20210831042530-f4d43177bf5e + golang.org/x/sys v0.0.0-20211216021012-1d35b9e2eb4e golang.org/x/text v0.3.7 - golang.org/x/tools v0.1.5 + golang.org/x/tools v0.1.8 google.golang.org/api v0.54.0 google.golang.org/grpc v1.40.0 gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c // indirect @@ -102,6 +102,3 @@ replace github.com/pingcap/tidb/parser => ./parser // fix potential security issue(CVE-2020-26160) introduced by indirect dependency. replace github.com/dgrijalva/jwt-go => github.com/form3tech-oss/jwt-go v3.2.6-0.20210809144907-32ab6a8243d7+incompatible - -// TODO(mornyx): remove -replace github.com/pingcap/tipb => github.com/mornyx/tipb v0.0.0-20211227042052-974e18b70381 diff --git a/go.sum b/go.sum index 8e710468dcf51..548616f165466 100644 --- a/go.sum +++ b/go.sum @@ -511,8 +511,6 @@ github.com/modern-go/reflect2 v1.0.1 h1:9f412s+6RmYXLWZSEzVVgPGK7C2PphHj5RJrvfx9 github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= github.com/montanaflynn/stats v0.5.0 h1:2EkzeTSqBB4V4bJwWrt5gIIrZmpJBcoIRGS2kWLgzmk= github.com/montanaflynn/stats v0.5.0/go.mod h1:wL8QJuTMNUDYhXwkmfOly8iTdp5TEcJFWZD2D7SIkUc= -github.com/mornyx/tipb v0.0.0-20211227042052-974e18b70381 h1:OlTIzfq5zFkfl33s0LUTr9wiutG5ZQNBd0ixlbnKVVU= -github.com/mornyx/tipb v0.0.0-20211227042052-974e18b70381/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= github.com/moul/http2curl v1.0.0/go.mod h1:8UbvGypXm98wA/IqH45anm5Y2Z6ep6O31QGOAZ3H0fQ= github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/nats-io/nats.go v1.8.1/go.mod h1:BrFz9vVn0fU3AcH9Vn4Kd7W0NpJ651tD5omQ3M8LwxM= @@ -600,6 +598,8 @@ github.com/pingcap/tidb-dashboard v0.0.0-20211008050453-a25c25809529/go.mod h1:O github.com/pingcap/tidb-dashboard v0.0.0-20211107164327-80363dfbe884/go.mod h1:OCXbZTBTIMRcIt0jFsuCakZP+goYRv6IjawKbwLS2TQ= github.com/pingcap/tidb-tools v5.2.2-0.20211019062242-37a8bef2fa17+incompatible h1:c7+izmker91NkjkZ6FgTlmD4k1A5FLOAq+li6Ki2/GY= github.com/pingcap/tidb-tools v5.2.2-0.20211019062242-37a8bef2fa17+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM= +github.com/pingcap/tipb v0.0.0-20211227115224-a06a85f9d2a5 h1:oKL5kSHoQAI9o6mdbxU0cB7fvqm9KyXZs9s/p0kZRnY= +github.com/pingcap/tipb v0.0.0-20211227115224-a06a85f9d2a5/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= @@ -776,6 +776,7 @@ github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9de github.com/yuin/goldmark v1.1.32/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= +github.com/yuin/goldmark v1.4.1/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= go.etcd.io/bbolt v1.3.2/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= go.etcd.io/bbolt v1.3.3/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= go.etcd.io/bbolt v1.3.5 h1:XAzx9gjCb0Rxj7EoqcClPD1d5ZBxZJk0jbuoPHenBt0= @@ -883,6 +884,7 @@ golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.4.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.4.1/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.4.2/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= +golang.org/x/mod v0.5.1/go.mod h1:5OXOZSfqPIIbmVBIIKWRFfZjPR0E5r58TLhUjH0a2Ro= golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180906233101-161cd47e91fd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20181005035420-146acd28ed58/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= @@ -928,8 +930,9 @@ golang.org/x/net v0.0.0-20210119194325-5f4716e94777/go.mod h1:m0MpNAwzfU5UDzcl9v golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= golang.org/x/net v0.0.0-20210316092652-d523dce5a7f4/go.mod h1:RBQZq4jEuRlivfhVLdyRGr576XBO4/greRjx4P4O3yc= golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= -golang.org/x/net v0.0.0-20210503060351-7fd8e65b6420 h1:a8jGStKg0XqKDlKqjLrXn0ioF5MH36pT7Z0BRTqLhbk= golang.org/x/net v0.0.0-20210503060351-7fd8e65b6420/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= +golang.org/x/net v0.0.0-20211015210444-4f30a5c0130f h1:OfiFi4JbukWwe3lzw+xunroH1mnC1e2Gy5cxNJApiSY= +golang.org/x/net v0.0.0-20211015210444-4f30a5c0130f/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= @@ -1022,8 +1025,10 @@ golang.org/x/sys v0.0.0-20210603125802-9665404d3644/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20210616094352-59db8d763f22/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210630005230-0f9fa26af87c/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210806184541-e5e7981a1069/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20210831042530-f4d43177bf5e h1:XMgFehsDnnLGtjvjOfqWSUzt0alpTR1RSEuznObga2c= golang.org/x/sys v0.0.0-20210831042530-f4d43177bf5e/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20211019181941-9d821ace8654/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20211216021012-1d35b9e2eb4e h1:fLOSk5Q00efkSvAm+4xcoXD+RRmLmmulPn5I3Y9F2EM= +golang.org/x/sys v0.0.0-20211216021012-1d35b9e2eb4e/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= @@ -1111,8 +1116,9 @@ golang.org/x/tools v0.1.1/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= golang.org/x/tools v0.1.2/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= golang.org/x/tools v0.1.3/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= golang.org/x/tools v0.1.4/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= -golang.org/x/tools v0.1.5 h1:ouewzE6p+/VEB31YYnTbEJdi8pFqKp4P4n85vwo3DHA= golang.org/x/tools v0.1.5/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= +golang.org/x/tools v0.1.8 h1:P1HhGGuLW4aAclzjtmJdf0mJOjVUZUzOTqkAkWL+l6w= +golang.org/x/tools v0.1.8/go.mod h1:nABZi5QlRsZVlzPpHl034qft6wpY4eDcsTt5AaioBiU= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= diff --git a/util/topsql/reporter/datamodel.go b/util/topsql/reporter/datamodel.go index 07dcc71625f94..5df04b23144dc 100644 --- a/util/topsql/reporter/datamodel.go +++ b/util/topsql/reporter/datamodel.go @@ -83,7 +83,7 @@ func (i *tsItem) toProto() *tipb.TopSQLRecordItem { StmtExecCount: i.stmtStats.ExecCount, StmtKvExecCount: i.stmtStats.KvStatsItem.KvExecCount, // TODO: add duration - // StmtDurationSumNs: ti.stmtStats., + // StmtDurationSumNs: i.stmtStats.DurationSumNs, // Convert more indicators here. } } diff --git a/util/topsql/reporter/mock/server.go b/util/topsql/reporter/mock/server.go index 04fb200d7ab84..59de46443c6e4 100644 --- a/util/topsql/reporter/mock/server.go +++ b/util/topsql/reporter/mock/server.go @@ -26,8 +26,6 @@ import ( "github.com/pingcap/tipb/go-tipb" "go.uber.org/zap" "google.golang.org/grpc" - "google.golang.org/grpc/codes" - "google.golang.org/grpc/status" ) type mockAgentServer struct { @@ -87,10 +85,6 @@ func (svr *mockAgentServer) mayHang() { } } -func (svr *mockAgentServer) ReportCPUTimeRecords(stream tipb.TopSQLAgent_ReportCPUTimeRecordsServer) error { - return status.Error(codes.Unimplemented, "unimplemented") -} - func (svr *mockAgentServer) ReportTopSQLRecords(stream tipb.TopSQLAgent_ReportTopSQLRecordsServer) error { records := make([]*tipb.TopSQLRecord, 0, 10) for { diff --git a/util/topsql/reporter/pubsub.go b/util/topsql/reporter/pubsub.go index b39a2a7b8b4fb..62e6d2af8c412 100644 --- a/util/topsql/reporter/pubsub.go +++ b/util/topsql/reporter/pubsub.go @@ -171,11 +171,11 @@ func (ds *pubSubDataSink) sendTopSQLRecords(ctx context.Context, records []tipb. } }() - topSQLRecord := &tipb.TopSQLSubResponse_DataRecord{} + topSQLRecord := &tipb.TopSQLSubResponse_Record{} r := &tipb.TopSQLSubResponse{RespOneof: topSQLRecord} for i := range records { - topSQLRecord.DataRecord = &records[i] + topSQLRecord.Record = &records[i] if err = ds.stream.Send(r); err != nil { return } diff --git a/util/topsql/reporter/reporter.go b/util/topsql/reporter/reporter.go index 9391a232c47bf..d4c8684f4fd17 100644 --- a/util/topsql/reporter/reporter.go +++ b/util/topsql/reporter/reporter.go @@ -25,7 +25,12 @@ import ( "github.com/pingcap/tidb/util/topsql/tracecpu" ) -const reportTimeout = 40 * time.Second +const ( + reportTimeout = 40 * time.Second + collectChanBufferSize = 2 +) + +var nowFunc = time.Now // TopSQLReporter collects Top SQL metrics. type TopSQLReporter interface { @@ -58,13 +63,14 @@ type RemoteTopSQLReporter struct { ctx context.Context cancel context.CancelFunc - collectCPUDataChan chan cpuData - collectStmtRecordsChan chan []stmtstats.StatementStatsRecord + collectCPUTimeChan chan []tracecpu.SQLCPUTimeRecord + collectStmtStatsChan chan stmtstats.StatementStatsMap reportCollectedDataChan chan collectedData collecting *collecting normalizedSQLMap *normalizedSQLMap normalizedPlanMap *normalizedPlanMap + stmtStatsMaps map[uint64]stmtstats.StatementStatsMap // timestamp => stmtstats.StatementStatsMap // calling decodePlan this can take a while, so should not block critical paths. decodePlan planBinaryDecodeFunc @@ -79,12 +85,13 @@ func NewRemoteTopSQLReporter(decodePlan planBinaryDecodeFunc) *RemoteTopSQLRepor DefaultDataSinkRegisterer: NewDefaultDataSinkRegisterer(ctx), ctx: ctx, cancel: cancel, - collectCPUDataChan: make(chan cpuData, 1), - collectStmtRecordsChan: make(chan []stmtstats.StatementStatsRecord, 1), + collectCPUTimeChan: make(chan []tracecpu.SQLCPUTimeRecord, collectChanBufferSize), + collectStmtStatsChan: make(chan stmtstats.StatementStatsMap, collectChanBufferSize), reportCollectedDataChan: make(chan collectedData, 1), collecting: newCollecting(), normalizedSQLMap: newNormalizedSQLMap(), normalizedPlanMap: newNormalizedPlanMap(), + stmtStatsMaps: map[uint64]stmtstats.StatementStatsMap{}, decodePlan: decodePlan, } go tsr.collectWorker() @@ -96,28 +103,28 @@ func NewRemoteTopSQLReporter(decodePlan planBinaryDecodeFunc) *RemoteTopSQLRepor // // WARN: It will drop the DataRecords if the processing is not in time. // This function is thread-safe and efficient. -func (tsr *RemoteTopSQLReporter) Collect(timestamp uint64, records []tracecpu.SQLCPUTimeRecord) { - if len(records) == 0 { +func (tsr *RemoteTopSQLReporter) Collect(data []tracecpu.SQLCPUTimeRecord) { + if len(data) == 0 { return } select { - case tsr.collectCPUDataChan <- cpuData{timestamp: timestamp, records: records}: + case tsr.collectCPUTimeChan <- data: default: // ignore if chan blocked ignoreCollectChannelFullCounter.Inc() } } -// CollectStmtStatsRecords implements stmtstats.Collector. +// CollectStmtStatsMap implements stmtstats.Collector. // // WARN: It will drop the DataRecords if the processing is not in time. // This function is thread-safe and efficient. -func (tsr *RemoteTopSQLReporter) CollectStmtStatsRecords(rs []stmtstats.StatementStatsRecord) { - if len(rs) == 0 { +func (tsr *RemoteTopSQLReporter) CollectStmtStatsMap(data stmtstats.StatementStatsMap) { + if len(data) == 0 { return } select { - case tsr.collectStmtRecordsChan <- rs: + case tsr.collectStmtStatsChan <- data: default: // ignore if chan blocked ignoreCollectStmtChannelFullCounter.Inc() @@ -149,37 +156,36 @@ func (tsr *RemoteTopSQLReporter) collectWorker() { defer util.Recover("top-sql", "collectWorker", nil, false) currentReportInterval := variable.TopSQLVariable.ReportIntervalSeconds.Load() + collectTicker := time.NewTicker(time.Second) reportTicker := time.NewTicker(time.Second * time.Duration(currentReportInterval)) for { select { - case data := <-tsr.collectCPUDataChan: - tsr.doCollect(data.timestamp, data.records) - case rs := <-tsr.collectStmtRecordsChan: - tsr.doCollectStmtRecords(rs) + case <-tsr.ctx.Done(): + return + case <-collectTicker.C: + tsr.takeDataFromCollectChanBuffer() case <-reportTicker.C: + tsr.doCollectStmtStatsMaps() tsr.takeDataAndSendToReportChan() // Update `reportTicker` if report interval changed. if newInterval := variable.TopSQLVariable.ReportIntervalSeconds.Load(); newInterval != currentReportInterval { currentReportInterval = newInterval reportTicker.Reset(time.Second * time.Duration(currentReportInterval)) } - case <-tsr.ctx.Done(): - return } } } // doCollect collects top N cpuRecords of each round into tsr.collecting, and evict the // data that is not in top N. All the evicted cpuRecords will be summary into the others. -func (tsr *RemoteTopSQLReporter) doCollect(timestamp uint64, rs cpuRecords) { +func (tsr *RemoteTopSQLReporter) doCollect(timestamp uint64, data cpuRecords) { defer util.Recover("top-sql", "doCollect", nil, false) // Get top N cpuRecords of each round cpuRecords. Collect the top N to tsr.collecting // for each round. SQL meta will not be evicted, since the evicted SQL can be appeared // on other components (TiKV) TopN DataRecords. - var evicted cpuRecords - rs, evicted = rs.topN(int(variable.TopSQLVariable.MaxStatementCount.Load())) - for _, r := range rs { + top, evicted := data.topN(int(variable.TopSQLVariable.MaxStatementCount.Load())) + for _, r := range top { tsr.collecting.getOrCreateRecord(r.SQLDigest, r.PlanDigest).appendCPUTime(timestamp, r.CPUTimeMs) } if len(evicted) == 0 { @@ -196,14 +202,13 @@ func (tsr *RemoteTopSQLReporter) doCollect(timestamp uint64, rs cpuRecords) { tsr.collecting.appendOthersCPUTime(timestamp, totalEvictedCPUTime) } -// doCollectStmtRecords collects []stmtstats.StatementStatsRecord into tsr.collecting. +// doCollectStmtStatsMaps collects tsr.stmtStatsMaps into tsr.collecting. // All the evicted items will be summary into the others. -func (tsr *RemoteTopSQLReporter) doCollectStmtRecords(rs []stmtstats.StatementStatsRecord) { +func (tsr *RemoteTopSQLReporter) doCollectStmtStatsMaps() { defer util.Recover("top-sql", "doCollectStmtRecords", nil, false) - for _, r := range rs { - timestamp := uint64(r.Timestamp) - for digest, item := range r.Data { + for timestamp, data := range tsr.stmtStatsMaps { + for digest, item := range data { sqlDigest, planDigest := []byte(digest.SQLDigest), []byte(digest.PlanDigest) if tsr.collecting.hasEvicted(timestamp, sqlDigest, planDigest) { // This timestamp+sql+plan has been evicted due to low CPUTime. @@ -213,6 +218,21 @@ func (tsr *RemoteTopSQLReporter) doCollectStmtRecords(rs []stmtstats.StatementSt tsr.collecting.getOrCreateRecord(sqlDigest, planDigest).appendStmtStatsItem(timestamp, *item) } } + tsr.stmtStatsMaps = map[uint64]stmtstats.StatementStatsMap{} +} + +func (tsr *RemoteTopSQLReporter) takeDataFromCollectChanBuffer() { + timestamp := uint64(nowFunc().Unix()) + for { + select { + case data := <-tsr.collectCPUTimeChan: + tsr.doCollect(timestamp, data) + case data := <-tsr.collectStmtStatsChan: + tsr.stmtStatsMaps[timestamp] = data + default: + return + } + } } // takeDataAndSendToReportChan takes records data and then send to the report channel for reporting. @@ -281,9 +301,3 @@ type collectedData struct { normalizedSQLMap *normalizedSQLMap normalizedPlanMap *normalizedPlanMap } - -// cpuData is used for transmission in the channel. -type cpuData struct { - timestamp uint64 - records []tracecpu.SQLCPUTimeRecord -} diff --git a/util/topsql/reporter/reporter_test.go b/util/topsql/reporter/reporter_test.go index df612fc6c5ba0..684e18307237c 100644 --- a/util/topsql/reporter/reporter_test.go +++ b/util/topsql/reporter/reporter_test.go @@ -55,7 +55,7 @@ func populateCache(tsr *RemoteTopSQLReporter, begin, end int, timestamp uint64) CPUTimeMs: uint32(i + 1), }) } - tsr.Collect(timestamp, records) + tsr.doCollect(timestamp, records) // sleep a while for the asynchronous collect time.Sleep(100 * time.Millisecond) } @@ -203,7 +203,7 @@ func newSQLCPUTimeRecord(tsr *RemoteTopSQLReporter, sqlID int, cpuTimeMs uint32) } func collectAndWait(tsr *RemoteTopSQLReporter, timestamp uint64, records []tracecpu.SQLCPUTimeRecord) { - tsr.Collect(timestamp, records) + tsr.doCollect(timestamp, records) time.Sleep(time.Millisecond * 100) } @@ -401,7 +401,7 @@ func TestMultipleDataSinks(t *testing.T) { records := []tracecpu.SQLCPUTimeRecord{ newSQLCPUTimeRecord(tsr, 1, 2), } - tsr.Collect(3, records) + tsr.doCollect(3, records) for _, ch := range chs { d := <-ch @@ -432,7 +432,7 @@ func TestMultipleDataSinks(t *testing.T) { records = []tracecpu.SQLCPUTimeRecord{ newSQLCPUTimeRecord(tsr, 4, 5), } - tsr.Collect(6, records) + tsr.doCollect(6, records) for i := 1; i < 7; i += 2 { d := <-chs[i] diff --git a/util/topsql/stmtstats/aggregator.go b/util/topsql/stmtstats/aggregator.go index db19731b607f9..648b00dd16948 100644 --- a/util/topsql/stmtstats/aggregator.go +++ b/util/topsql/stmtstats/aggregator.go @@ -25,12 +25,6 @@ import ( // globalAggregator is global *aggregator. var globalAggregator = newAggregator() -// StatementStatsRecord is the merged StatementStatsMap with timestamp. -type StatementStatsRecord struct { - Timestamp int64 - Data StatementStatsMap -} - // aggregator is used to collect and aggregate data from all StatementStats. // It is responsible for collecting data from all StatementStats, aggregating // them together, uploading them and regularly cleaning up the closed StatementStats. @@ -69,21 +63,18 @@ func (m *aggregator) run() { // aggregate data from all associated StatementStats. // If StatementStats has been closed, collect will remove it from the map. func (m *aggregator) aggregate() { - r := StatementStatsRecord{ - Timestamp: time.Now().Unix(), - Data: StatementStatsMap{}, - } + total := StatementStatsMap{} m.statsSet.Range(func(statsR, _ interface{}) bool { stats := statsR.(*StatementStats) if stats.Finished() { m.unregister(stats) } - r.Data.Merge(stats.Take()) + total.Merge(stats.Take()) return true }) - if len(r.Data) > 0 { + if len(total) > 0 { m.collectors.Range(func(c, _ interface{}) bool { - c.(Collector).CollectStmtStatsRecords([]StatementStatsRecord{r}) + c.(Collector).CollectStmtStatsMap(total) return true }) } @@ -151,8 +142,8 @@ func UnregisterCollector(collector Collector) { globalAggregator.unregisterCollector(collector) } -// Collector is used to collect StatementStatsRecord. +// Collector is used to collect StatementStatsMap. type Collector interface { - // CollectStmtStatsRecords is used to collect list of StatementStatsRecord. - CollectStmtStatsRecords([]StatementStatsRecord) + // CollectStmtStatsMap is used to collect StatementStatsMap. + CollectStmtStatsMap(StatementStatsMap) } diff --git a/util/topsql/stmtstats/aggregator_test.go b/util/topsql/stmtstats/aggregator_test.go index 24a72bb89131d..61e7dbf07ca33 100644 --- a/util/topsql/stmtstats/aggregator_test.go +++ b/util/topsql/stmtstats/aggregator_test.go @@ -38,7 +38,7 @@ func Test_RegisterUnregisterCollector(t *testing.T) { SetupAggregator() defer CloseAggregator() time.Sleep(100 * time.Millisecond) - collector := newMockCollector(func(records []StatementStatsRecord) {}) + collector := newMockCollector(func(data StatementStatsMap) {}) RegisterCollector(collector) _, ok := globalAggregator.collectors.Load(collector) assert.True(t, ok) @@ -55,13 +55,13 @@ func Test_aggregator_register_collect(t *testing.T) { } a.register(stats) stats.OnExecutionBegin([]byte("SQL-1"), []byte("")) - var records []StatementStatsRecord - a.registerCollector(newMockCollector(func(rs []StatementStatsRecord) { - records = append(records, rs...) + total := StatementStatsMap{} + a.registerCollector(newMockCollector(func(data StatementStatsMap) { + total.Merge(data) })) a.aggregate() - assert.NotEmpty(t, records) - assert.Equal(t, uint64(1), records[0].Data[SQLPlanDigest{SQLDigest: "SQL-1"}].ExecCount) + assert.NotEmpty(t, total) + assert.Equal(t, uint64(1), total[SQLPlanDigest{SQLDigest: "SQL-1"}].ExecCount) } func Test_aggregator_run_close(t *testing.T) { @@ -81,13 +81,13 @@ func Test_aggregator_run_close(t *testing.T) { } type mockCollector struct { - f func(records []StatementStatsRecord) + f func(data StatementStatsMap) } -func newMockCollector(f func(records []StatementStatsRecord)) Collector { +func newMockCollector(f func(data StatementStatsMap)) Collector { return &mockCollector{f: f} } -func (c *mockCollector) CollectStmtStatsRecords(records []StatementStatsRecord) { - c.f(records) +func (c *mockCollector) CollectStmtStatsMap(data StatementStatsMap) { + c.f(data) } diff --git a/util/topsql/stmtstats/stmtstatstest/stmtstats_test.go b/util/topsql/stmtstats/stmtstatstest/stmtstats_test.go index d37de52178e0e..09f830a0ec9a3 100644 --- a/util/topsql/stmtstats/stmtstatstest/stmtstats_test.go +++ b/util/topsql/stmtstats/stmtstatstest/stmtstats_test.go @@ -39,12 +39,10 @@ func TestExecCount(t *testing.T) { // Register stmt stats collector. var mu sync.Mutex total := stmtstats.StatementStatsMap{} - stmtstats.RegisterCollector(newMockCollector(func(rs []stmtstats.StatementStatsRecord) { + stmtstats.RegisterCollector(newMockCollector(func(data stmtstats.StatementStatsMap) { mu.Lock() defer mu.Unlock() - for _, r := range rs { - total.Merge(r.Data) - } + total.Merge(data) })) // Create mock store. @@ -138,13 +136,13 @@ func TestExecCount(t *testing.T) { } type mockCollector struct { - f func(records []stmtstats.StatementStatsRecord) + f func(data stmtstats.StatementStatsMap) } -func newMockCollector(f func(records []stmtstats.StatementStatsRecord)) stmtstats.Collector { +func newMockCollector(f func(data stmtstats.StatementStatsMap)) stmtstats.Collector { return &mockCollector{f: f} } -func (c *mockCollector) CollectStmtStatsRecords(records []stmtstats.StatementStatsRecord) { - c.f(records) +func (c *mockCollector) CollectStmtStatsMap(data stmtstats.StatementStatsMap) { + c.f(data) } diff --git a/util/topsql/topsql_test.go b/util/topsql/topsql_test.go index 82e9f0d282ec8..a8d589f073df1 100644 --- a/util/topsql/topsql_test.go +++ b/util/topsql/topsql_test.go @@ -287,8 +287,8 @@ func TestTopSQLPubSub(t *testing.T) { break } - if r.GetDataRecord() != nil { - rec := r.GetDataRecord() + if r.GetRecord() != nil { + rec := r.GetRecord() if _, ok := records[string(rec.SqlDigest)]; !ok { records[string(rec.SqlDigest)] = rec } else { diff --git a/util/topsql/tracecpu/mock/mock.go b/util/topsql/tracecpu/mock/mock.go index ff229783674b8..3d240e588008d 100644 --- a/util/topsql/tracecpu/mock/mock.go +++ b/util/topsql/tracecpu/mock/mock.go @@ -50,7 +50,7 @@ func NewTopSQLCollector() *TopSQLCollector { } // Collect uses for testing. -func (c *TopSQLCollector) Collect(ts uint64, stats []tracecpu.SQLCPUTimeRecord) { +func (c *TopSQLCollector) Collect(stats []tracecpu.SQLCPUTimeRecord) { defer c.collectCnt.Inc() if len(stats) == 0 { return @@ -74,8 +74,8 @@ func (c *TopSQLCollector) Collect(ts uint64, stats []tracecpu.SQLCPUTimeRecord) } } -// CollectStmtStatsRecords implements stmtstats.Collector. -func (c *TopSQLCollector) CollectStmtStatsRecords(rs []stmtstats.StatementStatsRecord) {} +// CollectStmtStatsMap implements stmtstats.Collector. +func (c *TopSQLCollector) CollectStmtStatsMap(_ stmtstats.StatementStatsMap) {} // GetSQLStatsBySQLWithRetry uses for testing. func (c *TopSQLCollector) GetSQLStatsBySQLWithRetry(sql string, planIsNotNull bool) []*tracecpu.SQLCPUTimeRecord { diff --git a/util/topsql/tracecpu/profile.go b/util/topsql/tracecpu/profile.go index b08569d147eec..7b5063419146a 100644 --- a/util/topsql/tracecpu/profile.go +++ b/util/topsql/tracecpu/profile.go @@ -47,8 +47,7 @@ var GlobalSQLCPUProfiler = newSQLCPUProfiler() // Collector uses to collect SQL execution cpu time. type Collector interface { // Collect uses to collect the SQL execution cpu time. - // ts is a Unix time, unit is second. - Collect(ts uint64, stats []SQLCPUTimeRecord) + Collect(stats []SQLCPUTimeRecord) } // SQLCPUTimeRecord represents a single record of how much cpu time a sql plan consumes in one second. @@ -129,10 +128,6 @@ func (sp *sqlCPUProfiler) doCPUProfile() { ns := int64(time.Second)*intervalSecond - int64(time.Now().Nanosecond()) time.Sleep(time.Nanosecond * time.Duration(ns)) pprof.StopCPUProfile() - task.end = time.Now().Unix() - if task.end < 0 { - task.end = 0 - } sp.taskCh <- task } @@ -149,7 +144,7 @@ func (sp *sqlCPUProfiler) startAnalyzeProfileWorker() { stats := sp.parseCPUProfileBySQLLabels(p) sp.handleExportProfileTask(p) if c := sp.GetCollector(); c != nil { - c.Collect(uint64(task.end), stats) + c.Collect(stats) } sp.putTaskToBuffer(task) } @@ -157,7 +152,6 @@ func (sp *sqlCPUProfiler) startAnalyzeProfileWorker() { type profileData struct { buf *bytes.Buffer - end int64 } func (sp *sqlCPUProfiler) newProfileTask() *profileData { From 620f10cf7044b5b6577f46f1fc1c5d43f62f15e7 Mon Sep 17 00:00:00 2001 From: mornyx Date: Tue, 28 Dec 2021 15:45:26 +0800 Subject: [PATCH 09/15] Resolve comments Signed-off-by: mornyx --- util/topsql/reporter/datamodel.go | 12 ++++++++- util/topsql/reporter/datamodel_test.go | 8 +++++- util/topsql/reporter/datasink.go | 34 ------------------------- util/topsql/reporter/datasink_test.go | 7 +++--- util/topsql/reporter/reporter.go | 35 ++++++++++++++++++++++++-- 5 files changed, 55 insertions(+), 41 deletions(-) diff --git a/util/topsql/reporter/datamodel.go b/util/topsql/reporter/datamodel.go index 5df04b23144dc..64b36778de64b 100644 --- a/util/topsql/reporter/datamodel.go +++ b/util/topsql/reporter/datamodel.go @@ -105,6 +105,15 @@ func (ts tsItems) Swap(i, j int) { ts[i], ts[j] = ts[j], ts[i] } +func (ts tsItems) sorted() bool { + for n := 0; n < len(ts)-1; n++ { + if ts[n].timestamp > ts[n+1].timestamp { + return false + } + } + return true +} + // toProto converts the tsItems to the corresponding protobuf representation. func (ts tsItems) toProto() []*tipb.TopSQLRecordItem { capacity := len(ts) @@ -297,6 +306,7 @@ func (r *record) merge(other *record) { if other == nil || len(other.tsItems) == 0 { return } + sort.Sort(other) if len(r.tsItems) == 0 { r.totalCPUTimeMs = other.totalCPUTimeMs @@ -465,7 +475,7 @@ func (c *collecting) appendOthersCPUTime(timestamp uint64, totalCPUTimeMs uint32 others.appendCPUTime(timestamp, totalCPUTimeMs) } -// appendOthersCPUTime appends stmtstats.StatementStatsItem to a special record named "others". +// appendOthersStmtStatsItem appends stmtstats.StatementStatsItem to a special record named "others". func (c *collecting) appendOthersStmtStatsItem(timestamp uint64, item stmtstats.StatementStatsItem) { others, ok := c.records[keyOthers] if !ok { diff --git a/util/topsql/reporter/datamodel_test.go b/util/topsql/reporter/datamodel_test.go index da5c7ac769ad9..a20f7c4baf08c 100644 --- a/util/topsql/reporter/datamodel_test.go +++ b/util/topsql/reporter/datamodel_test.go @@ -43,12 +43,18 @@ func Test_tsItem_toProto(t *testing.T) { } func Test_tsItems_Sort(t *testing.T) { - items := tsItems{ + items := tsItems{} + assert.True(t, items.sorted()) + items = nil + assert.True(t, items.sorted()) + items = tsItems{ {timestamp: 2}, {timestamp: 3}, {timestamp: 1}, } + assert.False(t, items.sorted()) sort.Sort(items) + assert.True(t, items.sorted()) assert.Equal(t, uint64(1), items[0].timestamp) assert.Equal(t, uint64(2), items[1].timestamp) assert.Equal(t, uint64(3), items[2].timestamp) diff --git a/util/topsql/reporter/datasink.go b/util/topsql/reporter/datasink.go index 809798619f1c3..e31a0c0695bf9 100644 --- a/util/topsql/reporter/datasink.go +++ b/util/topsql/reporter/datasink.go @@ -21,9 +21,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/sessionctx/variable" - "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tipb/go-tipb" - "go.uber.org/zap" ) // DataSink collects and sends data to a target. @@ -56,7 +54,6 @@ func (d *ReportData) hasData() bool { return len(d.DataRecords) != 0 || len(d.SQLMetas) != 0 || len(d.PlanMetas) != 0 } -var _ DataSink = &DefaultDataSinkRegisterer{} var _ DataSinkRegisterer = &DefaultDataSinkRegisterer{} // DefaultDataSinkRegisterer implements DataSinkRegisterer. @@ -108,34 +105,3 @@ func (r *DefaultDataSinkRegisterer) Deregister(dataSink DataSink) { } } } - -// TrySend implements DataSink. -// -// TrySend sends ReportData to all internal registered DataSinks. -func (r *DefaultDataSinkRegisterer) TrySend(data *ReportData, deadline time.Time) error { - r.Lock() - dataSinks := make([]DataSink, 0, len(r.dataSinks)) - for ds := range r.dataSinks { - dataSinks = append(dataSinks, ds) - } - r.Unlock() - for _, ds := range dataSinks { - if err := ds.TrySend(data, deadline); err != nil { - logutil.BgLogger().Warn("[top-sql] failed to send data to datasink", zap.Error(err)) - } - } - return nil -} - -// OnReporterClosing implements DataSink. -// -// OnReporterClosing calls the OnReporterClosing method of all internally registered DataSinks. -func (r *DefaultDataSinkRegisterer) OnReporterClosing() { - var m map[DataSink]struct{} - r.Lock() - m, r.dataSinks = r.dataSinks, make(map[DataSink]struct{}) - r.Unlock() - for d := range m { - d.OnReporterClosing() - } -} diff --git a/util/topsql/reporter/datasink_test.go b/util/topsql/reporter/datasink_test.go index b57d36515eb3d..fdb022b0494c5 100644 --- a/util/topsql/reporter/datasink_test.go +++ b/util/topsql/reporter/datasink_test.go @@ -31,17 +31,18 @@ func TestDefaultDataSinkRegisterer(t *testing.T) { assert.NoError(t, err) err = r.Register(m2) assert.NoError(t, err) - err = r.TrySend(&ReportData{}, time.Now().Add(time.Hour)) + reporter := RemoteTopSQLReporter{DefaultDataSinkRegisterer: r} + err = reporter.trySend(&ReportData{}, time.Now().Add(time.Hour)) assert.NoError(t, err) assert.Len(t, m1.data, 1) assert.Len(t, m2.data, 1) - err = r.TrySend(&ReportData{}, time.Now().Add(time.Hour)) + err = reporter.trySend(&ReportData{}, time.Now().Add(time.Hour)) assert.NoError(t, err) assert.Len(t, m1.data, 2) assert.Len(t, m2.data, 2) assert.False(t, m1.closed) assert.False(t, m2.closed) - r.OnReporterClosing() + reporter.onReporterClosing() assert.True(t, m1.closed) assert.True(t, m2.closed) } diff --git a/util/topsql/reporter/reporter.go b/util/topsql/reporter/reporter.go index d4c8684f4fd17..62b8e2f8fc3fe 100644 --- a/util/topsql/reporter/reporter.go +++ b/util/topsql/reporter/reporter.go @@ -21,8 +21,10 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/util" + "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/topsql/stmtstats" "github.com/pingcap/tidb/util/topsql/tracecpu" + "go.uber.org/zap" ) const ( @@ -148,7 +150,7 @@ func (tsr *RemoteTopSQLReporter) RegisterPlan(planDigest []byte, normalizedPlan // Close implements TopSQLReporter. func (tsr *RemoteTopSQLReporter) Close() { tsr.cancel() - tsr.OnReporterClosing() + tsr.onReporterClosing() } // collectWorker consumes and collects data from tracecpu.Collector/stmtstats.Collector. @@ -157,7 +159,9 @@ func (tsr *RemoteTopSQLReporter) collectWorker() { currentReportInterval := variable.TopSQLVariable.ReportIntervalSeconds.Load() collectTicker := time.NewTicker(time.Second) + defer collectTicker.Stop() reportTicker := time.NewTicker(time.Second * time.Duration(currentReportInterval)) + defer reportTicker.Stop() for { select { case <-tsr.ctx.Done(): @@ -292,7 +296,34 @@ func (tsr *RemoteTopSQLReporter) doReport(data *ReportData) { } } }) - _ = tsr.TrySend(data, time.Now().Add(timeout)) + _ = tsr.trySend(data, time.Now().Add(timeout)) +} + +// trySend sends ReportData to all internal registered DataSinks. +func (tsr *RemoteTopSQLReporter) trySend(data *ReportData, deadline time.Time) error { + tsr.DefaultDataSinkRegisterer.Lock() + dataSinks := make([]DataSink, 0, len(tsr.dataSinks)) + for ds := range tsr.dataSinks { + dataSinks = append(dataSinks, ds) + } + tsr.DefaultDataSinkRegisterer.Unlock() + for _, ds := range dataSinks { + if err := ds.TrySend(data, deadline); err != nil { + logutil.BgLogger().Warn("[top-sql] failed to send data to datasink", zap.Error(err)) + } + } + return nil +} + +// onReporterClosing calls the OnReporterClosing method of all internally registered DataSinks. +func (tsr *RemoteTopSQLReporter) onReporterClosing() { + var m map[DataSink]struct{} + tsr.DefaultDataSinkRegisterer.Lock() + m, tsr.dataSinks = tsr.dataSinks, make(map[DataSink]struct{}) + tsr.DefaultDataSinkRegisterer.Unlock() + for d := range m { + d.OnReporterClosing() + } } // collectedData is used for transmission in the channel. From baef9d6f1be74898ea5f720314c5ffac713f7251 Mon Sep 17 00:00:00 2001 From: mornyx Date: Tue, 28 Dec 2021 15:48:40 +0800 Subject: [PATCH 10/15] Avoid sort Signed-off-by: mornyx --- util/topsql/reporter/datamodel.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/util/topsql/reporter/datamodel.go b/util/topsql/reporter/datamodel.go index 64b36778de64b..1e6a1391365dd 100644 --- a/util/topsql/reporter/datamodel.go +++ b/util/topsql/reporter/datamodel.go @@ -307,7 +307,9 @@ func (r *record) merge(other *record) { return } - sort.Sort(other) + if !other.tsItems.sorted() { + sort.Sort(other) // this may never happen + } if len(r.tsItems) == 0 { r.totalCPUTimeMs = other.totalCPUTimeMs r.tsItems = other.tsItems From 7fb1c80cf9682af52af9d5b1593566cae9555a2e Mon Sep 17 00:00:00 2001 From: mornyx Date: Wed, 29 Dec 2021 01:08:00 +0800 Subject: [PATCH 11/15] Resolve comments Signed-off-by: mornyx --- util/topsql/reporter/datamodel.go | 28 +++++++++++++------------- util/topsql/reporter/datamodel_test.go | 20 +++++++++--------- util/topsql/reporter/reporter.go | 24 +++++++++++----------- util/topsql/reporter/reporter_test.go | 10 ++++----- 4 files changed, 41 insertions(+), 41 deletions(-) diff --git a/util/topsql/reporter/datamodel.go b/util/topsql/reporter/datamodel.go index 1e6a1391365dd..d53db68fb0840 100644 --- a/util/topsql/reporter/datamodel.go +++ b/util/topsql/reporter/datamodel.go @@ -65,7 +65,7 @@ type tsItem struct { stmtStats stmtstats.StatementStatsItem } -func newTsItem() tsItem { +func zeroTsItem() tsItem { return tsItem{ stmtStats: stmtstats.StatementStatsItem{ KvStatsItem: stmtstats.KvStatementStatsItem{ @@ -177,11 +177,11 @@ func (r *record) Swap(i, j int) { r.tsItems.Swap(i, j) } -// appendCPUTime appends a cpuTime under a certain timestamp to record. +// setCPUTime appends a cpuTime under a certain timestamp to record. // If the timestamp already exists in tsItems, then cpuTime will be replaced. -func (r *record) appendCPUTime(timestamp uint64, cpuTimeMs uint32) { +func (r *record) setCPUTime(timestamp uint64, cpuTimeMs uint32) { if index, ok := r.tsIndex[timestamp]; ok { - // For the same timestamp, we have already called appendStmtStatsItem, + // For the same timestamp, we have already called setStmtStatsItem, // r.tsItems already exists the corresponding timestamp, and the corresponding // cpuTimeMs has been set to 0, so we directly replace it. // @@ -230,7 +230,7 @@ func (r *record) appendCPUTime(timestamp uint64, cpuTimeMs uint32) { // stmtStats.KvExecCount: [map{}] // stmtStats.DurationSum: [0] // - newItem := newTsItem() + newItem := zeroTsItem() newItem.timestamp = timestamp newItem.cpuTimeMs = cpuTimeMs r.tsIndex[timestamp] = len(r.tsItems) @@ -239,11 +239,11 @@ func (r *record) appendCPUTime(timestamp uint64, cpuTimeMs uint32) { r.totalCPUTimeMs += uint64(cpuTimeMs) } -// appendStmtStatsItem appends a stmtstats.StatementStatsItem under a certain timestamp to record. +// setStmtStatsItem appends a stmtstats.StatementStatsItem under a certain timestamp to record. // If the timestamp already exists in tsItems, then stmtstats.StatementStatsItem will be replaced. -func (r *record) appendStmtStatsItem(timestamp uint64, item stmtstats.StatementStatsItem) { +func (r *record) setStmtStatsItem(timestamp uint64, item stmtstats.StatementStatsItem) { if index, ok := r.tsIndex[timestamp]; ok { - // For the same timestamp, we have already called appendCPUTime, + // For the same timestamp, we have already called setCPUTime, // r.tsItems already exists the corresponding timestamp, and the // corresponding stmtStats has been set to 0, so we directly replace it. // @@ -292,7 +292,7 @@ func (r *record) appendStmtStatsItem(timestamp uint64, item stmtstats.StatementS // stmtStats.KvExecCount: [map{"1.1.1.1:1": 123}] // stmtStats.DurationSum: [456] // - newItem := newTsItem() + newItem := zeroTsItem() newItem.timestamp = timestamp newItem.stmtStats = item r.tsIndex[timestamp] = len(r.tsItems) @@ -321,7 +321,7 @@ func (r *record) merge(other *record) { i, j := 0, 0 for i < len(r.tsItems) && j < len(other.tsItems) { if r.tsItems[i].timestamp == other.tsItems[j].timestamp { - newItem := newTsItem() + newItem := zeroTsItem() newItem.timestamp = r.tsItems[i].timestamp newItem.cpuTimeMs = r.tsItems[i].cpuTimeMs + other.tsItems[j].cpuTimeMs r.tsItems[i].stmtStats.Merge(&other.tsItems[j].stmtStats) @@ -330,14 +330,14 @@ func (r *record) merge(other *record) { i++ j++ } else if r.tsItems[i].timestamp < other.tsItems[j].timestamp { - newItem := newTsItem() + newItem := zeroTsItem() newItem.timestamp = r.tsItems[i].timestamp newItem.cpuTimeMs = r.tsItems[i].cpuTimeMs newItem.stmtStats = r.tsItems[i].stmtStats newTsItems = append(newTsItems, newItem) i++ } else { - newItem := newTsItem() + newItem := zeroTsItem() newItem.timestamp = other.tsItems[j].timestamp newItem.cpuTimeMs = other.tsItems[j].cpuTimeMs newItem.stmtStats = other.tsItems[j].stmtStats @@ -474,7 +474,7 @@ func (c *collecting) appendOthersCPUTime(timestamp uint64, totalCPUTimeMs uint32 others = newRecord(nil, nil) c.records[keyOthers] = others } - others.appendCPUTime(timestamp, totalCPUTimeMs) + others.setCPUTime(timestamp, totalCPUTimeMs) } // appendOthersStmtStatsItem appends stmtstats.StatementStatsItem to a special record named "others". @@ -484,7 +484,7 @@ func (c *collecting) appendOthersStmtStatsItem(timestamp uint64, item stmtstats. others = newRecord(nil, nil) c.records[keyOthers] = others } - others.appendStmtStatsItem(timestamp, item) + others.setStmtStatsItem(timestamp, item) } // topN returns the largest N records, other records will be packed and appended to the end. diff --git a/util/topsql/reporter/datamodel_test.go b/util/topsql/reporter/datamodel_test.go index a20f7c4baf08c..9261031fa8228 100644 --- a/util/topsql/reporter/datamodel_test.go +++ b/util/topsql/reporter/datamodel_test.go @@ -94,32 +94,32 @@ func Test_record_append(t *testing.T) { // CPUTimeList: [] // ExecCountList: [] - r.appendCPUTime(1, 1) + r.setCPUTime(1, 1) // TimestampList: [1] // CPUTimeList: [1] // ExecCountList: [0] - r.appendStmtStatsItem(1, stmtstats.StatementStatsItem{ExecCount: 1}) + r.setStmtStatsItem(1, stmtstats.StatementStatsItem{ExecCount: 1}) // TimestampList: [1] // CPUTimeList: [1] // ExecCountList: [1] - r.appendCPUTime(2, 1) + r.setCPUTime(2, 1) // TimestampList: [1, 2] // CPUTimeList: [1, 1] // ExecCountList: [1, 0] - r.appendCPUTime(3, 1) + r.setCPUTime(3, 1) // TimestampList: [1, 2, 3] // CPUTimeList: [1, 1, 1] // ExecCountList: [1, 0, 0] - r.appendStmtStatsItem(3, stmtstats.StatementStatsItem{ExecCount: 1}) + r.setStmtStatsItem(3, stmtstats.StatementStatsItem{ExecCount: 1}) // TimestampList: [1, 2, 3] // CPUTimeList: [1, 1, 1] // ExecCountList: [1, 0, 1] - r.appendStmtStatsItem(2, stmtstats.StatementStatsItem{ExecCount: 1}) + r.setStmtStatsItem(2, stmtstats.StatementStatsItem{ExecCount: 1}) // TimestampList: [1, 2, 3] // CPUTimeList: [1, 1, 1] // ExecCountList: [1, 1, 1] @@ -268,9 +268,9 @@ func Test_collecting_appendOthers(t *testing.T) { func Test_collecting_topN(t *testing.T) { c := newCollecting() - c.getOrCreateRecord([]byte("SQL-1"), []byte("PLAN-1")).appendCPUTime(1, 1) - c.getOrCreateRecord([]byte("SQL-2"), []byte("PLAN-2")).appendCPUTime(1, 2) - c.getOrCreateRecord([]byte("SQL-3"), []byte("PLAN-3")).appendCPUTime(1, 3) + c.getOrCreateRecord([]byte("SQL-1"), []byte("PLAN-1")).setCPUTime(1, 1) + c.getOrCreateRecord([]byte("SQL-2"), []byte("PLAN-2")).setCPUTime(1, 2) + c.getOrCreateRecord([]byte("SQL-3"), []byte("PLAN-3")).setCPUTime(1, 3) rs := c.topN(1) assert.Len(t, rs, 2) assert.Equal(t, []byte("SQL-3"), rs[0].sqlDigest) @@ -283,7 +283,7 @@ func Test_collecting_topN(t *testing.T) { func Test_collecting_take(t *testing.T) { c1 := newCollecting() - c1.getOrCreateRecord([]byte("SQL-1"), []byte("PLAN-1")).appendCPUTime(1, 1) + c1.getOrCreateRecord([]byte("SQL-1"), []byte("PLAN-1")).setCPUTime(1, 1) c2 := c1.take() assert.Empty(t, c1.records) assert.Len(t, c2.records, 1) diff --git a/util/topsql/reporter/reporter.go b/util/topsql/reporter/reporter.go index 62b8e2f8fc3fe..7e423a16fa067 100644 --- a/util/topsql/reporter/reporter.go +++ b/util/topsql/reporter/reporter.go @@ -72,7 +72,7 @@ type RemoteTopSQLReporter struct { collecting *collecting normalizedSQLMap *normalizedSQLMap normalizedPlanMap *normalizedPlanMap - stmtStatsMaps map[uint64]stmtstats.StatementStatsMap // timestamp => stmtstats.StatementStatsMap + stmtStatsBuffer map[uint64]stmtstats.StatementStatsMap // timestamp => stmtstats.StatementStatsMap // calling decodePlan this can take a while, so should not block critical paths. decodePlan planBinaryDecodeFunc @@ -93,7 +93,7 @@ func NewRemoteTopSQLReporter(decodePlan planBinaryDecodeFunc) *RemoteTopSQLRepor collecting: newCollecting(), normalizedSQLMap: newNormalizedSQLMap(), normalizedPlanMap: newNormalizedPlanMap(), - stmtStatsMaps: map[uint64]stmtstats.StatementStatsMap{}, + stmtStatsBuffer: map[uint64]stmtstats.StatementStatsMap{}, decodePlan: decodePlan, } go tsr.collectWorker() @@ -180,17 +180,17 @@ func (tsr *RemoteTopSQLReporter) collectWorker() { } } -// doCollect collects top N cpuRecords of each round into tsr.collecting, and evict the +// processCPUTimeData collects top N cpuRecords of each round into tsr.collecting, and evict the // data that is not in top N. All the evicted cpuRecords will be summary into the others. -func (tsr *RemoteTopSQLReporter) doCollect(timestamp uint64, data cpuRecords) { - defer util.Recover("top-sql", "doCollect", nil, false) +func (tsr *RemoteTopSQLReporter) processCPUTimeData(timestamp uint64, data cpuRecords) { + defer util.Recover("top-sql", "processCPUTimeData", nil, false) // Get top N cpuRecords of each round cpuRecords. Collect the top N to tsr.collecting // for each round. SQL meta will not be evicted, since the evicted SQL can be appeared // on other components (TiKV) TopN DataRecords. top, evicted := data.topN(int(variable.TopSQLVariable.MaxStatementCount.Load())) for _, r := range top { - tsr.collecting.getOrCreateRecord(r.SQLDigest, r.PlanDigest).appendCPUTime(timestamp, r.CPUTimeMs) + tsr.collecting.getOrCreateRecord(r.SQLDigest, r.PlanDigest).setCPUTime(timestamp, r.CPUTimeMs) } if len(evicted) == 0 { return @@ -206,12 +206,12 @@ func (tsr *RemoteTopSQLReporter) doCollect(timestamp uint64, data cpuRecords) { tsr.collecting.appendOthersCPUTime(timestamp, totalEvictedCPUTime) } -// doCollectStmtStatsMaps collects tsr.stmtStatsMaps into tsr.collecting. +// doCollectStmtStatsMaps collects tsr.stmtStatsBuffer into tsr.collecting. // All the evicted items will be summary into the others. func (tsr *RemoteTopSQLReporter) doCollectStmtStatsMaps() { defer util.Recover("top-sql", "doCollectStmtRecords", nil, false) - for timestamp, data := range tsr.stmtStatsMaps { + for timestamp, data := range tsr.stmtStatsBuffer { for digest, item := range data { sqlDigest, planDigest := []byte(digest.SQLDigest), []byte(digest.PlanDigest) if tsr.collecting.hasEvicted(timestamp, sqlDigest, planDigest) { @@ -219,10 +219,10 @@ func (tsr *RemoteTopSQLReporter) doCollectStmtStatsMaps() { tsr.collecting.appendOthersStmtStatsItem(timestamp, *item) continue } - tsr.collecting.getOrCreateRecord(sqlDigest, planDigest).appendStmtStatsItem(timestamp, *item) + tsr.collecting.getOrCreateRecord(sqlDigest, planDigest).setStmtStatsItem(timestamp, *item) } } - tsr.stmtStatsMaps = map[uint64]stmtstats.StatementStatsMap{} + tsr.stmtStatsBuffer = map[uint64]stmtstats.StatementStatsMap{} } func (tsr *RemoteTopSQLReporter) takeDataFromCollectChanBuffer() { @@ -230,9 +230,9 @@ func (tsr *RemoteTopSQLReporter) takeDataFromCollectChanBuffer() { for { select { case data := <-tsr.collectCPUTimeChan: - tsr.doCollect(timestamp, data) + tsr.processCPUTimeData(timestamp, data) case data := <-tsr.collectStmtStatsChan: - tsr.stmtStatsMaps[timestamp] = data + tsr.stmtStatsBuffer[timestamp] = data default: return } diff --git a/util/topsql/reporter/reporter_test.go b/util/topsql/reporter/reporter_test.go index 684e18307237c..3b80d51041676 100644 --- a/util/topsql/reporter/reporter_test.go +++ b/util/topsql/reporter/reporter_test.go @@ -55,7 +55,7 @@ func populateCache(tsr *RemoteTopSQLReporter, begin, end int, timestamp uint64) CPUTimeMs: uint32(i + 1), }) } - tsr.doCollect(timestamp, records) + tsr.processCPUTimeData(timestamp, records) // sleep a while for the asynchronous collect time.Sleep(100 * time.Millisecond) } @@ -203,7 +203,7 @@ func newSQLCPUTimeRecord(tsr *RemoteTopSQLReporter, sqlID int, cpuTimeMs uint32) } func collectAndWait(tsr *RemoteTopSQLReporter, timestamp uint64, records []tracecpu.SQLCPUTimeRecord) { - tsr.doCollect(timestamp, records) + tsr.processCPUTimeData(timestamp, records) time.Sleep(time.Millisecond * 100) } @@ -336,7 +336,7 @@ func TestCollectCapacity(t *testing.T) { require.Equal(t, int64(20000), tsr.normalizedPlanMap.length.Load()) variable.TopSQLVariable.MaxStatementCount.Store(5000) - tsr.doCollect(1, genRecord(20000)) + tsr.processCPUTimeData(1, genRecord(20000)) require.Equal(t, 5001, len(tsr.collecting.records)) require.Equal(t, int64(20000), tsr.normalizedSQLMap.length.Load()) require.Equal(t, int64(20000), tsr.normalizedPlanMap.length.Load()) @@ -401,7 +401,7 @@ func TestMultipleDataSinks(t *testing.T) { records := []tracecpu.SQLCPUTimeRecord{ newSQLCPUTimeRecord(tsr, 1, 2), } - tsr.doCollect(3, records) + tsr.processCPUTimeData(3, records) for _, ch := range chs { d := <-ch @@ -432,7 +432,7 @@ func TestMultipleDataSinks(t *testing.T) { records = []tracecpu.SQLCPUTimeRecord{ newSQLCPUTimeRecord(tsr, 4, 5), } - tsr.doCollect(6, records) + tsr.processCPUTimeData(6, records) for i := 1; i < 7; i += 2 { d := <-chs[i] From 93824c165ce8caaa0bf4a6f7d780d26fd3f99feb Mon Sep 17 00:00:00 2001 From: mornyx Date: Wed, 29 Dec 2021 01:43:39 +0800 Subject: [PATCH 12/15] Remove useless test Signed-off-by: mornyx --- util/topsql/reporter/datasink_test.go | 18 ++++-------------- 1 file changed, 4 insertions(+), 14 deletions(-) diff --git a/util/topsql/reporter/datasink_test.go b/util/topsql/reporter/datasink_test.go index fdb022b0494c5..d6a781e0148bb 100644 --- a/util/topsql/reporter/datasink_test.go +++ b/util/topsql/reporter/datasink_test.go @@ -31,20 +31,10 @@ func TestDefaultDataSinkRegisterer(t *testing.T) { assert.NoError(t, err) err = r.Register(m2) assert.NoError(t, err) - reporter := RemoteTopSQLReporter{DefaultDataSinkRegisterer: r} - err = reporter.trySend(&ReportData{}, time.Now().Add(time.Hour)) - assert.NoError(t, err) - assert.Len(t, m1.data, 1) - assert.Len(t, m2.data, 1) - err = reporter.trySend(&ReportData{}, time.Now().Add(time.Hour)) - assert.NoError(t, err) - assert.Len(t, m1.data, 2) - assert.Len(t, m2.data, 2) - assert.False(t, m1.closed) - assert.False(t, m2.closed) - reporter.onReporterClosing() - assert.True(t, m1.closed) - assert.True(t, m2.closed) + assert.Len(t, r.dataSinks, 2) + r.Deregister(m1) + r.Deregister(m2) + assert.Empty(t, r.dataSinks) } type mockDataSink2 struct { From 36a0b56a5a5eb39fd30e86001ab7bd95f8c0028f Mon Sep 17 00:00:00 2001 From: mornyx Date: Wed, 29 Dec 2021 10:46:32 +0800 Subject: [PATCH 13/15] Use add or merge instead of replace Signed-off-by: mornyx --- util/topsql/reporter/datamodel.go | 29 +++++++++++++------------- util/topsql/reporter/datamodel_test.go | 20 +++++++++--------- util/topsql/reporter/reporter.go | 4 ++-- 3 files changed, 27 insertions(+), 26 deletions(-) diff --git a/util/topsql/reporter/datamodel.go b/util/topsql/reporter/datamodel.go index d53db68fb0840..4ad6af6de8237 100644 --- a/util/topsql/reporter/datamodel.go +++ b/util/topsql/reporter/datamodel.go @@ -177,13 +177,13 @@ func (r *record) Swap(i, j int) { r.tsItems.Swap(i, j) } -// setCPUTime appends a cpuTime under a certain timestamp to record. -// If the timestamp already exists in tsItems, then cpuTime will be replaced. -func (r *record) setCPUTime(timestamp uint64, cpuTimeMs uint32) { +// appendCPUTime appends a cpuTime under a certain timestamp to record. +// If the timestamp already exists in tsItems, then cpuTime will be added. +func (r *record) appendCPUTime(timestamp uint64, cpuTimeMs uint32) { if index, ok := r.tsIndex[timestamp]; ok { - // For the same timestamp, we have already called setStmtStatsItem, + // For the same timestamp, we have already called appendStmtStatsItem, // r.tsItems already exists the corresponding timestamp, and the corresponding - // cpuTimeMs has been set to 0, so we directly replace it. + // cpuTimeMs has been set to 0 (or other values, although impossible), so we add it. // // let timestamp = 10000, cpuTimeMs = 123 // @@ -205,7 +205,7 @@ func (r *record) setCPUTime(timestamp uint64, cpuTimeMs uint32) { // stmtStats.KvExecCount: [map{"?": ?}] // stmtStats.DurationSum: [?] // - r.tsItems[index].cpuTimeMs = cpuTimeMs + r.tsItems[index].cpuTimeMs += cpuTimeMs } else { // For this timestamp, we have not appended any tsItem, so append it directly. // Other fields in tsItem except cpuTimeMs will be initialized to 0. @@ -239,13 +239,14 @@ func (r *record) setCPUTime(timestamp uint64, cpuTimeMs uint32) { r.totalCPUTimeMs += uint64(cpuTimeMs) } -// setStmtStatsItem appends a stmtstats.StatementStatsItem under a certain timestamp to record. -// If the timestamp already exists in tsItems, then stmtstats.StatementStatsItem will be replaced. -func (r *record) setStmtStatsItem(timestamp uint64, item stmtstats.StatementStatsItem) { +// appendStmtStatsItem appends a stmtstats.StatementStatsItem under a certain timestamp to record. +// If the timestamp already exists in tsItems, then stmtstats.StatementStatsItem will be merged. +func (r *record) appendStmtStatsItem(timestamp uint64, item stmtstats.StatementStatsItem) { if index, ok := r.tsIndex[timestamp]; ok { - // For the same timestamp, we have already called setCPUTime, + // For the same timestamp, we have already called appendCPUTime, // r.tsItems already exists the corresponding timestamp, and the - // corresponding stmtStats has been set to 0, so we directly replace it. + // corresponding stmtStats has been set to 0 (or other values, + // although impossible), so we merge it. // // let timestamp = 10000, execCount = 123, kvExecCount = map{"1.1.1.1:1": 123}, durationSum = 456 // @@ -267,7 +268,7 @@ func (r *record) setStmtStatsItem(timestamp uint64, item stmtstats.StatementStat // stmtStats.KvExecCount: [map{"1.1.1.1:1": 123}] // stmtStats.DurationSum: [456] // - r.tsItems[index].stmtStats = item + r.tsItems[index].stmtStats.Merge(&item) } else { // For this timestamp, we have not appended any tsItem, so append it directly. // Other fields in tsItem except stmtStats will be initialized to 0. @@ -474,7 +475,7 @@ func (c *collecting) appendOthersCPUTime(timestamp uint64, totalCPUTimeMs uint32 others = newRecord(nil, nil) c.records[keyOthers] = others } - others.setCPUTime(timestamp, totalCPUTimeMs) + others.appendCPUTime(timestamp, totalCPUTimeMs) } // appendOthersStmtStatsItem appends stmtstats.StatementStatsItem to a special record named "others". @@ -484,7 +485,7 @@ func (c *collecting) appendOthersStmtStatsItem(timestamp uint64, item stmtstats. others = newRecord(nil, nil) c.records[keyOthers] = others } - others.setStmtStatsItem(timestamp, item) + others.appendStmtStatsItem(timestamp, item) } // topN returns the largest N records, other records will be packed and appended to the end. diff --git a/util/topsql/reporter/datamodel_test.go b/util/topsql/reporter/datamodel_test.go index 9261031fa8228..a20f7c4baf08c 100644 --- a/util/topsql/reporter/datamodel_test.go +++ b/util/topsql/reporter/datamodel_test.go @@ -94,32 +94,32 @@ func Test_record_append(t *testing.T) { // CPUTimeList: [] // ExecCountList: [] - r.setCPUTime(1, 1) + r.appendCPUTime(1, 1) // TimestampList: [1] // CPUTimeList: [1] // ExecCountList: [0] - r.setStmtStatsItem(1, stmtstats.StatementStatsItem{ExecCount: 1}) + r.appendStmtStatsItem(1, stmtstats.StatementStatsItem{ExecCount: 1}) // TimestampList: [1] // CPUTimeList: [1] // ExecCountList: [1] - r.setCPUTime(2, 1) + r.appendCPUTime(2, 1) // TimestampList: [1, 2] // CPUTimeList: [1, 1] // ExecCountList: [1, 0] - r.setCPUTime(3, 1) + r.appendCPUTime(3, 1) // TimestampList: [1, 2, 3] // CPUTimeList: [1, 1, 1] // ExecCountList: [1, 0, 0] - r.setStmtStatsItem(3, stmtstats.StatementStatsItem{ExecCount: 1}) + r.appendStmtStatsItem(3, stmtstats.StatementStatsItem{ExecCount: 1}) // TimestampList: [1, 2, 3] // CPUTimeList: [1, 1, 1] // ExecCountList: [1, 0, 1] - r.setStmtStatsItem(2, stmtstats.StatementStatsItem{ExecCount: 1}) + r.appendStmtStatsItem(2, stmtstats.StatementStatsItem{ExecCount: 1}) // TimestampList: [1, 2, 3] // CPUTimeList: [1, 1, 1] // ExecCountList: [1, 1, 1] @@ -268,9 +268,9 @@ func Test_collecting_appendOthers(t *testing.T) { func Test_collecting_topN(t *testing.T) { c := newCollecting() - c.getOrCreateRecord([]byte("SQL-1"), []byte("PLAN-1")).setCPUTime(1, 1) - c.getOrCreateRecord([]byte("SQL-2"), []byte("PLAN-2")).setCPUTime(1, 2) - c.getOrCreateRecord([]byte("SQL-3"), []byte("PLAN-3")).setCPUTime(1, 3) + c.getOrCreateRecord([]byte("SQL-1"), []byte("PLAN-1")).appendCPUTime(1, 1) + c.getOrCreateRecord([]byte("SQL-2"), []byte("PLAN-2")).appendCPUTime(1, 2) + c.getOrCreateRecord([]byte("SQL-3"), []byte("PLAN-3")).appendCPUTime(1, 3) rs := c.topN(1) assert.Len(t, rs, 2) assert.Equal(t, []byte("SQL-3"), rs[0].sqlDigest) @@ -283,7 +283,7 @@ func Test_collecting_topN(t *testing.T) { func Test_collecting_take(t *testing.T) { c1 := newCollecting() - c1.getOrCreateRecord([]byte("SQL-1"), []byte("PLAN-1")).setCPUTime(1, 1) + c1.getOrCreateRecord([]byte("SQL-1"), []byte("PLAN-1")).appendCPUTime(1, 1) c2 := c1.take() assert.Empty(t, c1.records) assert.Len(t, c2.records, 1) diff --git a/util/topsql/reporter/reporter.go b/util/topsql/reporter/reporter.go index 7e423a16fa067..0fea2a79c253c 100644 --- a/util/topsql/reporter/reporter.go +++ b/util/topsql/reporter/reporter.go @@ -190,7 +190,7 @@ func (tsr *RemoteTopSQLReporter) processCPUTimeData(timestamp uint64, data cpuRe // on other components (TiKV) TopN DataRecords. top, evicted := data.topN(int(variable.TopSQLVariable.MaxStatementCount.Load())) for _, r := range top { - tsr.collecting.getOrCreateRecord(r.SQLDigest, r.PlanDigest).setCPUTime(timestamp, r.CPUTimeMs) + tsr.collecting.getOrCreateRecord(r.SQLDigest, r.PlanDigest).appendCPUTime(timestamp, r.CPUTimeMs) } if len(evicted) == 0 { return @@ -219,7 +219,7 @@ func (tsr *RemoteTopSQLReporter) doCollectStmtStatsMaps() { tsr.collecting.appendOthersStmtStatsItem(timestamp, *item) continue } - tsr.collecting.getOrCreateRecord(sqlDigest, planDigest).setStmtStatsItem(timestamp, *item) + tsr.collecting.getOrCreateRecord(sqlDigest, planDigest).appendStmtStatsItem(timestamp, *item) } } tsr.stmtStatsBuffer = map[uint64]stmtstats.StatementStatsMap{} From 32d746cf57f72f3204946f49e1bfe66402c6c806 Mon Sep 17 00:00:00 2001 From: mornyx Date: Wed, 29 Dec 2021 11:33:48 +0800 Subject: [PATCH 14/15] Resolve Signed-off-by: mornyx --- util/topsql/reporter/datamodel.go | 17 +++++++++++++---- util/topsql/reporter/datamodel_test.go | 2 +- util/topsql/reporter/datasink.go | 2 +- util/topsql/reporter/reporter.go | 10 +++++----- 4 files changed, 20 insertions(+), 11 deletions(-) diff --git a/util/topsql/reporter/datamodel.go b/util/topsql/reporter/datamodel.go index 4ad6af6de8237..38d8f627d27ca 100644 --- a/util/topsql/reporter/datamodel.go +++ b/util/topsql/reporter/datamodel.go @@ -55,8 +55,14 @@ import ( // // normalizePlanMap: { planDigest => normalizedPlan | planDigest => normalizedPlan | ... } -// keyOthers is the key to store the aggregation of all records that is out of Top N. -const keyOthers = "" +const ( + // keyOthers is the key to store the aggregation of all records that is out of Top N. + keyOthers = "" + + // maxTsItemsCapacity is a protection to avoid excessive memory usage caused by + // incorrect configuration. The corresponding value defaults to 60 (60 s/min). + maxTsItemsCapacity = 1000 +) // tsItem is a self-contained complete piece of data for a certain timestamp. type tsItem struct { @@ -144,6 +150,9 @@ type record struct { func newRecord(sqlDigest, planDigest []byte) *record { listCap := variable.TopSQLVariable.ReportIntervalSeconds.Load()/variable.TopSQLVariable.PrecisionSeconds.Load() + 1 + if listCap > maxTsItemsCapacity { + listCap = maxTsItemsCapacity + } return &record{ sqlDigest: sqlDigest, planDigest: planDigest, @@ -488,8 +497,8 @@ func (c *collecting) appendOthersStmtStatsItem(timestamp uint64, item stmtstats. others.appendStmtStatsItem(timestamp, item) } -// topN returns the largest N records, other records will be packed and appended to the end. -func (c *collecting) topN(n int) records { +// compactToTopNAndOthers returns the largest N records, other records will be packed and appended to the end. +func (c *collecting) compactToTopNAndOthers(n int) records { others := c.records[keyOthers] delete(c.records, keyOthers) rs := make(records, 0, len(c.records)) diff --git a/util/topsql/reporter/datamodel_test.go b/util/topsql/reporter/datamodel_test.go index a20f7c4baf08c..35f1b92815c7e 100644 --- a/util/topsql/reporter/datamodel_test.go +++ b/util/topsql/reporter/datamodel_test.go @@ -271,7 +271,7 @@ func Test_collecting_topN(t *testing.T) { c.getOrCreateRecord([]byte("SQL-1"), []byte("PLAN-1")).appendCPUTime(1, 1) c.getOrCreateRecord([]byte("SQL-2"), []byte("PLAN-2")).appendCPUTime(1, 2) c.getOrCreateRecord([]byte("SQL-3"), []byte("PLAN-3")).appendCPUTime(1, 3) - rs := c.topN(1) + rs := c.compactToTopNAndOthers(1) assert.Len(t, rs, 2) assert.Equal(t, []byte("SQL-3"), rs[0].sqlDigest) assert.Equal(t, []byte("PLAN-3"), rs[0].planDigest) diff --git a/util/topsql/reporter/datasink.go b/util/topsql/reporter/datasink.go index e31a0c0695bf9..abc24a7a02f7c 100644 --- a/util/topsql/reporter/datasink.go +++ b/util/topsql/reporter/datasink.go @@ -43,7 +43,7 @@ type DataSinkRegisterer interface { // ReportData contains data that reporter sends to the agent. type ReportData struct { - // DataRecords contains the topN records []tipb.TopSQLRecord and the `others` + // DataRecords contains the compactToTopNAndOthers records []tipb.TopSQLRecord and the `others` // record which aggregation all []tipb.TopSQLRecord that is out of Top N. DataRecords []tipb.TopSQLRecord SQLMetas []tipb.SQLMeta diff --git a/util/topsql/reporter/reporter.go b/util/topsql/reporter/reporter.go index 0fea2a79c253c..2e6c442815473 100644 --- a/util/topsql/reporter/reporter.go +++ b/util/topsql/reporter/reporter.go @@ -169,7 +169,7 @@ func (tsr *RemoteTopSQLReporter) collectWorker() { case <-collectTicker.C: tsr.takeDataFromCollectChanBuffer() case <-reportTicker.C: - tsr.doCollectStmtStatsMaps() + tsr.processStmtStatsData() tsr.takeDataAndSendToReportChan() // Update `reportTicker` if report interval changed. if newInterval := variable.TopSQLVariable.ReportIntervalSeconds.Load(); newInterval != currentReportInterval { @@ -206,10 +206,10 @@ func (tsr *RemoteTopSQLReporter) processCPUTimeData(timestamp uint64, data cpuRe tsr.collecting.appendOthersCPUTime(timestamp, totalEvictedCPUTime) } -// doCollectStmtStatsMaps collects tsr.stmtStatsBuffer into tsr.collecting. +// processStmtStatsData collects tsr.stmtStatsBuffer into tsr.collecting. // All the evicted items will be summary into the others. -func (tsr *RemoteTopSQLReporter) doCollectStmtStatsMaps() { - defer util.Recover("top-sql", "doCollectStmtRecords", nil, false) +func (tsr *RemoteTopSQLReporter) processStmtStatsData() { + defer util.Recover("top-sql", "processStmtStatsData", nil, false) for timestamp, data := range tsr.stmtStatsBuffer { for digest, item := range data { @@ -267,7 +267,7 @@ func (tsr *RemoteTopSQLReporter) reportWorker() { // are finished. time.Sleep(time.Millisecond * 100) // Get top N records from records. - rs := data.collected.topN(int(variable.TopSQLVariable.MaxStatementCount.Load())) + rs := data.collected.compactToTopNAndOthers(int(variable.TopSQLVariable.MaxStatementCount.Load())) // Convert to protobuf data and do report. tsr.doReport(&ReportData{ DataRecords: rs.toProto(), From 2af4950bc57ba3aeee5d5a5411590808f3eb99bb Mon Sep 17 00:00:00 2001 From: mornyx Date: Wed, 29 Dec 2021 11:41:56 +0800 Subject: [PATCH 15/15] Rename test Signed-off-by: mornyx --- util/topsql/reporter/datamodel_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/util/topsql/reporter/datamodel_test.go b/util/topsql/reporter/datamodel_test.go index 35f1b92815c7e..d65bf6ddb2a3d 100644 --- a/util/topsql/reporter/datamodel_test.go +++ b/util/topsql/reporter/datamodel_test.go @@ -266,7 +266,7 @@ func Test_collecting_appendOthers(t *testing.T) { assert.Equal(t, uint64(2), r.tsItems[1].stmtStats.ExecCount) } -func Test_collecting_topN(t *testing.T) { +func Test_collecting_compactToTopNAndOthers(t *testing.T) { c := newCollecting() c.getOrCreateRecord([]byte("SQL-1"), []byte("PLAN-1")).appendCPUTime(1, 1) c.getOrCreateRecord([]byte("SQL-2"), []byte("PLAN-2")).appendCPUTime(1, 2)