From b464ff49c1db34e44bbe5bc1fec9a871e939677b Mon Sep 17 00:00:00 2001 From: Zhenchi Date: Wed, 20 Oct 2021 14:46:48 +0800 Subject: [PATCH 01/47] register pubusb Signed-off-by: Zhenchi --- go.mod | 4 + go.sum | 15 +-- server/rpc_server.go | 2 + server/tidb_test.go | 2 +- sessionctx/variable/tidb_vars.go | 2 +- util/topsql/reporter/client.go | 46 ++++++++- util/topsql/reporter/reporter.go | 80 ++++++++++++---- util/topsql/reporter/reporter_test.go | 3 +- util/topsql/reporter/subscriber.go | 133 ++++++++++++++++++++++++++ util/topsql/topsql.go | 25 ++++- util/topsql/topsql_test.go | 3 +- 11 files changed, 278 insertions(+), 37 deletions(-) create mode 100644 util/topsql/reporter/subscriber.go diff --git a/go.mod b/go.mod index 5d93a62d43475..d9698169c37ee 100644 --- a/go.mod +++ b/go.mod @@ -102,3 +102,7 @@ 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 + +replace github.com/pingcap/kvproto => github.com/zhongzc/kvproto v0.0.0-20211014161008-5d539b8be018 + +replace github.com/pingcap/tipb => github.com/zhongzc/tipb v0.0.0-20211020034530-48fd22f28fcc diff --git a/go.sum b/go.sum index a61ff8747c3e2..ad109158693a7 100644 --- a/go.sum +++ b/go.sum @@ -269,7 +269,6 @@ github.com/gobwas/ws v1.0.2/go.mod h1:szmBTxLgaFppYjEmNtny/v3w89xOydFnnZMcgRRu/E github.com/goccy/go-graphviz v0.0.5/go.mod h1:wXVsXxmyMQU6TN3zGRttjNn3h+iCAS7xQFC6TlNvLhk= github.com/gogo/googleapis v0.0.0-20180223154316-0cd9801be74a/go.mod h1:gf4bu3Q80BeJ6H1S1vYPm8/ELATdvryBaNFGgqEef3s= github.com/gogo/protobuf v0.0.0-20171007142547-342cbe0a0415/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= -github.com/gogo/protobuf v0.0.0-20180717141946-636bf0302bc9/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= github.com/gogo/protobuf v1.2.0/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= github.com/gogo/protobuf v1.2.1/go.mod h1:hp+jE20tsWTFYpLwKvXlhS1hjn+gTNwPg2I6zVXpSg4= @@ -296,7 +295,6 @@ github.com/golang/mock v1.4.4/go.mod h1:l3mdAwkq5BuhzHwde/uurv3sEJeZMXNpwsxVWU71 github.com/golang/mock v1.5.0/go.mod h1:CWnOUgYIOo4TcNZ0wHX3YZCqsaM1I1Jvs6v3mP3KVu8= github.com/golang/mock v1.6.0 h1:ErTB+efbowRARo13NNdxyJji2egdxLGQhRaY+DUumQc= github.com/golang/mock v1.6.0/go.mod h1:p6yTPP+5HYm5mzsMV8JkE6ZKdX+/wYM6Hr+LicevLPs= -github.com/golang/protobuf v0.0.0-20180814211427-aa810b61a9c7/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.1.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= @@ -591,12 +589,6 @@ github.com/pingcap/fn v0.0.0-20200306044125-d5540d389059 h1:Pe2LbxRmbTfAoKJ65bZL github.com/pingcap/fn v0.0.0-20200306044125-d5540d389059/go.mod h1:fMRU1BA1y+r89AxUoaAar4JjrhUkVDt0o0Np6V8XbDQ= github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 h1:surzm05a8C9dN8dIUmo4Be2+pMRb6f55i+UIYrluu2E= github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989/go.mod h1:O17XtbryoCJhkKGbT62+L2OlrniwqiGLSqrmdHCMzZw= -github.com/pingcap/kvproto v0.0.0-20191211054548-3c6b38ea5107/go.mod h1:WWLmULLO7l8IOcQG+t+ItJ3fEcrL5FxF0Wu+HrMy26w= -github.com/pingcap/kvproto v0.0.0-20200411081810-b85805c9476c/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= -github.com/pingcap/kvproto v0.0.0-20210219064844-c1844a4775d6/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= -github.com/pingcap/kvproto v0.0.0-20210805052247-76981389e818/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= -github.com/pingcap/kvproto v0.0.0-20210915062418-0f5764a128ad h1:suBPTeuY6yVF7xvTGeTQ9+tiGzufnORJpCRwzbdN2sc= -github.com/pingcap/kvproto v0.0.0-20210915062418-0f5764a128ad/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= github.com/pingcap/log v0.0.0-20191012051959-b742a5d432e9/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20200511115504-543df19646ad/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20201112100606-8f1e84a3abc8/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= @@ -613,8 +605,6 @@ github.com/pingcap/tidb-dashboard v0.0.0-20210312062513-eef5d6404638/go.mod h1:O github.com/pingcap/tidb-dashboard v0.0.0-20210716172320-2226872e3296/go.mod h1:OCXbZTBTIMRcIt0jFsuCakZP+goYRv6IjawKbwLS2TQ= github.com/pingcap/tidb-tools v5.0.3+incompatible h1:vYMrW9ux+3HRMeRZ1fUOjy2nyiodtuVyAyK270EKBEs= github.com/pingcap/tidb-tools v5.0.3+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM= -github.com/pingcap/tipb v0.0.0-20211008080435-3fd327dfce0e h1:fZY5T65QWiPc9noQJ1UkdwejZyBZjNfxzSyTcBjKrEU= -github.com/pingcap/tipb v0.0.0-20211008080435-3fd327dfce0e/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= @@ -795,6 +785,10 @@ 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/zhongzc/kvproto v0.0.0-20211014161008-5d539b8be018 h1:pvM0foFtJrO4aBtzlmA4q8K4Aem0xEyQCDUckp4mhck= +github.com/zhongzc/kvproto v0.0.0-20211014161008-5d539b8be018/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= +github.com/zhongzc/tipb v0.0.0-20211020034530-48fd22f28fcc h1:kIiwwnffNvd3PjjCjV2HGzX+EyLfnBnVYbPQcqfpOWE= +github.com/zhongzc/tipb v0.0.0-20211020034530-48fd22f28fcc/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= 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= @@ -1179,7 +1173,6 @@ google.golang.org/appengine v1.6.7 h1:FZR1q0exgwxzPzp/aF+VccGrSfxfPpkBqjIIEq3ru6 google.golang.org/appengine v1.6.7/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCIDZVag1xfc= google.golang.org/genproto v0.0.0-20180518175338-11a468237815/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= -google.golang.org/genproto v0.0.0-20181004005441-af9cb2a35e7f/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20190307195333-5fe7a883aa19/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= google.golang.org/genproto v0.0.0-20190418145605-e7d98fc518a7/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= google.golang.org/genproto v0.0.0-20190425155659-357c62f0e4bb/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= diff --git a/server/rpc_server.go b/server/rpc_server.go index 67965ac381f4d..ff17419a0b69e 100644 --- a/server/rpc_server.go +++ b/server/rpc_server.go @@ -32,6 +32,7 @@ import ( "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/memory" + "github.com/pingcap/tidb/util/topsql" "go.uber.org/zap" "google.golang.org/grpc" "google.golang.org/grpc/peer" @@ -54,6 +55,7 @@ func NewRPCServer(config *config.Config, dom *domain.Domain, sm util.SessionMana } diagnosticspb.RegisterDiagnosticsServer(s, rpcSrv) tikvpb.RegisterTikvServer(s, rpcSrv) + topsql.RegisterPubSubServer(s) return s } diff --git a/server/tidb_test.go b/server/tidb_test.go index d788f276ec10a..7f54a6985d99e 100644 --- a/server/tidb_test.go +++ b/server/tidb_test.go @@ -1873,7 +1873,7 @@ func (ts *tidbTestTopSQLSuite) TestTopSQLAgent(c *C) { dbt.mustExec("set @@global.tidb_top_sql_report_interval_seconds=2;") dbt.mustExec("set @@global.tidb_top_sql_max_statement_count=5;") - r := reporter.NewRemoteTopSQLReporter(reporter.NewGRPCReportClient(plancodec.DecodeNormalizedPlan)) + r := reporter.NewRemoteTopSQLReporter(reporter.NewReportClientRegistry(), reporter.NewGRPCReportClient(plancodec.DecodeNormalizedPlan)) tracecpu.GlobalSQLCPUProfiler.SetCollector(&collectorWrapper{r}) // TODO: change to ensure that the right sql statements are reported, not just counts diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 1f95b938c6456..67ef94ee2eb28 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -795,5 +795,5 @@ type TopSQL struct { // TopSQLEnabled uses to check whether enabled the top SQL feature. func TopSQLEnabled() bool { - return TopSQLVariable.Enable.Load() && config.GetGlobalConfig().TopSQL.ReceiverAddress != "" + return TopSQLVariable.Enable.Load() } diff --git a/util/topsql/reporter/client.go b/util/topsql/reporter/client.go index 994189250e52b..d02d15b7f8cfc 100644 --- a/util/topsql/reporter/client.go +++ b/util/topsql/reporter/client.go @@ -16,6 +16,7 @@ package reporter import ( "context" + "github.com/pingcap/tidb/config" "math" "sync" "time" @@ -29,10 +30,39 @@ import ( // ReportClient send data to the target server. type ReportClient interface { - Send(ctx context.Context, addr string, data reportData) error + Send(ctx context.Context, data reportData) error + IsPending() bool + IsDown() bool Close() } +type ReportClientRegistry struct { + sync.Mutex + newClients []ReportClient +} + +func NewReportClientRegistry() *ReportClientRegistry { + return &ReportClientRegistry{} +} + +func (r *ReportClientRegistry) visitAndClear(visit func(client ReportClient)) { + r.Lock() + defer r.Unlock() + + for i := range r.newClients { + visit(r.newClients[i]) + } + r.newClients = r.newClients[:0] + return +} + +func (r *ReportClientRegistry) register(client ReportClient) { + r.Lock() + defer r.Unlock() + + r.newClients = append(r.newClients, client) +} + // GRPCReportClient reports data to grpc servers. type GRPCReportClient struct { curRPCAddr string @@ -52,7 +82,9 @@ var _ ReportClient = &GRPCReportClient{} // Send implements the ReportClient interface. // Currently the implementation will establish a new connection every time, which is suitable for a per-minute sending period -func (r *GRPCReportClient) Send(ctx context.Context, targetRPCAddr string, data reportData) error { +func (r *GRPCReportClient) Send(ctx context.Context, data reportData) error { + targetRPCAddr := config.GetGlobalConfig().TopSQL.ReceiverAddress + if targetRPCAddr == "" { return nil } @@ -87,6 +119,14 @@ func (r *GRPCReportClient) Send(ctx context.Context, targetRPCAddr string, data return nil } +func (r *GRPCReportClient) IsPending() bool { + return len(config.GetGlobalConfig().TopSQL.ReceiverAddress) == 0 +} + +func (r *GRPCReportClient) IsDown() bool { + return false +} + // Close uses to close grpc connection. func (r *GRPCReportClient) Close() { if r.conn == nil { @@ -94,7 +134,7 @@ func (r *GRPCReportClient) Close() { } err := r.conn.Close() if err != nil { - logutil.BgLogger().Warn("[top-sql] grpc client close connection failed", zap.Error(err)) + logutil.BgLogger().Warn("[top-sql] grpc clients close connection failed", zap.Error(err)) } r.conn = nil } diff --git a/util/topsql/reporter/reporter.go b/util/topsql/reporter/reporter.go index 08503610734da..844f500a07c4a 100644 --- a/util/topsql/reporter/reporter.go +++ b/util/topsql/reporter/reporter.go @@ -23,7 +23,6 @@ import ( "time" "github.com/pingcap/failpoint" - "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/util" @@ -120,7 +119,9 @@ type planBinaryDecodeFunc func(string) (string, error) type RemoteTopSQLReporter struct { ctx context.Context cancel context.CancelFunc - client ReportClient + + clients []ReportClient + clientRegistry *ReportClientRegistry // normalizedSQLMap is an map, whose keys are SQL digest strings and values are SQLMeta. normalizedSQLMap atomic.Value // sync.Map @@ -145,12 +146,15 @@ type SQLMeta struct { // // 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 -func NewRemoteTopSQLReporter(client ReportClient) *RemoteTopSQLReporter { +func NewRemoteTopSQLReporter(clientRegistry *ReportClientRegistry, clients ...ReportClient) *RemoteTopSQLReporter { ctx, cancel := context.WithCancel(context.Background()) tsr := &RemoteTopSQLReporter{ - ctx: ctx, - cancel: cancel, - client: client, + ctx: ctx, + cancel: cancel, + + clients: clients, + clientRegistry: clientRegistry, + collectCPUDataChan: make(chan cpuData, 1), reportCollectedDataChan: make(chan collectedData, 1), } @@ -238,7 +242,9 @@ func (tsr *RemoteTopSQLReporter) Collect(timestamp uint64, records []tracecpu.SQ // Close uses to close and release the reporter resource. func (tsr *RemoteTopSQLReporter) Close() { tsr.cancel() - tsr.client.Close() + for i := range tsr.clients { + tsr.clients[i].Close() + } } func addEvictedCPUTime(collectTarget map[string]*dataPoints, timestamp uint64, totalCPUTimeMs uint32) { @@ -324,6 +330,8 @@ func (tsr *RemoteTopSQLReporter) collectWorker() { currentReportInterval := variable.TopSQLVariable.ReportIntervalSeconds.Load() reportTicker := time.NewTicker(time.Second * time.Duration(currentReportInterval)) for { + tsr.handleClientRegistry() + select { case data := <-tsr.collectCPUDataChan: // On receiving data to collect: Write to local data array, and retain records with most CPU time. @@ -341,6 +349,41 @@ func (tsr *RemoteTopSQLReporter) collectWorker() { } } +// for simplify slice removal +var tmpSlice []ReportClient + +func (tsr *RemoteTopSQLReporter) handleClientRegistry() { + tsr.clientRegistry.visitAndClear(func(client ReportClient) { + tsr.clients = append(tsr.clients, client) + }) + + for i := range tsr.clients { + client := tsr.clients[i] + if client.IsDown() { + client.Close() + } else { + tmpSlice = append(tmpSlice, client) + } + } + tsr.clients, tmpSlice = tmpSlice, tsr.clients + tmpSlice = tmpSlice[:0] + + if len(tsr.clients) == 0 { + variable.TopSQLVariable.Enable.Store(false) + return + } + + pendingCnt := 0 + for i := range tsr.clients { + client := tsr.clients[i] + if client.IsPending() { + pendingCnt += 1 + } + } + runningCnt := len(tsr.clients) - pendingCnt + variable.TopSQLVariable.Enable.Store(runningCnt > 0) +} + func encodeKey(buf *bytes.Buffer, sqlDigest, planDigest []byte) string { buf.Reset() buf.Write(sqlDigest) @@ -553,7 +596,6 @@ func (tsr *RemoteTopSQLReporter) doReport(data reportData) { return } - agentAddr := config.GetGlobalConfig().TopSQL.ReceiverAddress timeout := reportTimeout failpoint.Inject("resetTimeoutForTest", func(val failpoint.Value) { if val.(bool) { @@ -563,14 +605,18 @@ func (tsr *RemoteTopSQLReporter) doReport(data reportData) { } } }) - ctx, cancel := context.WithTimeout(tsr.ctx, timeout) - start := time.Now() - err := tsr.client.Send(ctx, agentAddr, data) - if err != nil { - logutil.BgLogger().Warn("[top-sql] client failed to send data", zap.Error(err)) - reportAllDurationFailedHistogram.Observe(time.Since(start).Seconds()) - } else { - reportAllDurationSuccHistogram.Observe(time.Since(start).Seconds()) + + for i := range tsr.clients { + ctx, cancel := context.WithTimeout(tsr.ctx, timeout) + start := time.Now() + + err := tsr.clients[i].Send(ctx, data) + if err != nil { + logutil.BgLogger().Warn("[top-sql] client failed to send data", zap.Error(err)) + reportAllDurationFailedHistogram.Observe(time.Since(start).Seconds()) + } else { + reportAllDurationSuccHistogram.Observe(time.Since(start).Seconds()) + } + cancel() } - cancel() } diff --git a/util/topsql/reporter/reporter_test.go b/util/topsql/reporter/reporter_test.go index d4be70450e596..6eece4fbe6ca2 100644 --- a/util/topsql/reporter/reporter_test.go +++ b/util/topsql/reporter/reporter_test.go @@ -72,7 +72,8 @@ func setupRemoteTopSQLReporter(maxStatementsNum, interval int, addr string) *Rem }) rc := NewGRPCReportClient(mockPlanBinaryDecoderFunc) - ts := NewRemoteTopSQLReporter(rc) + cr := NewReportClientRegistry() + ts := NewRemoteTopSQLReporter(cr, rc) return ts } diff --git a/util/topsql/reporter/subscriber.go b/util/topsql/reporter/subscriber.go new file mode 100644 index 0000000000000..afbfa0fbbc945 --- /dev/null +++ b/util/topsql/reporter/subscriber.go @@ -0,0 +1,133 @@ +package reporter + +import ( + "context" + + "github.com/pingcap/tidb/util/logutil" + "github.com/pingcap/tipb/go-tipb" + "go.uber.org/atomic" + "go.uber.org/zap" +) + +type TopSQLPublisher struct { + decodePlan planBinaryDecodeFunc + clientRegistry *ReportClientRegistry +} + +func NewTopSQLPublisher(decodePlan planBinaryDecodeFunc, clientRegistry *ReportClientRegistry) *TopSQLPublisher { + return &TopSQLPublisher{decodePlan: decodePlan, clientRegistry: clientRegistry} +} + +var _ tipb.TopSQLPubSubServer = &TopSQLPublisher{} + +func (t *TopSQLPublisher) Subscribe(_ *tipb.TopSQLSubRequest, stream tipb.TopSQLPubSub_SubscribeServer) error { + sc := newSubClient(stream, t.decodePlan) + go sc.run() + + t.clientRegistry.register(sc) + return nil +} + +type subClient struct { + stream tipb.TopSQLPubSub_SubscribeServer + dataCh chan reportData + isDown *atomic.Bool + + decodePlan planBinaryDecodeFunc +} + +func newSubClient(stream tipb.TopSQLPubSub_SubscribeServer, decodePlan planBinaryDecodeFunc) *subClient { + dataCh := make(chan reportData) + return &subClient{ + stream: stream, + dataCh: dataCh, + isDown: atomic.NewBool(false), + + decodePlan: decodePlan, + } +} + +func (s *subClient) run() { + defer s.isDown.Store(true) + + for data := range s.dataCh { + var err error + r := &tipb.TopSQLSubResponse{} + + record := &tipb.CPUTimeRecord{} + r.RespOneof = &tipb.TopSQLSubResponse_Record{Record: record} + for i := range data.collectedData { + point := data.collectedData[i] + record.SqlDigest = point.SQLDigest + record.PlanDigest = point.PlanDigest + record.RecordListCpuTimeMs = point.CPUTimeMsList + record.RecordListTimestampSec = point.TimestampList + if err = s.stream.Send(r); err != nil { + logutil.BgLogger().Warn("[top-sql] failed to send record to the subscriber", zap.Error(err)) + return + } + } + + sqlMeta := &tipb.SQLMeta{} + r.RespOneof = &tipb.TopSQLSubResponse_SqlMeta{SqlMeta: sqlMeta} + data.normalizedSQLMap.Range(func(key, value interface{}) bool { + meta := value.(SQLMeta) + sqlMeta.SqlDigest = []byte(key.(string)) + sqlMeta.NormalizedSql = meta.normalizedSQL + sqlMeta.IsInternalSql = meta.isInternal + if err = s.stream.Send(r); err != nil { + return false + } + return true + }) + if err != nil { + logutil.BgLogger().Warn("[top-sql] failed to send SQL meta to the subscriber", zap.Error(err)) + return + } + + planMeta := &tipb.PlanMeta{} + r.RespOneof = &tipb.TopSQLSubResponse_PlanMeta{PlanMeta: planMeta} + data.normalizedPlanMap.Range(func(key, value interface{}) bool { + planDecoded, err1 := s.decodePlan(value.(string)) + if err1 != nil { + logutil.BgLogger().Warn("[top-sql] decode plan failed", zap.Error(err1)) + return true + } + + planMeta.PlanDigest = []byte(key.(string)) + planMeta.NormalizedPlan = planDecoded + if err = s.stream.Send(r); err != nil { + return false + } + return true + }) + if err != nil { + logutil.BgLogger().Warn("[top-sql] failed to send plan meta to the subscriber", zap.Error(err)) + return + } + } +} + +var _ ReportClient = &subClient{} + +func (s *subClient) Send(_ context.Context, data reportData) error { + select { + case s.dataCh <- data: + // sent successfully + default: + logutil.BgLogger().Warn("[top-sql] data channel is full") + } + return nil +} + +func (s *subClient) IsPending() bool { + return false +} + +func (s *subClient) IsDown() bool { + return s.isDown.Load() +} + +func (s *subClient) Close() { + close(s.dataCh) +} diff --git a/util/topsql/topsql.go b/util/topsql/topsql.go index 2c227b4fd76d3..0d5e7505a8462 100644 --- a/util/topsql/topsql.go +++ b/util/topsql/topsql.go @@ -21,12 +21,16 @@ import ( "time" "github.com/pingcap/failpoint" + "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/parser" + "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/plancodec" "github.com/pingcap/tidb/util/topsql/reporter" "github.com/pingcap/tidb/util/topsql/tracecpu" + "github.com/pingcap/tipb/go-tipb" "go.uber.org/zap" + "google.golang.org/grpc" ) const ( @@ -36,14 +40,31 @@ const ( MaxPlanTextSize = 32 * 1024 ) -var globalTopSQLReport reporter.TopSQLReporter +var ( + globalTopSQLReport reporter.TopSQLReporter + globalTopSQLPubSubService tipb.TopSQLPubSubServer +) // SetupTopSQL sets up the top-sql worker. func SetupTopSQL() { rc := reporter.NewGRPCReportClient(plancodec.DecodeNormalizedPlan) - globalTopSQLReport = reporter.NewRemoteTopSQLReporter(rc) + cr := reporter.NewReportClientRegistry() + globalTopSQLReport = reporter.NewRemoteTopSQLReporter(cr, rc) tracecpu.GlobalSQLCPUProfiler.SetCollector(globalTopSQLReport) tracecpu.GlobalSQLCPUProfiler.Run() + + publisher := reporter.NewTopSQLPublisher(plancodec.DecodeNormalizedPlan, cr) + globalTopSQLPubSubService = publisher + + if config.GetGlobalConfig().TopSQL.ReceiverAddress != "" { + variable.TopSQLVariable.Enable.Store(true) + } +} + +func RegisterPubSubServer(s *grpc.Server) { + if globalTopSQLPubSubService != nil { + tipb.RegisterTopSQLPubSubServer(s, globalTopSQLPubSubService) + } } // Close uses to close and release the top sql resource. diff --git a/util/topsql/topsql_test.go b/util/topsql/topsql_test.go index 9bb9968aea52b..f3ffb0d3b1fdf 100644 --- a/util/topsql/topsql_test.go +++ b/util/topsql/topsql_test.go @@ -116,7 +116,8 @@ func TestTopSQLReporter(t *testing.T) { }) client := reporter.NewGRPCReportClient(mockPlanBinaryDecoderFunc) - report := reporter.NewRemoteTopSQLReporter(client) + cr := reporter.NewReportClientRegistry() + report := reporter.NewRemoteTopSQLReporter(cr, client) defer report.Close() tracecpu.GlobalSQLCPUProfiler.SetCollector(&collectorWrapper{report}) From ab683376287fc5ad38e5f0a4ab8e27b740124c6e Mon Sep 17 00:00:00 2001 From: Zhenchi Date: Wed, 20 Oct 2021 19:01:26 +0000 Subject: [PATCH 02/47] run fine Signed-off-by: Zhenchi --- util/topsql/reporter/client.go | 3 +-- util/topsql/reporter/subscriber.go | 37 +++++++++++++++++++++++------- util/topsql/tracecpu/profile.go | 4 ++-- 3 files changed, 32 insertions(+), 12 deletions(-) diff --git a/util/topsql/reporter/client.go b/util/topsql/reporter/client.go index d02d15b7f8cfc..3dd46c60a9d47 100644 --- a/util/topsql/reporter/client.go +++ b/util/topsql/reporter/client.go @@ -16,11 +16,11 @@ package reporter import ( "context" - "github.com/pingcap/tidb/config" "math" "sync" "time" + "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tipb/go-tipb" "go.uber.org/zap" @@ -53,7 +53,6 @@ func (r *ReportClientRegistry) visitAndClear(visit func(client ReportClient)) { visit(r.newClients[i]) } r.newClients = r.newClients[:0] - return } func (r *ReportClientRegistry) register(client ReportClient) { diff --git a/util/topsql/reporter/subscriber.go b/util/topsql/reporter/subscriber.go index afbfa0fbbc945..ad7880e46d5a4 100644 --- a/util/topsql/reporter/subscriber.go +++ b/util/topsql/reporter/subscriber.go @@ -2,6 +2,7 @@ package reporter import ( "context" + "sync" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tipb/go-tipb" @@ -10,21 +11,35 @@ import ( ) type TopSQLPublisher struct { - decodePlan planBinaryDecodeFunc + decodePlan planBinaryDecodeFunc clientRegistry *ReportClientRegistry } -func NewTopSQLPublisher(decodePlan planBinaryDecodeFunc, clientRegistry *ReportClientRegistry) *TopSQLPublisher { - return &TopSQLPublisher{decodePlan: decodePlan, clientRegistry: clientRegistry} +func NewTopSQLPublisher( + decodePlan planBinaryDecodeFunc, + clientRegistry *ReportClientRegistry, +) *TopSQLPublisher { + return &TopSQLPublisher{ + decodePlan: decodePlan, + clientRegistry: clientRegistry, + } } var _ tipb.TopSQLPubSubServer = &TopSQLPublisher{} -func (t *TopSQLPublisher) Subscribe(_ *tipb.TopSQLSubRequest, stream tipb.TopSQLPubSub_SubscribeServer) error { +func (t *TopSQLPublisher) Subscribe( + _ *tipb.TopSQLSubRequest, + stream tipb.TopSQLPubSub_SubscribeServer, +) error { sc := newSubClient(stream, t.decodePlan) - go sc.run() + + var wg sync.WaitGroup + wg.Add(1) + go sc.run(&wg) t.clientRegistry.register(sc) + + wg.Wait() return nil } @@ -36,7 +51,10 @@ type subClient struct { decodePlan planBinaryDecodeFunc } -func newSubClient(stream tipb.TopSQLPubSub_SubscribeServer, decodePlan planBinaryDecodeFunc) *subClient { +func newSubClient( + stream tipb.TopSQLPubSub_SubscribeServer, + decodePlan planBinaryDecodeFunc, +) *subClient { dataCh := make(chan reportData) return &subClient{ stream: stream, @@ -47,8 +65,11 @@ func newSubClient(stream tipb.TopSQLPubSub_SubscribeServer, decodePlan planBinar } } -func (s *subClient) run() { - defer s.isDown.Store(true) +func (s *subClient) run(wg *sync.WaitGroup) { + defer func() { + wg.Done() + s.isDown.Store(true) + }() for data := range s.dataCh { var err error diff --git a/util/topsql/tracecpu/profile.go b/util/topsql/tracecpu/profile.go index 2fe6697dd4ff9..278254902b5f4 100644 --- a/util/topsql/tracecpu/profile.go +++ b/util/topsql/tracecpu/profile.go @@ -285,7 +285,7 @@ func (sp *sqlCPUProfiler) IsEnabled() bool { // Because the GlobalSQLCPUProfiler keep calling pprof.StartCPUProfile to fetch SQL cpu stats, other place (such pprof profile HTTP API handler) call pprof.StartCPUProfile will be failed, // other place should call tracecpu.StartCPUProfile instead of pprof.StartCPUProfile. func StartCPUProfile(w io.Writer) error { - if GlobalSQLCPUProfiler.IsEnabled() { + if variable.TopSQLEnabled() { return GlobalSQLCPUProfiler.startExportCPUProfile(w) } return pprof.StartCPUProfile(w) @@ -294,7 +294,7 @@ func StartCPUProfile(w io.Writer) error { // StopCPUProfile same like pprof.StopCPUProfile. // other place should call tracecpu.StopCPUProfile instead of pprof.StopCPUProfile. func StopCPUProfile() error { - if GlobalSQLCPUProfiler.IsEnabled() { + if GlobalSQLCPUProfiler.hasExportProfileTask() { return GlobalSQLCPUProfiler.stopExportCPUProfile() } pprof.StopCPUProfile() From 8d6b03e0dad26476e48eb001b21338bd0e44dcce Mon Sep 17 00:00:00 2001 From: Zhenchi Date: Thu, 21 Oct 2021 17:33:44 +0800 Subject: [PATCH 03/47] globally enabled not to affect instance level enabled directly Signed-off-by: Zhenchi --- ddl/ddl_worker.go | 2 +- distsql/request_builder.go | 2 +- domain/sysvar_cache.go | 2 +- executor/adapter.go | 2 +- executor/executor.go | 6 +++--- executor/executor_test.go | 2 -- executor/prepared.go | 2 +- executor/set_test.go | 6 ------ executor/update.go | 2 +- server/conn.go | 2 +- server/conn_stmt.go | 4 ++-- session/session.go | 8 ++++---- sessionctx/variable/sysvar.go | 4 ++-- sessionctx/variable/tidb_vars.go | 12 ++++++------ util/topsql/main_test.go | 2 +- util/topsql/reporter/reporter.go | 7 +++---- util/topsql/topsql.go | 2 +- util/topsql/topsql_test.go | 2 +- util/topsql/tracecpu/profile.go | 4 ++-- 19 files changed, 32 insertions(+), 41 deletions(-) diff --git a/ddl/ddl_worker.go b/ddl/ddl_worker.go index 099bb900ae561..d28fc8752c8e1 100644 --- a/ddl/ddl_worker.go +++ b/ddl/ddl_worker.go @@ -468,7 +468,7 @@ func newMetaWithQueueTp(txn kv.Transaction, tp workerType) *meta.Meta { } func (w *worker) setDDLLabelForTopSQL(job *model.Job) { - if !variable.TopSQLEnabled() || job == nil { + if !variable.TopSQLInstanceEnabled() || job == nil { return } diff --git a/distsql/request_builder.go b/distsql/request_builder.go index 44378b2a262a2..c0d7037341227 100644 --- a/distsql/request_builder.go +++ b/distsql/request_builder.go @@ -284,7 +284,7 @@ func (builder *RequestBuilder) SetFromInfoSchema(pis interface{}) *RequestBuilde // SetResourceGroupTag sets the request resource group tag. func (builder *RequestBuilder) SetResourceGroupTag(sc *stmtctx.StatementContext) *RequestBuilder { - if variable.TopSQLEnabled() { + if variable.TopSQLInstanceEnabled() { builder.Request.ResourceGroupTag = sc.GetResourceGroupTag() } return builder diff --git a/domain/sysvar_cache.go b/domain/sysvar_cache.go index c8094b59a4b00..056d07c6bc5b7 100644 --- a/domain/sysvar_cache.go +++ b/domain/sysvar_cache.go @@ -179,7 +179,7 @@ func checkEnableServerGlobalVar(name, sVal string) { case variable.TiDBCapturePlanBaseline: variable.CapturePlanBaseline.Set(sVal, false) case variable.TiDBEnableTopSQL: - variable.TopSQLVariable.Enable.Store(variable.TiDBOptOn(sVal)) + // TODO: whether the topsql global variable is enabled or not doesn't affect instance enabled case variable.TiDBTopSQLPrecisionSeconds: var val int64 val, err = strconv.ParseInt(sVal, 10, 64) diff --git a/executor/adapter.go b/executor/adapter.go index 457e0fee67579..6bdb3143d12ca 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -309,7 +309,7 @@ func (a *ExecStmt) RebuildPlan(ctx context.Context) (int64, error) { } func (a *ExecStmt) setPlanLabelForTopSQL(ctx context.Context) context.Context { - if a.Plan == nil || !variable.TopSQLEnabled() { + if a.Plan == nil || !variable.TopSQLInstanceEnabled() { return ctx } vars := a.Ctx.GetSessionVars() diff --git a/executor/executor.go b/executor/executor.go index 572f30649a6ad..8a4980e8380a5 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -970,7 +970,7 @@ func (e *SelectLockExec) Next(ctx context.Context, req *chunk.Chunk) error { func newLockCtx(seVars *variable.SessionVars, lockWaitTime int64) *tikvstore.LockCtx { var planDigest *parser.Digest _, sqlDigest := seVars.StmtCtx.SQLDigest() - if variable.TopSQLEnabled() { + if variable.TopSQLInstanceEnabled() { _, planDigest = seVars.StmtCtx.GetPlanDigest() } lockCtx := tikvstore.NewLockCtx(seVars.TxnCtx.GetForUpdateTS(), lockWaitTime, seVars.StmtCtx.GetLockWaitStartTime()) @@ -1715,7 +1715,7 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) { goCtx = pprof.WithLabels(goCtx, pprof.Labels("sql", util.QueryStrForLog(prepareStmt.NormalizedSQL))) pprof.SetGoroutineLabels(goCtx) } - if variable.TopSQLEnabled() && prepareStmt.SQLDigest != nil { + if variable.TopSQLInstanceEnabled() && prepareStmt.SQLDigest != nil { topsql.AttachSQLInfo(goCtx, prepareStmt.NormalizedSQL, prepareStmt.SQLDigest, "", nil, vars.InRestrictedSQL) } } @@ -1895,7 +1895,7 @@ func FillVirtualColumnValue(virtualRetTypes []*types.FieldType, virtualColumnInd } func setResourceGroupTagForTxn(sc *stmtctx.StatementContext, snapshot kv.Snapshot) { - if snapshot != nil && variable.TopSQLEnabled() { + if snapshot != nil && variable.TopSQLInstanceEnabled() { snapshot.SetOption(kv.ResourceGroupTag, sc.GetResourceGroupTag()) } } diff --git a/executor/executor_test.go b/executor/executor_test.go index 54d6f37ddb92c..1f0a815969814 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -8652,8 +8652,6 @@ func (s *testResourceTagSuite) TestResourceGroupTag(c *C) { tk.MustExec("create table t(a int, b int, unique index idx(a));") tbInfo := testGetTableByName(c, tk.Se, "test", "t") - // Enable Top SQL - variable.TopSQLVariable.Enable.Store(true) config.UpdateGlobal(func(conf *config.Config) { conf.TopSQL.ReceiverAddress = "mock-agent" }) diff --git a/executor/prepared.go b/executor/prepared.go index d4db1337ee396..34ab0218860ce 100644 --- a/executor/prepared.go +++ b/executor/prepared.go @@ -190,7 +190,7 @@ func (e *PrepareExec) Next(ctx context.Context, req *chunk.Chunk) error { SchemaVersion: ret.InfoSchema.SchemaMetaVersion(), } normalizedSQL, digest := parser.NormalizeDigest(prepared.Stmt.Text()) - if variable.TopSQLEnabled() { + if variable.TopSQLInstanceEnabled() { ctx = topsql.AttachSQLInfo(ctx, normalizedSQL, digest, "", nil, vars.InRestrictedSQL) } diff --git a/executor/set_test.go b/executor/set_test.go index dc2f8e1ffdaca..aab8e9d2e6d2b 100644 --- a/executor/set_test.go +++ b/executor/set_test.go @@ -1422,12 +1422,6 @@ func (s *testSerialSuite) TestSetTopSQLVariables(c *C) { }() tk := testkit.NewTestKit(c, s.store) - tk.MustExec("set @@global.tidb_enable_top_sql='On';") - tk.MustQuery("select @@global.tidb_enable_top_sql;").Check(testkit.Rows("1")) - c.Assert(variable.TopSQLVariable.Enable.Load(), IsTrue) - tk.MustExec("set @@global.tidb_enable_top_sql='off';") - tk.MustQuery("select @@global.tidb_enable_top_sql;").Check(testkit.Rows("0")) - c.Assert(variable.TopSQLVariable.Enable.Load(), IsFalse) tk.MustExec("set @@global.tidb_top_sql_precision_seconds=2;") tk.MustQuery("select @@global.tidb_top_sql_precision_seconds;").Check(testkit.Rows("2")) diff --git a/executor/update.go b/executor/update.go index 03a9979878ab1..ef589dd314ca9 100644 --- a/executor/update.go +++ b/executor/update.go @@ -271,7 +271,7 @@ func (e *UpdateExec) updateRows(ctx context.Context) (int, error) { txn.GetSnapshot().SetOption(kv.CollectRuntimeStats, e.stats.SnapshotRuntimeStats) } } - if variable.TopSQLEnabled() { + if variable.TopSQLInstanceEnabled() { txn, err := e.ctx.Txn(true) if err == nil { txn.SetOption(kv.ResourceGroupTag, e.ctx.GetSessionVars().StmtCtx.GetResourceGroupTag()) diff --git a/server/conn.go b/server/conn.go index 9c48abe1cbb12..d71ada8e33809 100644 --- a/server/conn.go +++ b/server/conn.go @@ -1165,7 +1165,7 @@ func (cc *clientConn) dispatch(ctx context.Context, data []byte) error { cc.lastPacket = data cmd := data[0] data = data[1:] - if variable.TopSQLEnabled() { + if variable.TopSQLInstanceEnabled() { defer pprof.SetGoroutineLabels(ctx) } if variable.EnablePProfSQLCPU.Load() { diff --git a/server/conn_stmt.go b/server/conn_stmt.go index 980de55c6c896..203cbc5f2da79 100644 --- a/server/conn_stmt.go +++ b/server/conn_stmt.go @@ -127,7 +127,7 @@ func (cc *clientConn) handleStmtExecute(ctx context.Context, data []byte) (err e stmtID := binary.LittleEndian.Uint32(data[0:4]) pos += 4 - if variable.TopSQLEnabled() { + if variable.TopSQLInstanceEnabled() { preparedStmt, _ := cc.preparedStmtID2CachePreparedStmt(stmtID) if preparedStmt != nil && preparedStmt.SQLDigest != nil { ctx = topsql.AttachSQLInfo(ctx, preparedStmt.NormalizedSQL, preparedStmt.SQLDigest, "", nil, false) @@ -273,7 +273,7 @@ func (cc *clientConn) handleStmtFetch(ctx context.Context, data []byte) (err err return errors.Annotate(mysql.NewErr(mysql.ErrUnknownStmtHandler, strconv.FormatUint(uint64(stmtID), 10), "stmt_fetch"), cc.preparedStmt2String(stmtID)) } - if variable.TopSQLEnabled() { + if variable.TopSQLInstanceEnabled() { prepareObj, _ := cc.preparedStmtID2CachePreparedStmt(stmtID) if prepareObj != nil && prepareObj.SQLDigest != nil { ctx = topsql.AttachSQLInfo(ctx, prepareObj.NormalizedSQL, prepareObj.SQLDigest, "", nil, false) diff --git a/session/session.go b/session/session.go index 6353b2c00112a..f65aa50444bcf 100644 --- a/session/session.go +++ b/session/session.go @@ -1268,7 +1268,7 @@ func (s *session) ExecuteInternal(ctx context.Context, sql string, args ...inter s.sessionVars.InRestrictedSQL = true defer func() { s.sessionVars.InRestrictedSQL = origin - if variable.TopSQLEnabled() { + if variable.TopSQLInstanceEnabled() { // Restore the goroutine label by using the original ctx after execution is finished. pprof.SetGoroutineLabels(ctx) } @@ -1410,7 +1410,7 @@ func (s *session) ParseWithParams(ctx context.Context, sql string, args ...inter for _, warn := range warns { s.sessionVars.StmtCtx.AppendWarning(util.SyntaxWarn(warn)) } - if variable.TopSQLEnabled() { + if variable.TopSQLInstanceEnabled() { normalized, digest := parser.NormalizeDigest(sql) if digest != nil { // Reset the goroutine label when internal sql execute finish. @@ -1424,7 +1424,7 @@ func (s *session) ParseWithParams(ctx context.Context, sql string, args ...inter // ExecRestrictedStmt implements RestrictedSQLExecutor interface. func (s *session) ExecRestrictedStmt(ctx context.Context, stmtNode ast.StmtNode, opts ...sqlexec.OptionFuncAlias) ( []chunk.Row, []*ast.ResultField, error) { - if variable.TopSQLEnabled() { + if variable.TopSQLInstanceEnabled() { defer pprof.SetGoroutineLabels(ctx) } var execOption sqlexec.ExecOption @@ -1532,7 +1532,7 @@ func (s *session) ExecuteStmt(ctx context.Context, stmtNode ast.StmtNode) (sqlex return nil, err } normalizedSQL, digest := s.sessionVars.StmtCtx.SQLDigest() - if variable.TopSQLEnabled() { + if variable.TopSQLInstanceEnabled() { ctx = topsql.AttachSQLInfo(ctx, normalizedSQL, digest, "", nil, s.sessionVars.InRestrictedSQL) } diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 2c678796d3bca..d4c836eb335fd 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -1743,9 +1743,9 @@ var defaultSysVars = []*SysVar{ }}, // variable for top SQL feature. {Scope: ScopeGlobal, Name: TiDBEnableTopSQL, Value: BoolToOnOff(DefTiDBTopSQLEnable), Type: TypeBool, Hidden: true, AllowEmpty: true, GetGlobal: func(s *SessionVars) (string, error) { - return BoolToOnOff(TopSQLVariable.Enable.Load()), nil + return Off, nil }, SetGlobal: func(vars *SessionVars, s string) error { - TopSQLVariable.Enable.Store(TiDBOptOn(s)) + // TODO: call pd to enable topsql globally. Not to affect the current instance directly. return nil }}, {Scope: ScopeGlobal, Name: TiDBTopSQLPrecisionSeconds, Value: strconv.Itoa(DefTiDBTopSQLPrecisionSeconds), Type: TypeInt, Hidden: true, MinValue: 1, MaxValue: math.MaxInt64, GetGlobal: func(s *SessionVars) (string, error) { diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 67ef94ee2eb28..64861bd125d14 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -769,7 +769,7 @@ var ( DefExecutorConcurrency = 5 MemoryUsageAlarmRatio = atomic.NewFloat64(config.GetGlobalConfig().Performance.MemoryUsageAlarmRatio) TopSQLVariable = TopSQL{ - Enable: atomic.NewBool(DefTiDBTopSQLEnable), + InstanceEnable: atomic.NewBool(DefTiDBTopSQLEnable), PrecisionSeconds: atomic.NewInt64(DefTiDBTopSQLPrecisionSeconds), MaxStatementCount: atomic.NewInt64(DefTiDBTopSQLMaxStatementCount), MaxCollect: atomic.NewInt64(DefTiDBTopSQLMaxCollect), @@ -781,8 +781,8 @@ var ( // TopSQL is the variable for control top sql feature. type TopSQL struct { - // Enable top-sql or not. - Enable *atomic.Bool + // InstanceEnable represents if TopSQL is enabled on the current instance or not. + InstanceEnable *atomic.Bool // The refresh interval of top-sql. PrecisionSeconds *atomic.Int64 // The maximum number of statements kept in memory. @@ -793,7 +793,7 @@ type TopSQL struct { ReportIntervalSeconds *atomic.Int64 } -// TopSQLEnabled uses to check whether enabled the top SQL feature. -func TopSQLEnabled() bool { - return TopSQLVariable.Enable.Load() +// TopSQLInstanceEnabled is used to check if TopSQL is enabled on the current instance. +func TopSQLInstanceEnabled() bool { + return TopSQLVariable.InstanceEnable.Load() } diff --git a/util/topsql/main_test.go b/util/topsql/main_test.go index f5e3dc3f7d0cf..60254b0729401 100644 --- a/util/topsql/main_test.go +++ b/util/topsql/main_test.go @@ -28,7 +28,7 @@ func TestMain(m *testing.M) { testbridge.WorkaroundGoCheckFlags() // set up - variable.TopSQLVariable.Enable.Store(true) + variable.TopSQLVariable.InstanceEnable.Store(true) config.UpdateGlobal(func(conf *config.Config) { conf.TopSQL.ReceiverAddress = "mock" }) diff --git a/util/topsql/reporter/reporter.go b/util/topsql/reporter/reporter.go index 844f500a07c4a..5c280d9072081 100644 --- a/util/topsql/reporter/reporter.go +++ b/util/topsql/reporter/reporter.go @@ -368,9 +368,8 @@ func (tsr *RemoteTopSQLReporter) handleClientRegistry() { tsr.clients, tmpSlice = tmpSlice, tsr.clients tmpSlice = tmpSlice[:0] - if len(tsr.clients) == 0 { - variable.TopSQLVariable.Enable.Store(false) - return + if len(tsr.clients) > 256 { + logutil.BgLogger().Warn("[top-sql] too many clients", zap.Int("count", len(tsr.clients))) } pendingCnt := 0 @@ -381,7 +380,7 @@ func (tsr *RemoteTopSQLReporter) handleClientRegistry() { } } runningCnt := len(tsr.clients) - pendingCnt - variable.TopSQLVariable.Enable.Store(runningCnt > 0) + variable.TopSQLVariable.InstanceEnable.Store(runningCnt > 0) } func encodeKey(buf *bytes.Buffer, sqlDigest, planDigest []byte) string { diff --git a/util/topsql/topsql.go b/util/topsql/topsql.go index 0d5e7505a8462..0127254b58b74 100644 --- a/util/topsql/topsql.go +++ b/util/topsql/topsql.go @@ -57,7 +57,7 @@ func SetupTopSQL() { globalTopSQLPubSubService = publisher if config.GetGlobalConfig().TopSQL.ReceiverAddress != "" { - variable.TopSQLVariable.Enable.Store(true) + variable.TopSQLVariable.InstanceEnable.Store(true) } } diff --git a/util/topsql/topsql_test.go b/util/topsql/topsql_test.go index f3ffb0d3b1fdf..877f87feabbcb 100644 --- a/util/topsql/topsql_test.go +++ b/util/topsql/topsql_test.go @@ -211,7 +211,7 @@ func TestMaxSQLAndPlanTest(t *testing.T) { } func setTopSQLEnable(enabled bool) { - variable.TopSQLVariable.Enable.Store(enabled) + variable.TopSQLVariable.InstanceEnable.Store(enabled) } func mockExecuteSQL(sql, plan string) { diff --git a/util/topsql/tracecpu/profile.go b/util/topsql/tracecpu/profile.go index 278254902b5f4..036e84428c21a 100644 --- a/util/topsql/tracecpu/profile.go +++ b/util/topsql/tracecpu/profile.go @@ -278,14 +278,14 @@ func (sp *sqlCPUProfiler) hasExportProfileTask() bool { // IsEnabled return true if it is(should be) enabled. It exports for tests. func (sp *sqlCPUProfiler) IsEnabled() bool { - return variable.TopSQLEnabled() || sp.hasExportProfileTask() + return variable.TopSQLInstanceEnabled() || sp.hasExportProfileTask() } // StartCPUProfile same like pprof.StartCPUProfile. // Because the GlobalSQLCPUProfiler keep calling pprof.StartCPUProfile to fetch SQL cpu stats, other place (such pprof profile HTTP API handler) call pprof.StartCPUProfile will be failed, // other place should call tracecpu.StartCPUProfile instead of pprof.StartCPUProfile. func StartCPUProfile(w io.Writer) error { - if variable.TopSQLEnabled() { + if variable.TopSQLInstanceEnabled() { return GlobalSQLCPUProfiler.startExportCPUProfile(w) } return pprof.StartCPUProfile(w) From 38fccf9a826563cfc3b0434348e150dc7dcbe4f3 Mon Sep 17 00:00:00 2001 From: Zhenchi Date: Thu, 21 Oct 2021 17:37:16 +0800 Subject: [PATCH 04/47] tiny fix Signed-off-by: Zhenchi --- util/topsql/reporter/client.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/util/topsql/reporter/client.go b/util/topsql/reporter/client.go index 3dd46c60a9d47..1ab9267188d77 100644 --- a/util/topsql/reporter/client.go +++ b/util/topsql/reporter/client.go @@ -133,7 +133,7 @@ func (r *GRPCReportClient) Close() { } err := r.conn.Close() if err != nil { - logutil.BgLogger().Warn("[top-sql] grpc clients close connection failed", zap.Error(err)) + logutil.BgLogger().Warn("[top-sql] grpc client close connection failed", zap.Error(err)) } r.conn = nil } From 6d4a115c26ef50a59ed17260007e9c6ead2cd0c1 Mon Sep 17 00:00:00 2001 From: Zhenchi Date: Thu, 21 Oct 2021 17:45:03 +0800 Subject: [PATCH 05/47] add license Signed-off-by: Zhenchi --- util/topsql/reporter/subscriber.go | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/util/topsql/reporter/subscriber.go b/util/topsql/reporter/subscriber.go index ad7880e46d5a4..a4e44df852211 100644 --- a/util/topsql/reporter/subscriber.go +++ b/util/topsql/reporter/subscriber.go @@ -1,3 +1,17 @@ +// 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 ( From 91d7143f7d612412a4a7466756263c8428d2a3fd Mon Sep 17 00:00:00 2001 From: Zhenchi Date: Mon, 25 Oct 2021 20:30:52 +0800 Subject: [PATCH 06/47] add tests & add comment Signed-off-by: Zhenchi --- go.mod | 4 +- go.sum | 15 +++- server/tidb_test.go | 50 +++++------ util/topsql/reporter/client.go | 6 ++ util/topsql/reporter/mock/publisher.go | 69 +++++++++++++++ util/topsql/reporter/mock/server.go | 40 ++++----- util/topsql/reporter/reporter.go | 1 + util/topsql/reporter/reporter_test.go | 8 +- util/topsql/reporter/subscriber.go | 10 +++ util/topsql/topsql_test.go | 117 ++++++++++++++++++++++++- 10 files changed, 262 insertions(+), 58 deletions(-) create mode 100644 util/topsql/reporter/mock/publisher.go diff --git a/go.mod b/go.mod index d9698169c37ee..e2873cf529cda 100644 --- a/go.mod +++ b/go.mod @@ -103,6 +103,4 @@ 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 -replace github.com/pingcap/kvproto => github.com/zhongzc/kvproto v0.0.0-20211014161008-5d539b8be018 - -replace github.com/pingcap/tipb => github.com/zhongzc/tipb v0.0.0-20211020034530-48fd22f28fcc +replace github.com/pingcap/tipb => github.com/zhongzc/tipb v0.0.0-20211025093821-d1bfd2ef18e8 diff --git a/go.sum b/go.sum index ad109158693a7..0fc66658f7647 100644 --- a/go.sum +++ b/go.sum @@ -269,6 +269,7 @@ github.com/gobwas/ws v1.0.2/go.mod h1:szmBTxLgaFppYjEmNtny/v3w89xOydFnnZMcgRRu/E github.com/goccy/go-graphviz v0.0.5/go.mod h1:wXVsXxmyMQU6TN3zGRttjNn3h+iCAS7xQFC6TlNvLhk= github.com/gogo/googleapis v0.0.0-20180223154316-0cd9801be74a/go.mod h1:gf4bu3Q80BeJ6H1S1vYPm8/ELATdvryBaNFGgqEef3s= github.com/gogo/protobuf v0.0.0-20171007142547-342cbe0a0415/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= +github.com/gogo/protobuf v0.0.0-20180717141946-636bf0302bc9/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= github.com/gogo/protobuf v1.2.0/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= github.com/gogo/protobuf v1.2.1/go.mod h1:hp+jE20tsWTFYpLwKvXlhS1hjn+gTNwPg2I6zVXpSg4= @@ -295,6 +296,7 @@ github.com/golang/mock v1.4.4/go.mod h1:l3mdAwkq5BuhzHwde/uurv3sEJeZMXNpwsxVWU71 github.com/golang/mock v1.5.0/go.mod h1:CWnOUgYIOo4TcNZ0wHX3YZCqsaM1I1Jvs6v3mP3KVu8= github.com/golang/mock v1.6.0 h1:ErTB+efbowRARo13NNdxyJji2egdxLGQhRaY+DUumQc= github.com/golang/mock v1.6.0/go.mod h1:p6yTPP+5HYm5mzsMV8JkE6ZKdX+/wYM6Hr+LicevLPs= +github.com/golang/protobuf v0.0.0-20180814211427-aa810b61a9c7/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.1.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= @@ -589,6 +591,12 @@ github.com/pingcap/fn v0.0.0-20200306044125-d5540d389059 h1:Pe2LbxRmbTfAoKJ65bZL github.com/pingcap/fn v0.0.0-20200306044125-d5540d389059/go.mod h1:fMRU1BA1y+r89AxUoaAar4JjrhUkVDt0o0Np6V8XbDQ= github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 h1:surzm05a8C9dN8dIUmo4Be2+pMRb6f55i+UIYrluu2E= github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989/go.mod h1:O17XtbryoCJhkKGbT62+L2OlrniwqiGLSqrmdHCMzZw= +github.com/pingcap/kvproto v0.0.0-20191211054548-3c6b38ea5107/go.mod h1:WWLmULLO7l8IOcQG+t+ItJ3fEcrL5FxF0Wu+HrMy26w= +github.com/pingcap/kvproto v0.0.0-20200411081810-b85805c9476c/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= +github.com/pingcap/kvproto v0.0.0-20210219064844-c1844a4775d6/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= +github.com/pingcap/kvproto v0.0.0-20210805052247-76981389e818/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= +github.com/pingcap/kvproto v0.0.0-20210915062418-0f5764a128ad h1:suBPTeuY6yVF7xvTGeTQ9+tiGzufnORJpCRwzbdN2sc= +github.com/pingcap/kvproto v0.0.0-20210915062418-0f5764a128ad/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= github.com/pingcap/log v0.0.0-20191012051959-b742a5d432e9/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20200511115504-543df19646ad/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20201112100606-8f1e84a3abc8/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= @@ -785,10 +793,8 @@ 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/zhongzc/kvproto v0.0.0-20211014161008-5d539b8be018 h1:pvM0foFtJrO4aBtzlmA4q8K4Aem0xEyQCDUckp4mhck= -github.com/zhongzc/kvproto v0.0.0-20211014161008-5d539b8be018/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= -github.com/zhongzc/tipb v0.0.0-20211020034530-48fd22f28fcc h1:kIiwwnffNvd3PjjCjV2HGzX+EyLfnBnVYbPQcqfpOWE= -github.com/zhongzc/tipb v0.0.0-20211020034530-48fd22f28fcc/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= +github.com/zhongzc/tipb v0.0.0-20211025093821-d1bfd2ef18e8 h1:THw6y9o24NejDF9/JaTMXhUaNLlgvfbJbvVABy89Y9s= +github.com/zhongzc/tipb v0.0.0-20211025093821-d1bfd2ef18e8/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= 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= @@ -1173,6 +1179,7 @@ google.golang.org/appengine v1.6.7 h1:FZR1q0exgwxzPzp/aF+VccGrSfxfPpkBqjIIEq3ru6 google.golang.org/appengine v1.6.7/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCIDZVag1xfc= google.golang.org/genproto v0.0.0-20180518175338-11a468237815/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= +google.golang.org/genproto v0.0.0-20181004005441-af9cb2a35e7f/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20190307195333-5fe7a883aa19/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= google.golang.org/genproto v0.0.0-20190418145605-e7d98fc518a7/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= google.golang.org/genproto v0.0.0-20190425155659-357c62f0e4bb/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= diff --git a/server/tidb_test.go b/server/tidb_test.go index 7f54a6985d99e..b344961fe3165 100644 --- a/server/tidb_test.go +++ b/server/tidb_test.go @@ -1630,7 +1630,6 @@ func (ts *tidbTestTopSQLSuite) TestTopSQLCPUProfile(c *C) { dbt.mustExec("create table t (a int auto_increment, b int, unique index idx(a));") dbt.mustExec("create table t1 (a int auto_increment, b int, unique index idx(a));") dbt.mustExec("create table t2 (a int auto_increment, b int, unique index idx(a));") - dbt.mustExec("set @@global.tidb_enable_top_sql='On';") config.UpdateGlobal(func(conf *config.Config) { conf.TopSQL.ReceiverAddress = "127.0.0.1:4001" }) @@ -1826,7 +1825,7 @@ func (ts *tidbTestTopSQLSuite) TestTopSQLCPUProfile(c *C) { checkFn("commit", "") } -func (ts *tidbTestTopSQLSuite) TestTopSQLAgent(c *C) { +func (ts *tidbTestTopSQLSuite) TestTopSQLReceiver(c *C) { c.Skip("unstable, skip it and fix it before 20210702") db, err := sql.Open("mysql", ts.getDSN()) c.Assert(err, IsNil, Commentf("Error connecting")) @@ -1834,10 +1833,10 @@ func (ts *tidbTestTopSQLSuite) TestTopSQLAgent(c *C) { err := db.Close() c.Assert(err, IsNil) }() - agentServer, err := mockTopSQLReporter.StartMockAgentServer() + receiverServer, err := mockTopSQLReporter.StartMockReceiverServer() c.Assert(err, IsNil) defer func() { - agentServer.Stop() + receiverServer.Stop() }() c.Assert(failpoint.Enable("github.com/pingcap/tidb/util/topsql/reporter/resetTimeoutForTest", `return(true)`), IsNil) @@ -1867,7 +1866,6 @@ func (ts *tidbTestTopSQLSuite) TestTopSQLAgent(c *C) { conf.TopSQL.ReceiverAddress = addr }) } - dbt.mustExec("set @@global.tidb_enable_top_sql='On';") setTopSQLReceiverAddress("") dbt.mustExec("set @@global.tidb_top_sql_precision_seconds=1;") dbt.mustExec("set @@global.tidb_top_sql_report_interval_seconds=2;") @@ -1878,16 +1876,16 @@ func (ts *tidbTestTopSQLSuite) TestTopSQLAgent(c *C) { // TODO: change to ensure that the right sql statements are reported, not just counts checkFn := func(n int) { - records := agentServer.GetLatestRecords() + records := receiverServer.GetLatestRecords() c.Assert(len(records), Equals, n) for _, r := range records { - sqlMeta, exist := agentServer.GetSQLMetaByDigestBlocking(r.SqlDigest, time.Second) + sqlMeta, exist := receiverServer.GetSQLMetaByDigestBlocking(r.SqlDigest, time.Second) c.Assert(exist, IsTrue) c.Check(sqlMeta.NormalizedSql, Matches, "select.*from.*join.*") if len(r.PlanDigest) == 0 { continue } - plan, exist := agentServer.GetPlanMetaByDigestBlocking(r.PlanDigest, time.Second) + plan, exist := receiverServer.GetPlanMetaByDigestBlocking(r.PlanDigest, time.Second) c.Assert(exist, IsTrue) plan = strings.Replace(plan, "\n", " ", -1) plan = strings.Replace(plan, "\t", " ", -1) @@ -1912,22 +1910,22 @@ func (ts *tidbTestTopSQLSuite) TestTopSQLAgent(c *C) { cancel := runWorkload(0, 10) // Test with null agent address, the agent server can't receive any record. setTopSQLReceiverAddress("") - agentServer.WaitCollectCnt(1, time.Second*4) + receiverServer.WaitCollectCnt(1, time.Second*4) checkFn(0) // Test after set agent address and the evict take effect. dbt.mustExec("set @@global.tidb_top_sql_max_statement_count=5;") - setTopSQLReceiverAddress(agentServer.Address()) - agentServer.WaitCollectCnt(1, time.Second*4) + setTopSQLReceiverAddress(receiverServer.Address()) + receiverServer.WaitCollectCnt(1, time.Second*4) checkFn(5) // Test with wrong agent address, the agent server can't receive any record. dbt.mustExec("set @@global.tidb_top_sql_max_statement_count=8;") setTopSQLReceiverAddress("127.0.0.1:65530") - agentServer.WaitCollectCnt(1, time.Second*4) + receiverServer.WaitCollectCnt(1, time.Second*4) checkFn(0) // Test after set agent address and the evict take effect. - setTopSQLReceiverAddress(agentServer.Address()) - agentServer.WaitCollectCnt(1, time.Second*4) + setTopSQLReceiverAddress(receiverServer.Address()) + receiverServer.WaitCollectCnt(1, time.Second*4) checkFn(8) cancel() // cancel case 1 @@ -1936,19 +1934,19 @@ func (ts *tidbTestTopSQLSuite) TestTopSQLAgent(c *C) { // empty agent address, should not collect records dbt.mustExec("set @@global.tidb_top_sql_max_statement_count=5;") setTopSQLReceiverAddress("") - agentServer.WaitCollectCnt(1, time.Second*4) + receiverServer.WaitCollectCnt(1, time.Second*4) checkFn(0) // set correct address, should collect records - setTopSQLReceiverAddress(agentServer.Address()) - agentServer.WaitCollectCnt(1, time.Second*4) + setTopSQLReceiverAddress(receiverServer.Address()) + receiverServer.WaitCollectCnt(1, time.Second*4) checkFn(5) // agent server hangs for a while - agentServer.HangFromNow(time.Second * 6) + receiverServer.HangFromNow(time.Second * 6) // run another set of SQL queries cancel2() cancel3 := runWorkload(11, 20) - agentServer.WaitCollectCnt(1, time.Second*8) + receiverServer.WaitCollectCnt(1, time.Second*8) checkFn(5) cancel3() @@ -1956,24 +1954,24 @@ func (ts *tidbTestTopSQLSuite) TestTopSQLAgent(c *C) { cancel4 := runWorkload(0, 10) // empty agent address, should not collect records setTopSQLReceiverAddress("") - agentServer.WaitCollectCnt(1, time.Second*4) + receiverServer.WaitCollectCnt(1, time.Second*4) checkFn(0) // set correct address, should collect records - setTopSQLReceiverAddress(agentServer.Address()) - agentServer.WaitCollectCnt(1, time.Second*8) + setTopSQLReceiverAddress(receiverServer.Address()) + receiverServer.WaitCollectCnt(1, time.Second*8) checkFn(5) // run another set of SQL queries cancel4() cancel5 := runWorkload(11, 20) // agent server shutdown - agentServer.Stop() + receiverServer.Stop() // agent server restart - agentServer, err = mockTopSQLReporter.StartMockAgentServer() + receiverServer, err = mockTopSQLReporter.StartMockReceiverServer() c.Assert(err, IsNil) - setTopSQLReceiverAddress(agentServer.Address()) + setTopSQLReceiverAddress(receiverServer.Address()) // check result - agentServer.WaitCollectCnt(2, time.Second*8) + receiverServer.WaitCollectCnt(2, time.Second*8) checkFn(5) cancel5() } diff --git a/util/topsql/reporter/client.go b/util/topsql/reporter/client.go index 1ab9267188d77..06e45ee13ad51 100644 --- a/util/topsql/reporter/client.go +++ b/util/topsql/reporter/client.go @@ -31,8 +31,14 @@ import ( // ReportClient send data to the target server. type ReportClient interface { Send(ctx context.Context, data reportData) error + + // IsPending indicates that ReportClient is not expecting to receive records for now and may resume in the future. IsPending() bool + + // IsDown indicates that the client has been closed and can be cleared. + // Note that: once a ReportClient is closed, it cannot go back to be non-closed. IsDown() bool + Close() } diff --git a/util/topsql/reporter/mock/publisher.go b/util/topsql/reporter/mock/publisher.go new file mode 100644 index 0000000000000..82844fd39c54d --- /dev/null +++ b/util/topsql/reporter/mock/publisher.go @@ -0,0 +1,69 @@ +// 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 mock + +import ( + "fmt" + "net" + + "github.com/pingcap/tidb/util/logutil" + "github.com/pingcap/tidb/util/topsql/reporter" + "github.com/pingcap/tipb/go-tipb" + "go.uber.org/zap" + "google.golang.org/grpc" +) + +type mockPublisherServer struct { + addr string + grpcServer *grpc.Server +} + +// StartMockPublisherServer starts the mock publisher server. +func StartMockPublisherServer( + planBinaryDecodeFunc func(string) (string, error), + clientRegistry *reporter.ReportClientRegistry, +) (*mockPublisherServer, error) { + addr := "127.0.0.1:0" + lis, err := net.Listen("tcp", addr) + if err != nil { + return nil, err + } + server := grpc.NewServer() + + publisherServer := reporter.NewTopSQLPublisher(planBinaryDecodeFunc, clientRegistry) + tipb.RegisterTopSQLPubSubServer(server, publisherServer) + + go func() { + err := server.Serve(lis) + if err != nil { + logutil.BgLogger().Warn("[top-sql] mock publisher server serve failed", zap.Error(err)) + } + }() + + return &mockPublisherServer{ + addr: fmt.Sprintf("127.0.0.1:%d", lis.Addr().(*net.TCPAddr).Port), + grpcServer: server, + }, nil +} + +func (svr *mockPublisherServer) Address() string { + return svr.addr +} + +func (svr *mockPublisherServer) Stop() { + if svr.grpcServer != nil { + svr.grpcServer.Stop() + } +} diff --git a/util/topsql/reporter/mock/server.go b/util/topsql/reporter/mock/server.go index 5dca814fb30e5..aed6280956268 100644 --- a/util/topsql/reporter/mock/server.go +++ b/util/topsql/reporter/mock/server.go @@ -28,7 +28,7 @@ import ( "google.golang.org/grpc" ) -type mockAgentServer struct { +type mockReceiverServer struct { sync.Mutex addr string grpcServer *grpc.Server @@ -41,42 +41,42 @@ type mockAgentServer struct { } } -// StartMockAgentServer starts the mock agent server. -func StartMockAgentServer() (*mockAgentServer, error) { +// StartMockReceiverServer starts the mock receiver server. +func StartMockReceiverServer() (*mockReceiverServer, error) { addr := "127.0.0.1:0" lis, err := net.Listen("tcp", addr) if err != nil { return nil, err } server := grpc.NewServer() - agentServer := &mockAgentServer{ + receiverServer := &mockReceiverServer{ addr: fmt.Sprintf("127.0.0.1:%d", lis.Addr().(*net.TCPAddr).Port), grpcServer: server, sqlMetas: make(map[string]tipb.SQLMeta, 5000), planMetas: make(map[string]string, 5000), } - agentServer.hang.beginTime.Store(time.Now()) - agentServer.hang.endTime.Store(time.Now()) - tipb.RegisterTopSQLAgentServer(server, agentServer) + receiverServer.hang.beginTime.Store(time.Now()) + receiverServer.hang.endTime.Store(time.Now()) + tipb.RegisterTopSQLAgentServer(server, receiverServer) go func() { err := server.Serve(lis) if err != nil { - logutil.BgLogger().Warn("[top-sql] mock agent server serve failed", zap.Error(err)) + logutil.BgLogger().Warn("[top-sql] mock receiver server serve failed", zap.Error(err)) } }() - return agentServer, nil + return receiverServer, nil } -func (svr *mockAgentServer) HangFromNow(duration time.Duration) { +func (svr *mockReceiverServer) HangFromNow(duration time.Duration) { now := time.Now() svr.hang.beginTime.Store(now) svr.hang.endTime.Store(now.Add(duration)) } // mayHang will check the hanging period, and ensure to sleep through it -func (svr *mockAgentServer) mayHang() { +func (svr *mockReceiverServer) mayHang() { now := time.Now() beginTime := svr.hang.beginTime.Load().(time.Time) endTime := svr.hang.endTime.Load().(time.Time) @@ -85,7 +85,7 @@ func (svr *mockAgentServer) mayHang() { } } -func (svr *mockAgentServer) ReportCPUTimeRecords(stream tipb.TopSQLAgent_ReportCPUTimeRecordsServer) error { +func (svr *mockReceiverServer) ReportCPUTimeRecords(stream tipb.TopSQLAgent_ReportCPUTimeRecordsServer) error { records := make([]*tipb.CPUTimeRecord, 0, 10) for { svr.mayHang() @@ -103,7 +103,7 @@ func (svr *mockAgentServer) ReportCPUTimeRecords(stream tipb.TopSQLAgent_ReportC return stream.SendAndClose(&tipb.EmptyResponse{}) } -func (svr *mockAgentServer) ReportSQLMeta(stream tipb.TopSQLAgent_ReportSQLMetaServer) error { +func (svr *mockReceiverServer) ReportSQLMeta(stream tipb.TopSQLAgent_ReportSQLMetaServer) error { for { svr.mayHang() req, err := stream.Recv() @@ -119,7 +119,7 @@ func (svr *mockAgentServer) ReportSQLMeta(stream tipb.TopSQLAgent_ReportSQLMetaS return stream.SendAndClose(&tipb.EmptyResponse{}) } -func (svr *mockAgentServer) ReportPlanMeta(stream tipb.TopSQLAgent_ReportPlanMetaServer) error { +func (svr *mockReceiverServer) ReportPlanMeta(stream tipb.TopSQLAgent_ReportPlanMetaServer) error { for { svr.mayHang() req, err := stream.Recv() @@ -135,7 +135,7 @@ func (svr *mockAgentServer) ReportPlanMeta(stream tipb.TopSQLAgent_ReportPlanMet return stream.SendAndClose(&tipb.EmptyResponse{}) } -func (svr *mockAgentServer) WaitCollectCnt(cnt int, timeout time.Duration) { +func (svr *mockReceiverServer) WaitCollectCnt(cnt int, timeout time.Duration) { start := time.Now() svr.Lock() old := len(svr.records) @@ -154,7 +154,7 @@ func (svr *mockAgentServer) WaitCollectCnt(cnt int, timeout time.Duration) { } } -func (svr *mockAgentServer) GetSQLMetaByDigestBlocking(digest []byte, timeout time.Duration) (meta tipb.SQLMeta, exist bool) { +func (svr *mockReceiverServer) GetSQLMetaByDigestBlocking(digest []byte, timeout time.Duration) (meta tipb.SQLMeta, exist bool) { start := time.Now() for { svr.Lock() @@ -167,7 +167,7 @@ func (svr *mockAgentServer) GetSQLMetaByDigestBlocking(digest []byte, timeout ti } } -func (svr *mockAgentServer) GetPlanMetaByDigestBlocking(digest []byte, timeout time.Duration) (normalizedPlan string, exist bool) { +func (svr *mockReceiverServer) GetPlanMetaByDigestBlocking(digest []byte, timeout time.Duration) (normalizedPlan string, exist bool) { start := time.Now() for { svr.Lock() @@ -180,7 +180,7 @@ func (svr *mockAgentServer) GetPlanMetaByDigestBlocking(digest []byte, timeout t } } -func (svr *mockAgentServer) GetLatestRecords() []*tipb.CPUTimeRecord { +func (svr *mockReceiverServer) GetLatestRecords() []*tipb.CPUTimeRecord { svr.Lock() records := svr.records svr.records = [][]*tipb.CPUTimeRecord{} @@ -192,11 +192,11 @@ func (svr *mockAgentServer) GetLatestRecords() []*tipb.CPUTimeRecord { return records[len(records)-1] } -func (svr *mockAgentServer) Address() string { +func (svr *mockReceiverServer) Address() string { return svr.addr } -func (svr *mockAgentServer) Stop() { +func (svr *mockReceiverServer) Stop() { if svr.grpcServer != nil { svr.grpcServer.Stop() } diff --git a/util/topsql/reporter/reporter.go b/util/topsql/reporter/reporter.go index 5c280d9072081..be671990a6d09 100644 --- a/util/topsql/reporter/reporter.go +++ b/util/topsql/reporter/reporter.go @@ -245,6 +245,7 @@ func (tsr *RemoteTopSQLReporter) Close() { for i := range tsr.clients { tsr.clients[i].Close() } + tsr.clients = nil } func addEvictedCPUTime(collectTarget map[string]*dataPoints, timestamp uint64, totalCPUTimeMs uint32) { diff --git a/util/topsql/reporter/reporter_test.go b/util/topsql/reporter/reporter_test.go index 6eece4fbe6ca2..ff524ebdd0ef2 100644 --- a/util/topsql/reporter/reporter_test.go +++ b/util/topsql/reporter/reporter_test.go @@ -84,7 +84,7 @@ func initializeCache(maxStatementsNum, interval int, addr string) *RemoteTopSQLR } func TestCollectAndSendBatch(t *testing.T) { - agentServer, err := mock.StartMockAgentServer() + agentServer, err := mock.StartMockReceiverServer() require.NoError(t, err) defer agentServer.Stop() @@ -123,7 +123,7 @@ func TestCollectAndSendBatch(t *testing.T) { } func TestCollectAndEvicted(t *testing.T) { - agentServer, err := mock.StartMockAgentServer() + agentServer, err := mock.StartMockReceiverServer() require.NoError(t, err) defer agentServer.Stop() @@ -188,7 +188,7 @@ func collectAndWait(tsr *RemoteTopSQLReporter, timestamp uint64, records []trace } func TestCollectAndTopN(t *testing.T) { - agentServer, err := mock.StartMockAgentServer() + agentServer, err := mock.StartMockReceiverServer() require.NoError(t, err) defer agentServer.Stop() @@ -390,7 +390,7 @@ func TestDataPoints(t *testing.T) { } func TestCollectInternal(t *testing.T) { - agentServer, err := mock.StartMockAgentServer() + agentServer, err := mock.StartMockReceiverServer() require.NoError(t, err) defer agentServer.Stop() diff --git a/util/topsql/reporter/subscriber.go b/util/topsql/reporter/subscriber.go index a4e44df852211..2e77db38277bf 100644 --- a/util/topsql/reporter/subscriber.go +++ b/util/topsql/reporter/subscriber.go @@ -24,6 +24,10 @@ import ( "go.uber.org/zap" ) +// TopSQLPublisher implements TopSQLPublisher. +// +// If a client subscribes to TopSQL records, the TopSQLPublisher is responsible for registering them to the reporter. +// Then the reporter sends data to the client periodically. type TopSQLPublisher struct { decodePlan planBinaryDecodeFunc clientRegistry *ReportClientRegistry @@ -41,6 +45,8 @@ func NewTopSQLPublisher( var _ tipb.TopSQLPubSubServer = &TopSQLPublisher{} +// Subscribe registers clients to the reporter and redirects data received from reporter +// to subscribers associated with those clients. func (t *TopSQLPublisher) Subscribe( _ *tipb.TopSQLSubRequest, stream tipb.TopSQLPubSub_SubscribeServer, @@ -146,6 +152,10 @@ func (s *subClient) run(wg *sync.WaitGroup) { var _ ReportClient = &subClient{} func (s *subClient) Send(_ context.Context, data reportData) error { + if s.IsDown() { + return nil + } + select { case s.dataCh <- data: // sent successfully diff --git a/util/topsql/topsql_test.go b/util/topsql/topsql_test.go index 877f87feabbcb..26369d3caf503 100644 --- a/util/topsql/topsql_test.go +++ b/util/topsql/topsql_test.go @@ -17,6 +17,8 @@ package topsql_test import ( "bytes" "context" + "github.com/pingcap/tipb/go-tipb" + "google.golang.org/grpc" "testing" "time" @@ -107,7 +109,7 @@ func mockPlanBinaryDecoderFunc(plan string) (string, error) { } func TestTopSQLReporter(t *testing.T) { - server, err := mockServer.StartMockAgentServer() + server, err := mockServer.StartMockReceiverServer() require.NoError(t, err) variable.TopSQLVariable.MaxStatementCount.Store(200) variable.TopSQLVariable.ReportIntervalSeconds.Store(1) @@ -177,6 +179,119 @@ func TestTopSQLReporter(t *testing.T) { require.Equal(t, 2, len(checkSQLPlanMap)) } +func TestTopSQLPubSub(t *testing.T) { + variable.TopSQLVariable.MaxStatementCount.Store(200) + variable.TopSQLVariable.ReportIntervalSeconds.Store(1) + + cr := reporter.NewReportClientRegistry() + report := reporter.NewRemoteTopSQLReporter(cr) + defer report.Close() + + server, err := mockServer.StartMockPublisherServer(mockPlanBinaryDecoderFunc, cr) + require.NoError(t, err) + defer server.Stop() + + tracecpu.GlobalSQLCPUProfiler.SetCollector(&collectorWrapper{report}) + conn, err := grpc.Dial(server.Address(), grpc.WithBlock(), grpc.WithInsecure()) + require.NoError(t, err) + defer conn.Close() + + ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) + defer cancel() + client := tipb.NewTopSQLPubSubClient(conn) + stream, err := client.Subscribe(ctx, &tipb.TopSQLSubRequest{}) + require.NoError(t, err) + + reqs := []struct { + sql string + plan string + }{ + {"select * from t where a=?", "point-get"}, + {"select * from t where a>?", "table-scan"}, + {"insert into t values (?)", ""}, + } + + closeCh := make(chan struct{}) + defer close(closeCh) + + digest2sql := make(map[string]string) + sql2plan := make(map[string]string) + for _, req := range reqs { + sql2plan[req.sql] = req.plan + sqlDigest := mock.GenSQLDigest(req.sql) + digest2sql[string(sqlDigest.Bytes())] = req.sql + + go func(sql, plan string) { + for { + select { + case <-closeCh: + return + default: + mockExecuteSQL(sql, plan) + } + } + }(req.sql, req.plan) + } + + sqlMetas := make(map[string]*tipb.SQLMeta) + planMetas := make(map[string]string) + records := make(map[string]*tipb.CPUTimeRecord) + + for { + r, err := stream.Recv() + if err != nil { + break + } + + if r.GetRecord() != nil { + rec := r.GetRecord() + if _, ok := records[string(rec.SqlDigest)]; !ok { + records[string(rec.SqlDigest)] = rec + } else { + cpu := records[string(rec.SqlDigest)] + if rec.PlanDigest != nil { + cpu.PlanDigest = rec.PlanDigest + } + cpu.RecordListTimestampSec = append(cpu.RecordListTimestampSec, rec.RecordListTimestampSec...) + cpu.RecordListCpuTimeMs = append(cpu.RecordListCpuTimeMs, rec.RecordListCpuTimeMs...) + } + } else if r.GetSqlMeta() != nil { + sql := r.GetSqlMeta() + if _, ok := sqlMetas[string(sql.SqlDigest)]; !ok { + sqlMetas[string(sql.SqlDigest)] = sql + } + } else if r.GetPlanMeta() != nil { + plan := r.GetPlanMeta() + if _, ok := planMetas[string(plan.PlanDigest)]; !ok { + planMetas[string(plan.PlanDigest)] = plan.NormalizedPlan + } + } + } + + 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)) + sqlMeta, exist := sqlMetas[string(record.SqlDigest)] + require.True(t, exist) + expectedNormalizedSQL, exist := digest2sql[string(record.SqlDigest)] + require.True(t, exist) + require.Equal(t, expectedNormalizedSQL, sqlMeta.NormalizedSql) + + expectedNormalizedPlan := sql2plan[expectedNormalizedSQL] + if expectedNormalizedPlan == "" || len(record.PlanDigest) == 0 { + require.Equal(t, len(record.PlanDigest), 0) + continue + } + normalizedPlan, exist := planMetas[string(record.PlanDigest)] + require.True(t, exist) + require.Equal(t, expectedNormalizedPlan, normalizedPlan) + checkSQLPlanMap[expectedNormalizedSQL] = struct{}{} + } + require.Equal(t, len(checkSQLPlanMap), 2) +} + func TestMaxSQLAndPlanTest(t *testing.T) { collector := mock.NewTopSQLCollector() tracecpu.GlobalSQLCPUProfiler.SetCollector(&collectorWrapper{collector}) From 8f2eae6cc8a00d7cc15503de1f2eb09891958a9b Mon Sep 17 00:00:00 2001 From: Zhenchi Date: Mon, 25 Oct 2021 20:49:55 +0800 Subject: [PATCH 07/47] fix tests Signed-off-by: Zhenchi --- util/topsql/main_test.go | 1 - util/topsql/topsql_test.go | 3 +++ util/topsql/tracecpu/profile.go | 2 +- 3 files changed, 4 insertions(+), 2 deletions(-) diff --git a/util/topsql/main_test.go b/util/topsql/main_test.go index 60254b0729401..4e156ba4b8d36 100644 --- a/util/topsql/main_test.go +++ b/util/topsql/main_test.go @@ -28,7 +28,6 @@ func TestMain(m *testing.M) { testbridge.WorkaroundGoCheckFlags() // set up - variable.TopSQLVariable.InstanceEnable.Store(true) config.UpdateGlobal(func(conf *config.Config) { conf.TopSQL.ReceiverAddress = "mock" }) diff --git a/util/topsql/topsql_test.go b/util/topsql/topsql_test.go index 26369d3caf503..e4ae254971473 100644 --- a/util/topsql/topsql_test.go +++ b/util/topsql/topsql_test.go @@ -39,6 +39,9 @@ type collectorWrapper struct { } func TestTopSQLCPUProfile(t *testing.T) { + variable.TopSQLVariable.InstanceEnable.Store(true) + defer variable.TopSQLVariable.InstanceEnable.Store(false) + collector := mock.NewTopSQLCollector() tracecpu.GlobalSQLCPUProfiler.SetCollector(&collectorWrapper{collector}) reqs := []struct { diff --git a/util/topsql/tracecpu/profile.go b/util/topsql/tracecpu/profile.go index 036e84428c21a..78c1591ebc821 100644 --- a/util/topsql/tracecpu/profile.go +++ b/util/topsql/tracecpu/profile.go @@ -285,7 +285,7 @@ func (sp *sqlCPUProfiler) IsEnabled() bool { // Because the GlobalSQLCPUProfiler keep calling pprof.StartCPUProfile to fetch SQL cpu stats, other place (such pprof profile HTTP API handler) call pprof.StartCPUProfile will be failed, // other place should call tracecpu.StartCPUProfile instead of pprof.StartCPUProfile. func StartCPUProfile(w io.Writer) error { - if variable.TopSQLInstanceEnabled() { + if GlobalSQLCPUProfiler.IsEnabled() { return GlobalSQLCPUProfiler.startExportCPUProfile(w) } return pprof.StartCPUProfile(w) From b75478efbfd03df3333f0aa4ce1d8a6099d94df6 Mon Sep 17 00:00:00 2001 From: Zhenchi Date: Mon, 25 Oct 2021 20:56:49 +0800 Subject: [PATCH 08/47] fix tests Signed-off-by: Zhenchi --- server/tidb_test.go | 2 ++ util/topsql/reporter/mock/publisher.go | 2 +- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/server/tidb_test.go b/server/tidb_test.go index b344961fe3165..e7d938b8d006b 100644 --- a/server/tidb_test.go +++ b/server/tidb_test.go @@ -1620,6 +1620,8 @@ func (ts *tidbTestTopSQLSuite) TestTopSQLCPUProfile(c *C) { c.Assert(err, IsNil) }() + variable.TopSQLVariable.InstanceEnable.Store(true) + defer variable.TopSQLVariable.InstanceEnable.Store(false) collector := mockTopSQLTraceCPU.NewTopSQLCollector() tracecpu.GlobalSQLCPUProfiler.SetCollector(&collectorWrapper{collector}) diff --git a/util/topsql/reporter/mock/publisher.go b/util/topsql/reporter/mock/publisher.go index 82844fd39c54d..695ffed6a9544 100644 --- a/util/topsql/reporter/mock/publisher.go +++ b/util/topsql/reporter/mock/publisher.go @@ -26,7 +26,7 @@ import ( ) type mockPublisherServer struct { - addr string + addr string grpcServer *grpc.Server } From 30201ec6400e4e612d12309c05e58b731481a8e0 Mon Sep 17 00:00:00 2001 From: Zhenchi Date: Tue, 26 Oct 2021 01:15:48 +0800 Subject: [PATCH 09/47] fix Signed-off-by: Zhenchi --- util/topsql/reporter/client.go | 6 +++++- util/topsql/reporter/mock/publisher.go | 8 ++------ util/topsql/reporter/subscriber.go | 1 + util/topsql/topsql.go | 3 ++- util/topsql/topsql_test.go | 7 ++++--- 5 files changed, 14 insertions(+), 11 deletions(-) diff --git a/util/topsql/reporter/client.go b/util/topsql/reporter/client.go index 06e45ee13ad51..10159daaabf90 100644 --- a/util/topsql/reporter/client.go +++ b/util/topsql/reporter/client.go @@ -28,7 +28,7 @@ import ( "google.golang.org/grpc/backoff" ) -// ReportClient send data to the target server. +// ReportClient sends data to the target server. type ReportClient interface { Send(ctx context.Context, data reportData) error @@ -42,11 +42,13 @@ type ReportClient interface { Close() } +// ReportClientRegistry is used to receive ReportClient registrations. type ReportClientRegistry struct { sync.Mutex newClients []ReportClient } +// NewReportClientRegistry creates a new ReportClientRegistry. func NewReportClientRegistry() *ReportClientRegistry { return &ReportClientRegistry{} } @@ -124,10 +126,12 @@ func (r *GRPCReportClient) Send(ctx context.Context, data reportData) error { return nil } +// IsPending implements ReportClient interface. func (r *GRPCReportClient) IsPending() bool { return len(config.GetGlobalConfig().TopSQL.ReceiverAddress) == 0 } +// IsDown implements ReportClient interface. func (r *GRPCReportClient) IsDown() bool { return false } diff --git a/util/topsql/reporter/mock/publisher.go b/util/topsql/reporter/mock/publisher.go index 695ffed6a9544..ebe92eeb98ac1 100644 --- a/util/topsql/reporter/mock/publisher.go +++ b/util/topsql/reporter/mock/publisher.go @@ -19,7 +19,6 @@ import ( "net" "github.com/pingcap/tidb/util/logutil" - "github.com/pingcap/tidb/util/topsql/reporter" "github.com/pingcap/tipb/go-tipb" "go.uber.org/zap" "google.golang.org/grpc" @@ -32,8 +31,7 @@ type mockPublisherServer struct { // StartMockPublisherServer starts the mock publisher server. func StartMockPublisherServer( - planBinaryDecodeFunc func(string) (string, error), - clientRegistry *reporter.ReportClientRegistry, + service tipb.TopSQLPubSubServer, ) (*mockPublisherServer, error) { addr := "127.0.0.1:0" lis, err := net.Listen("tcp", addr) @@ -41,9 +39,7 @@ func StartMockPublisherServer( return nil, err } server := grpc.NewServer() - - publisherServer := reporter.NewTopSQLPublisher(planBinaryDecodeFunc, clientRegistry) - tipb.RegisterTopSQLPubSubServer(server, publisherServer) + tipb.RegisterTopSQLPubSubServer(server, service) go func() { err := server.Serve(lis) diff --git a/util/topsql/reporter/subscriber.go b/util/topsql/reporter/subscriber.go index 2e77db38277bf..c21aad052309b 100644 --- a/util/topsql/reporter/subscriber.go +++ b/util/topsql/reporter/subscriber.go @@ -33,6 +33,7 @@ type TopSQLPublisher struct { clientRegistry *ReportClientRegistry } +// NewTopSQLPublisher creates a new TopSQLPublisher. func NewTopSQLPublisher( decodePlan planBinaryDecodeFunc, clientRegistry *ReportClientRegistry, diff --git a/util/topsql/topsql.go b/util/topsql/topsql.go index 0127254b58b74..fda7ea41f4942 100644 --- a/util/topsql/topsql.go +++ b/util/topsql/topsql.go @@ -56,11 +56,12 @@ func SetupTopSQL() { publisher := reporter.NewTopSQLPublisher(plancodec.DecodeNormalizedPlan, cr) globalTopSQLPubSubService = publisher - if config.GetGlobalConfig().TopSQL.ReceiverAddress != "" { + if len(config.GetGlobalConfig().TopSQL.ReceiverAddress) != 0 { variable.TopSQLVariable.InstanceEnable.Store(true) } } +// RegisterPubSubServer registers TopSQLPubSubService to the given gRPC server. func RegisterPubSubServer(s *grpc.Server) { if globalTopSQLPubSubService != nil { tipb.RegisterTopSQLPubSubServer(s, globalTopSQLPubSubService) diff --git a/util/topsql/topsql_test.go b/util/topsql/topsql_test.go index e4ae254971473..b924bca1017d4 100644 --- a/util/topsql/topsql_test.go +++ b/util/topsql/topsql_test.go @@ -17,8 +17,6 @@ package topsql_test import ( "bytes" "context" - "github.com/pingcap/tipb/go-tipb" - "google.golang.org/grpc" "testing" "time" @@ -31,7 +29,9 @@ import ( mockServer "github.com/pingcap/tidb/util/topsql/reporter/mock" "github.com/pingcap/tidb/util/topsql/tracecpu" "github.com/pingcap/tidb/util/topsql/tracecpu/mock" + "github.com/pingcap/tipb/go-tipb" "github.com/stretchr/testify/require" + "google.golang.org/grpc" ) type collectorWrapper struct { @@ -190,7 +190,8 @@ func TestTopSQLPubSub(t *testing.T) { report := reporter.NewRemoteTopSQLReporter(cr) defer report.Close() - server, err := mockServer.StartMockPublisherServer(mockPlanBinaryDecoderFunc, cr) + publisherServer := reporter.NewTopSQLPublisher(mockPlanBinaryDecoderFunc, cr) + server, err := mockServer.StartMockPublisherServer(publisherServer) require.NoError(t, err) defer server.Stop() From 9c3f72300774674c3009d88f58e9b444b1188f15 Mon Sep 17 00:00:00 2001 From: Zhenchi Date: Tue, 26 Oct 2021 11:28:29 +0800 Subject: [PATCH 10/47] update tipb Signed-off-by: Zhenchi --- go.mod | 4 +--- go.sum | 4 ++-- 2 files changed, 3 insertions(+), 5 deletions(-) diff --git a/go.mod b/go.mod index e2873cf529cda..9bb0e3d4a19be 100644 --- a/go.mod +++ b/go.mod @@ -53,7 +53,7 @@ require ( github.com/pingcap/sysutil v0.0.0-20210730114356-fcd8a63f68c5 github.com/pingcap/tidb-tools v5.0.3+incompatible github.com/pingcap/tidb/parser v0.0.0-20211011031125-9b13dc409c5e - github.com/pingcap/tipb v0.0.0-20211008080435-3fd327dfce0e + github.com/pingcap/tipb v0.0.0-20211026032243-186050f116ed github.com/prometheus/client_golang v1.5.1 github.com/prometheus/client_model v0.2.0 github.com/prometheus/common v0.9.1 @@ -102,5 +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 - -replace github.com/pingcap/tipb => github.com/zhongzc/tipb v0.0.0-20211025093821-d1bfd2ef18e8 diff --git a/go.sum b/go.sum index 0fc66658f7647..648f0457a3479 100644 --- a/go.sum +++ b/go.sum @@ -613,6 +613,8 @@ github.com/pingcap/tidb-dashboard v0.0.0-20210312062513-eef5d6404638/go.mod h1:O github.com/pingcap/tidb-dashboard v0.0.0-20210716172320-2226872e3296/go.mod h1:OCXbZTBTIMRcIt0jFsuCakZP+goYRv6IjawKbwLS2TQ= github.com/pingcap/tidb-tools v5.0.3+incompatible h1:vYMrW9ux+3HRMeRZ1fUOjy2nyiodtuVyAyK270EKBEs= github.com/pingcap/tidb-tools v5.0.3+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM= +github.com/pingcap/tipb v0.0.0-20211026032243-186050f116ed h1:46eHok6tYe87/+RQCWRzbCnQFlL65K/Nv2Z98LhMl5Y= +github.com/pingcap/tipb v0.0.0-20211026032243-186050f116ed/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= @@ -793,8 +795,6 @@ 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/zhongzc/tipb v0.0.0-20211025093821-d1bfd2ef18e8 h1:THw6y9o24NejDF9/JaTMXhUaNLlgvfbJbvVABy89Y9s= -github.com/zhongzc/tipb v0.0.0-20211025093821-d1bfd2ef18e8/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= 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= From a6aa4f84261cb550af69388e0f31560e49513b5d Mon Sep 17 00:00:00 2001 From: Zhenchi Date: Tue, 26 Oct 2021 12:55:13 +0800 Subject: [PATCH 11/47] fix test Signed-off-by: Zhenchi --- executor/executor_test.go | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/executor/executor_test.go b/executor/executor_test.go index a4619e8e1a28b..6d3110e288377 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -8653,9 +8653,8 @@ func (s *testResourceTagSuite) TestResourceGroupTag(c *C) { tk.MustExec("create table t(a int, b int, unique index idx(a));") tbInfo := testGetTableByName(c, tk.Se, "test", "t") - config.UpdateGlobal(func(conf *config.Config) { - conf.TopSQL.ReceiverAddress = "mock-agent" - }) + variable.TopSQLVariable.InstanceEnable.Store(true) + defer variable.TopSQLVariable.InstanceEnable.Store(false) c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/mockstore/unistore/unistoreRPCClientSendHook", `return(true)`), IsNil) defer failpoint.Disable("github.com/pingcap/tidb/store/mockstore/unistore/unistoreRPCClientSendHook") From f6537e2965ec6f77f7326d97ac3ae6f330e9ccc6 Mon Sep 17 00:00:00 2001 From: Zhenchi Date: Tue, 26 Oct 2021 21:56:13 +0800 Subject: [PATCH 12/47] add metrics Signed-off-by: Zhenchi --- util/topsql/reporter/client.go | 71 +++++++-- util/topsql/reporter/reporter.go | 14 +- util/topsql/reporter/reporter_test.go | 4 + util/topsql/reporter/subscriber.go | 219 +++++++++++++++++++------- util/topsql/tracecpu/main_test.go | 5 +- 5 files changed, 224 insertions(+), 89 deletions(-) diff --git a/util/topsql/reporter/client.go b/util/topsql/reporter/client.go index 10159daaabf90..a7a25e81a5ed5 100644 --- a/util/topsql/reporter/client.go +++ b/util/topsql/reporter/client.go @@ -30,13 +30,13 @@ import ( // ReportClient sends data to the target server. type ReportClient interface { - Send(ctx context.Context, data reportData) error + Send(data reportData, timeout time.Duration) // IsPending indicates that ReportClient is not expecting to receive records for now and may resume in the future. IsPending() bool - // IsDown indicates that the client has been closed and can be cleared. - // Note that: once a ReportClient is closed, it cannot go back to be non-closed. + // IsDown indicates that the client has been down and can be cleared. + // Note that: once a ReportClient is down, it cannot go back to be up. IsDown() bool Close() @@ -74,30 +74,69 @@ func (r *ReportClientRegistry) register(client ReportClient) { type GRPCReportClient struct { curRPCAddr string conn *grpc.ClientConn + sendTask chan struct { + data reportData + timeout time.Duration + } // calling decodePlan this can take a while, so should not block critical paths decodePlan planBinaryDecodeFunc } // NewGRPCReportClient returns a new GRPCReportClient func NewGRPCReportClient(decodePlan planBinaryDecodeFunc) *GRPCReportClient { - return &GRPCReportClient{ + client := &GRPCReportClient{ decodePlan: decodePlan, + sendTask: make(chan struct { + data reportData + timeout time.Duration + }), + } + + go client.run() + return client +} + +func (r *GRPCReportClient) run() { + for task := range r.sendTask { + targetRPCAddr := config.GetGlobalConfig().TopSQL.ReceiverAddress + if targetRPCAddr == "" { + continue + } + + ctx, cancel := context.WithTimeout(context.Background(), task.timeout) + start := time.Now() + err := r.doSend(ctx, targetRPCAddr, task.data) + cancel() + if err != nil { + logutil.BgLogger().Warn("[top-sql] client failed to send data to receiver", zap.Error(err)) + reportAllDurationFailedHistogram.Observe(time.Since(start).Seconds()) + } else { + reportAllDurationSuccHistogram.Observe(time.Since(start).Seconds()) + } } } var _ ReportClient = &GRPCReportClient{} // Send implements the ReportClient interface. -// Currently the implementation will establish a new connection every time, which is suitable for a per-minute sending period -func (r *GRPCReportClient) Send(ctx context.Context, data reportData) error { - targetRPCAddr := config.GetGlobalConfig().TopSQL.ReceiverAddress - - if targetRPCAddr == "" { - return nil +func (r *GRPCReportClient) Send(data reportData, timeout time.Duration) { + select { + case r.sendTask <- struct { + data reportData + timeout time.Duration + }{data: data, timeout: timeout}: + // sent successfully + default: + ignoreReportChannelFullCounter.Inc() + logutil.BgLogger().Warn("[top-sql] report channel is full") } - err := r.tryEstablishConnection(ctx, targetRPCAddr) +} + +// Currently the doSend will establish a new connection every time, which is suitable for a per-minute sending period +func (r *GRPCReportClient) doSend(ctx context.Context, addr string, data reportData) (err error) { + err = r.tryEstablishConnection(ctx, addr) if err != nil { - return err + return } var wg sync.WaitGroup @@ -118,12 +157,13 @@ func (r *GRPCReportClient) Send(ctx context.Context, data reportData) error { }() wg.Wait() close(errCh) - for err := range errCh { + for err = range errCh { if err != nil { - return err + return } } - return nil + + return } // IsPending implements ReportClient interface. @@ -138,6 +178,7 @@ func (r *GRPCReportClient) IsDown() bool { // Close uses to close grpc connection. func (r *GRPCReportClient) Close() { + close(r.sendTask) if r.conn == nil { return } diff --git a/util/topsql/reporter/reporter.go b/util/topsql/reporter/reporter.go index be671990a6d09..d4cf15ead9a29 100644 --- a/util/topsql/reporter/reporter.go +++ b/util/topsql/reporter/reporter.go @@ -496,8 +496,6 @@ func (tsr *RemoteTopSQLReporter) takeDataAndSendToReportChan(collectedDataPtr *m select { case tsr.reportCollectedDataChan <- data: default: - // ignore if chan blocked - ignoreReportChannelFullCounter.Inc() } } @@ -607,16 +605,6 @@ func (tsr *RemoteTopSQLReporter) doReport(data reportData) { }) for i := range tsr.clients { - ctx, cancel := context.WithTimeout(tsr.ctx, timeout) - start := time.Now() - - err := tsr.clients[i].Send(ctx, data) - if err != nil { - logutil.BgLogger().Warn("[top-sql] client failed to send data", zap.Error(err)) - reportAllDurationFailedHistogram.Observe(time.Since(start).Seconds()) - } else { - reportAllDurationSuccHistogram.Observe(time.Since(start).Seconds()) - } - cancel() + tsr.clients[i].Send(data, timeout) } } diff --git a/util/topsql/reporter/reporter_test.go b/util/topsql/reporter/reporter_test.go index ff524ebdd0ef2..8de2e4bbe912e 100644 --- a/util/topsql/reporter/reporter_test.go +++ b/util/topsql/reporter/reporter_test.go @@ -426,6 +426,8 @@ func TestCollectInternal(t *testing.T) { func BenchmarkTopSQL_CollectAndIncrementFrequency(b *testing.B) { tsr := initializeCache(maxSQLNum, 120, ":23333") + defer tsr.Close() + for i := 0; i < b.N; i++ { populateCache(tsr, 0, maxSQLNum, uint64(i)) } @@ -433,6 +435,8 @@ func BenchmarkTopSQL_CollectAndIncrementFrequency(b *testing.B) { func BenchmarkTopSQL_CollectAndEvict(b *testing.B) { tsr := initializeCache(maxSQLNum, 120, ":23333") + defer tsr.Close() + begin := 0 end := maxSQLNum for i := 0; i < b.N; i++ { diff --git a/util/topsql/reporter/subscriber.go b/util/topsql/reporter/subscriber.go index c21aad052309b..a5f51b68626b1 100644 --- a/util/topsql/reporter/subscriber.go +++ b/util/topsql/reporter/subscriber.go @@ -17,6 +17,7 @@ package reporter import ( "context" "sync" + "time" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tipb/go-tipb" @@ -65,8 +66,11 @@ func (t *TopSQLPublisher) Subscribe( } type subClient struct { - stream tipb.TopSQLPubSub_SubscribeServer - dataCh chan reportData + stream tipb.TopSQLPubSub_SubscribeServer + sendTask chan struct { + data reportData + timeout time.Duration + } isDown *atomic.Bool decodePlan planBinaryDecodeFunc @@ -76,11 +80,14 @@ func newSubClient( stream tipb.TopSQLPubSub_SubscribeServer, decodePlan planBinaryDecodeFunc, ) *subClient { - dataCh := make(chan reportData) + sendTask := make(chan struct { + data reportData + timeout time.Duration + }) return &subClient{ - stream: stream, - dataCh: dataCh, - isDown: atomic.NewBool(false), + stream: stream, + sendTask: sendTask, + isDown: atomic.NewBool(false), decodePlan: decodePlan, } @@ -92,78 +99,172 @@ func (s *subClient) run(wg *sync.WaitGroup) { s.isDown.Store(true) }() - for data := range s.dataCh { - var err error - r := &tipb.TopSQLSubResponse{} - - record := &tipb.CPUTimeRecord{} - r.RespOneof = &tipb.TopSQLSubResponse_Record{Record: record} - for i := range data.collectedData { - point := data.collectedData[i] - record.SqlDigest = point.SQLDigest - record.PlanDigest = point.PlanDigest - record.RecordListCpuTimeMs = point.CPUTimeMsList - record.RecordListTimestampSec = point.TimestampList - if err = s.stream.Send(r); err != nil { - logutil.BgLogger().Warn("[top-sql] failed to send record to the subscriber", zap.Error(err)) - return - } + for task := range s.sendTask { + ctx, cancel := context.WithTimeout(context.Background(), task.timeout) + start := time.Now() + err := s.doSend(ctx, task.data) + cancel() + if err != nil { + logutil.BgLogger().Warn("[top-sql] client failed to send data to subscriber", zap.Error(err)) + reportAllDurationFailedHistogram.Observe(time.Since(start).Seconds()) + } else { + reportAllDurationSuccHistogram.Observe(time.Since(start).Seconds()) } + } +} - sqlMeta := &tipb.SQLMeta{} - r.RespOneof = &tipb.TopSQLSubResponse_SqlMeta{SqlMeta: sqlMeta} - data.normalizedSQLMap.Range(func(key, value interface{}) bool { - meta := value.(SQLMeta) - sqlMeta.SqlDigest = []byte(key.(string)) - sqlMeta.NormalizedSql = meta.normalizedSQL - sqlMeta.IsInternalSql = meta.isInternal - if err = s.stream.Send(r); err != nil { - return false - } - return true - }) +func (s *subClient) doSend(ctx context.Context, data reportData) error { + if err := s.sendCPUTime(ctx, data.collectedData); err != nil { + return err + } + if err := s.sendSQLMeta(ctx, data.normalizedSQLMap); err != nil { + return err + } + return s.sendPlanMeta(ctx, data.normalizedPlanMap) +} + +func (s *subClient) sendCPUTime(ctx context.Context, data []*dataPoints) (err error) { + start := time.Now() + sentCount := 0 + defer func() { + topSQLReportRecordCounterHistogram.Observe(float64(sentCount)) if err != nil { - logutil.BgLogger().Warn("[top-sql] failed to send SQL meta to the subscriber", zap.Error(err)) + reportRecordDurationFailedHistogram.Observe(time.Since(start).Seconds()) + } else { + reportRecordDurationSuccHistogram.Observe(time.Since(start).Seconds()) + } + }() + + r := &tipb.TopSQLSubResponse{} + record := &tipb.CPUTimeRecord{} + r.RespOneof = &tipb.TopSQLSubResponse_Record{Record: record} + + for i := range data { + point := data[i] + record.SqlDigest = point.SQLDigest + record.PlanDigest = point.PlanDigest + record.RecordListCpuTimeMs = point.CPUTimeMsList + record.RecordListTimestampSec = point.TimestampList + if err = s.stream.Send(r); err != nil { return } + sentCount += 1 - planMeta := &tipb.PlanMeta{} - r.RespOneof = &tipb.TopSQLSubResponse_PlanMeta{PlanMeta: planMeta} - data.normalizedPlanMap.Range(func(key, value interface{}) bool { - planDecoded, err1 := s.decodePlan(value.(string)) - if err1 != nil { - logutil.BgLogger().Warn("[top-sql] decode plan failed", zap.Error(err1)) - return true - } - - planMeta.PlanDigest = []byte(key.(string)) - planMeta.NormalizedPlan = planDecoded - if err = s.stream.Send(r); err != nil { - return false - } - return true - }) - if err != nil { - logutil.BgLogger().Warn("[top-sql] failed to send plan meta to the subscriber", zap.Error(err)) + select { + case <-ctx.Done(): + err = ctx.Err() return + default: + } + } + + return +} + +func (s *subClient) sendSQLMeta(ctx context.Context, sqlMetaMap *sync.Map) (err error) { + start := time.Now() + sentCount := 0 + defer func() { + topSQLReportSQLCountHistogram.Observe(float64(sentCount)) + if err != nil { + reportSQLDurationFailedHistogram.Observe(time.Since(start).Seconds()) + } else { + reportSQLDurationSuccHistogram.Observe(time.Since(start).Seconds()) + } + }() + + r := &tipb.TopSQLSubResponse{} + sqlMeta := &tipb.SQLMeta{} + r.RespOneof = &tipb.TopSQLSubResponse_SqlMeta{SqlMeta: sqlMeta} + + sqlMetaMap.Range(func(key, value interface{}) bool { + meta := value.(SQLMeta) + sqlMeta.SqlDigest = []byte(key.(string)) + sqlMeta.NormalizedSql = meta.normalizedSQL + sqlMeta.IsInternalSql = meta.isInternal + if err = s.stream.Send(r); err != nil { + return false + } + sentCount += 1 + + select { + case <-ctx.Done(): + err = ctx.Err() + return false + default: + } + + return true + }) + if err != nil { + return err + } + + return +} + +func (s *subClient) sendPlanMeta(ctx context.Context, planMetaMap *sync.Map) (err error) { + start := time.Now() + sentCount := 0 + defer func() { + topSQLReportPlanCountHistogram.Observe(float64(sentCount)) + if err != nil { + reportPlanDurationFailedHistogram.Observe(time.Since(start).Seconds()) + } else { + reportPlanDurationSuccHistogram.Observe(time.Since(start).Seconds()) + } + }() + + r := &tipb.TopSQLSubResponse{} + planMeta := &tipb.PlanMeta{} + r.RespOneof = &tipb.TopSQLSubResponse_PlanMeta{PlanMeta: planMeta} + planMetaMap.Range(func(key, value interface{}) bool { + planDecoded, err1 := s.decodePlan(value.(string)) + if err1 != nil { + logutil.BgLogger().Warn("[top-sql] decode plan failed", zap.Error(err1)) + return true + } + + planMeta.PlanDigest = []byte(key.(string)) + planMeta.NormalizedPlan = planDecoded + if err = s.stream.Send(r); err != nil { + return false + } + sentCount += 1 + + select { + case <-ctx.Done(): + err = ctx.Err() + return false + default: } + + return true + }) + if err != nil { + return err } + + return } var _ ReportClient = &subClient{} -func (s *subClient) Send(_ context.Context, data reportData) error { +func (s *subClient) Send(data reportData, timeout time.Duration) { if s.IsDown() { - return nil + return } select { - case s.dataCh <- data: + case s.sendTask <- struct { + data reportData + timeout time.Duration + }{data: data, timeout: timeout}: // sent successfully default: - logutil.BgLogger().Warn("[top-sql] data channel is full") + ignoreReportChannelFullCounter.Inc() + logutil.BgLogger().Warn("[top-sql] report channel is full") } - return nil } func (s *subClient) IsPending() bool { @@ -175,5 +276,5 @@ func (s *subClient) IsDown() bool { } func (s *subClient) Close() { - close(s.dataCh) + close(s.sendTask) } diff --git a/util/topsql/tracecpu/main_test.go b/util/topsql/tracecpu/main_test.go index 74352d78d7419..dfbe2aca633df 100644 --- a/util/topsql/tracecpu/main_test.go +++ b/util/topsql/tracecpu/main_test.go @@ -30,7 +30,7 @@ import ( func TestMain(m *testing.M) { testbridge.WorkaroundGoCheckFlags() - variable.TopSQLVariable.Enable.Store(false) + variable.TopSQLVariable.InstanceEnable.Store(false) config.UpdateGlobal(func(conf *config.Config) { conf.TopSQL.ReceiverAddress = "mock" }) @@ -52,9 +52,10 @@ func TestPProfCPUProfile(t *testing.T) { err := tracecpu.StartCPUProfile(buf) require.NoError(t, err) // enable top sql. - variable.TopSQLVariable.Enable.Store(true) + variable.TopSQLVariable.InstanceEnable.Store(true) err = tracecpu.StopCPUProfile() require.NoError(t, err) _, err = profile.Parse(buf) require.NoError(t, err) + variable.TopSQLVariable.InstanceEnable.Store(false) } From 24fb6cac02e72672543522ec4d2f29b8aef22d0f Mon Sep 17 00:00:00 2001 From: Zhenchi Date: Wed, 27 Oct 2021 12:34:37 +0800 Subject: [PATCH 13/47] remove topsql var case Signed-off-by: Zhenchi --- domain/sysvar_cache.go | 2 -- 1 file changed, 2 deletions(-) diff --git a/domain/sysvar_cache.go b/domain/sysvar_cache.go index 839853a571851..1d9d6e1744eea 100644 --- a/domain/sysvar_cache.go +++ b/domain/sysvar_cache.go @@ -181,8 +181,6 @@ func checkEnableServerGlobalVar(name, sVal string) { err = stmtsummary.StmtSummaryByDigestMap.SetMaxSQLLength(sVal, false) case variable.TiDBCapturePlanBaseline: variable.CapturePlanBaseline.Set(sVal, false) - case variable.TiDBEnableTopSQL: - // TODO: whether the topsql global variable is enabled or not doesn't affect instance enabled case variable.TiDBTopSQLPrecisionSeconds: var val int64 val, err = strconv.ParseInt(sVal, 10, 64) From 5abcea51a2262fa19d38d1b3defa134f76969830 Mon Sep 17 00:00:00 2001 From: Zhenchi Date: Wed, 27 Oct 2021 16:00:20 +0800 Subject: [PATCH 14/47] add profile metrics Signed-off-by: Zhenchi --- metrics/topsql.go | 6 ++++++ util/topsql/tracecpu/profile.go | 2 ++ 2 files changed, 8 insertions(+) diff --git a/metrics/topsql.go b/metrics/topsql.go index 7ee6e55ae3f71..f07b200c5a9e9 100644 --- a/metrics/topsql.go +++ b/metrics/topsql.go @@ -18,6 +18,12 @@ import "github.com/prometheus/client_golang/prometheus" // Top SQL metrics. var ( + TopSQLProfileCounter = prometheus.NewCounter(prometheus.CounterOpts{ + Namespace: "tidb", + Subsystem: "topsql", + Name: "profile_total", + Help: "Counter of profiling", + }) TopSQLIgnoredCounter = prometheus.NewCounterVec( prometheus.CounterOpts{ Namespace: "tidb", diff --git a/util/topsql/tracecpu/profile.go b/util/topsql/tracecpu/profile.go index 78c1591ebc821..6733afa4e7810 100644 --- a/util/topsql/tracecpu/profile.go +++ b/util/topsql/tracecpu/profile.go @@ -19,6 +19,7 @@ import ( "context" "errors" "fmt" + "github.com/pingcap/tidb/metrics" "io" "net/http" "runtime/pprof" @@ -118,6 +119,7 @@ func (sp *sqlCPUProfiler) startCPUProfileWorker() { } func (sp *sqlCPUProfiler) doCPUProfile() { + metrics.TopSQLProfileCounter.Inc() intervalSecond := variable.TopSQLVariable.PrecisionSeconds.Load() task := sp.newProfileTask() if err := pprof.StartCPUProfile(task.buf); err != nil { From 373197d0b5f05150e29dc1ac297c4d5c83854f81 Mon Sep 17 00:00:00 2001 From: Zhenchi Date: Wed, 27 Oct 2021 16:22:23 +0800 Subject: [PATCH 15/47] add profile metrics Signed-off-by: Zhenchi --- metrics/metrics.go | 1 + metrics/topsql.go | 14 ++++++++------ 2 files changed, 9 insertions(+), 6 deletions(-) diff --git a/metrics/metrics.go b/metrics/metrics.go index 772663f530575..4d51647e71934 100644 --- a/metrics/metrics.go +++ b/metrics/metrics.go @@ -154,6 +154,7 @@ func RegisterMetrics() { prometheus.MustRegister(SmallTxnWriteDuration) prometheus.MustRegister(TxnWriteThroughput) prometheus.MustRegister(LoadSysVarCacheCounter) + prometheus.MustRegister(TopSQLProfileCounter) prometheus.MustRegister(TopSQLIgnoredCounter) prometheus.MustRegister(TopSQLReportDurationHistogram) prometheus.MustRegister(TopSQLReportDataHistogram) diff --git a/metrics/topsql.go b/metrics/topsql.go index f07b200c5a9e9..e6816cd3f9243 100644 --- a/metrics/topsql.go +++ b/metrics/topsql.go @@ -18,12 +18,14 @@ import "github.com/prometheus/client_golang/prometheus" // Top SQL metrics. var ( - TopSQLProfileCounter = prometheus.NewCounter(prometheus.CounterOpts{ - Namespace: "tidb", - Subsystem: "topsql", - Name: "profile_total", - Help: "Counter of profiling", - }) + TopSQLProfileCounter = prometheus.NewCounter( + prometheus.CounterOpts{ + Namespace: "tidb", + Subsystem: "topsql", + Name: "profile_total", + Help: "Counter of profiling", + }) + TopSQLIgnoredCounter = prometheus.NewCounterVec( prometheus.CounterOpts{ Namespace: "tidb", From cb2fce3bb5f5fc0eeb2447367e9d7811aaf71b2e Mon Sep 17 00:00:00 2001 From: Zhenchi Date: Wed, 27 Oct 2021 20:21:39 +0800 Subject: [PATCH 16/47] fix unsubscribe Signed-off-by: Zhenchi --- util/topsql/reporter/subscriber.go | 1 + 1 file changed, 1 insertion(+) diff --git a/util/topsql/reporter/subscriber.go b/util/topsql/reporter/subscriber.go index a5f51b68626b1..00dd8680c07ac 100644 --- a/util/topsql/reporter/subscriber.go +++ b/util/topsql/reporter/subscriber.go @@ -107,6 +107,7 @@ func (s *subClient) run(wg *sync.WaitGroup) { if err != nil { logutil.BgLogger().Warn("[top-sql] client failed to send data to subscriber", zap.Error(err)) reportAllDurationFailedHistogram.Observe(time.Since(start).Seconds()) + return } else { reportAllDurationSuccHistogram.Observe(time.Since(start).Seconds()) } From 1b076d9e869c375c1c1f7886423d1ff1b2cdc2fa Mon Sep 17 00:00:00 2001 From: Zhenchi Date: Tue, 16 Nov 2021 12:08:29 +0800 Subject: [PATCH 17/47] fix timeout Signed-off-by: Zhenchi --- util/topsql/reporter/subscriber.go | 43 ++++++++++++++++++++++++------ 1 file changed, 35 insertions(+), 8 deletions(-) diff --git a/util/topsql/reporter/subscriber.go b/util/topsql/reporter/subscriber.go index 00dd8680c07ac..0e1c88f46d694 100644 --- a/util/topsql/reporter/subscriber.go +++ b/util/topsql/reporter/subscriber.go @@ -100,16 +100,43 @@ func (s *subClient) run(wg *sync.WaitGroup) { }() for task := range s.sendTask { - ctx, cancel := context.WithTimeout(context.Background(), task.timeout) + ctx, cancel := context.WithCancel(context.Background()) start := time.Now() - err := s.doSend(ctx, task.data) - cancel() - if err != nil { - logutil.BgLogger().Warn("[top-sql] client failed to send data to subscriber", zap.Error(err)) - reportAllDurationFailedHistogram.Observe(time.Since(start).Seconds()) + + var err error + doneCh := make(chan struct{}) + go func(task struct { + data reportData + timeout time.Duration + }) { + defer func() { + doneCh <- struct{}{} + }() + err = s.doSend(ctx, task.data) + if err != nil { + reportAllDurationFailedHistogram.Observe(time.Since(start).Seconds()) + } else { + reportAllDurationSuccHistogram.Observe(time.Since(start).Seconds()) + } + }(task) + + select { + case <-doneCh: + cancel() + if err != nil { + logutil.BgLogger().Warn( + "[top-sql] client failed to send data to subscriber", + zap.Error(err), + ) + return + } + case <-time.After(task.timeout): + cancel() + logutil.BgLogger().Warn( + "[top-sql] client failed to send data to subscriber due to timeout", + zap.Duration("timeout", task.timeout), + ) return - } else { - reportAllDurationSuccHistogram.Observe(time.Since(start).Seconds()) } } } From 2437176a74c562f7ae00cf721cdf6c21bb24871e Mon Sep 17 00:00:00 2001 From: Zhenchi Date: Wed, 1 Dec 2021 16:46:14 +0800 Subject: [PATCH 18/47] address comment Signed-off-by: Zhenchi --- util/topsql/reporter/client.go | 3 ++- util/topsql/reporter/subscriber.go | 32 +++++++++++++++++------------- util/topsql/tracecpu/profile.go | 2 +- 3 files changed, 21 insertions(+), 16 deletions(-) diff --git a/util/topsql/reporter/client.go b/util/topsql/reporter/client.go index a7a25e81a5ed5..c4554bfebd653 100644 --- a/util/topsql/reporter/client.go +++ b/util/topsql/reporter/client.go @@ -16,6 +16,7 @@ package reporter import ( "context" + "github.com/pingcap/tidb/util" "math" "sync" "time" @@ -92,7 +93,7 @@ func NewGRPCReportClient(decodePlan planBinaryDecodeFunc) *GRPCReportClient { }), } - go client.run() + go util.WithRecovery(client.run, nil) return client } diff --git a/util/topsql/reporter/subscriber.go b/util/topsql/reporter/subscriber.go index 0e1c88f46d694..07757be9f7f6d 100644 --- a/util/topsql/reporter/subscriber.go +++ b/util/topsql/reporter/subscriber.go @@ -19,6 +19,7 @@ import ( "sync" "time" + "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tipb/go-tipb" "go.uber.org/atomic" @@ -100,24 +101,27 @@ func (s *subClient) run(wg *sync.WaitGroup) { }() for task := range s.sendTask { - ctx, cancel := context.WithCancel(context.Background()) + ctx, cancel := context.WithTimeout(context.Background(), task.timeout) start := time.Now() var err error doneCh := make(chan struct{}) - go func(task struct { - data reportData - timeout time.Duration - }) { - defer func() { - doneCh <- struct{}{} - }() - err = s.doSend(ctx, task.data) - if err != nil { - reportAllDurationFailedHistogram.Observe(time.Since(start).Seconds()) - } else { - reportAllDurationSuccHistogram.Observe(time.Since(start).Seconds()) - } + go func( + task struct { + data reportData + timeout time.Duration + }) { + util.WithRecovery(func() { + defer func() { + doneCh <- struct{}{} + }() + err = s.doSend(ctx, task.data) + if err != nil { + reportAllDurationFailedHistogram.Observe(time.Since(start).Seconds()) + } else { + reportAllDurationSuccHistogram.Observe(time.Since(start).Seconds()) + } + }, nil) }(task) select { diff --git a/util/topsql/tracecpu/profile.go b/util/topsql/tracecpu/profile.go index 6733afa4e7810..2d5d294d143d0 100644 --- a/util/topsql/tracecpu/profile.go +++ b/util/topsql/tracecpu/profile.go @@ -19,7 +19,6 @@ import ( "context" "errors" "fmt" - "github.com/pingcap/tidb/metrics" "io" "net/http" "runtime/pprof" @@ -29,6 +28,7 @@ import ( "time" "github.com/google/pprof/profile" + "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/hack" From 433c1f38992bd696b8935090aba6db307b5b16ad Mon Sep 17 00:00:00 2001 From: Zhenchi Date: Mon, 6 Dec 2021 12:18:13 +0800 Subject: [PATCH 19/47] tiny refactor Signed-off-by: Zhenchi --- server/tidb_test.go | 2 +- util/topsql/reporter/client.go | 26 +++++++---------- util/topsql/reporter/reporter.go | 2 ++ util/topsql/reporter/subscriber.go | 47 +++++++++--------------------- 4 files changed, 27 insertions(+), 50 deletions(-) diff --git a/server/tidb_test.go b/server/tidb_test.go index acc54a1b92e0a..e37d195c99886 100644 --- a/server/tidb_test.go +++ b/server/tidb_test.go @@ -27,7 +27,6 @@ import ( "database/sql" "encoding/pem" "fmt" - "github.com/pingcap/tidb/sessionctx/variable" "math/big" "net/http" "os" @@ -45,6 +44,7 @@ import ( "github.com/pingcap/tidb/parser" tmysql "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/session" + "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/util" diff --git a/util/topsql/reporter/client.go b/util/topsql/reporter/client.go index c4554bfebd653..f63f47eff3abf 100644 --- a/util/topsql/reporter/client.go +++ b/util/topsql/reporter/client.go @@ -16,12 +16,12 @@ package reporter import ( "context" - "github.com/pingcap/tidb/util" "math" "sync" "time" "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tipb/go-tipb" "go.uber.org/zap" @@ -75,22 +75,21 @@ func (r *ReportClientRegistry) register(client ReportClient) { type GRPCReportClient struct { curRPCAddr string conn *grpc.ClientConn - sendTask chan struct { - data reportData - timeout time.Duration - } + sendTaskCh chan sendTask // calling decodePlan this can take a while, so should not block critical paths decodePlan planBinaryDecodeFunc } +type sendTask struct { + data reportData + timeout time.Duration +} + // NewGRPCReportClient returns a new GRPCReportClient func NewGRPCReportClient(decodePlan planBinaryDecodeFunc) *GRPCReportClient { client := &GRPCReportClient{ decodePlan: decodePlan, - sendTask: make(chan struct { - data reportData - timeout time.Duration - }), + sendTaskCh: make(chan sendTask), } go util.WithRecovery(client.run, nil) @@ -98,7 +97,7 @@ func NewGRPCReportClient(decodePlan planBinaryDecodeFunc) *GRPCReportClient { } func (r *GRPCReportClient) run() { - for task := range r.sendTask { + for task := range r.sendTaskCh { targetRPCAddr := config.GetGlobalConfig().TopSQL.ReceiverAddress if targetRPCAddr == "" { continue @@ -122,10 +121,7 @@ var _ ReportClient = &GRPCReportClient{} // Send implements the ReportClient interface. func (r *GRPCReportClient) Send(data reportData, timeout time.Duration) { select { - case r.sendTask <- struct { - data reportData - timeout time.Duration - }{data: data, timeout: timeout}: + case r.sendTaskCh <- sendTask{data: data, timeout: timeout}: // sent successfully default: ignoreReportChannelFullCounter.Inc() @@ -179,7 +175,7 @@ func (r *GRPCReportClient) IsDown() bool { // Close uses to close grpc connection. func (r *GRPCReportClient) Close() { - close(r.sendTask) + close(r.sendTaskCh) if r.conn == nil { return } diff --git a/util/topsql/reporter/reporter.go b/util/topsql/reporter/reporter.go index d4cf15ead9a29..f01a7fc8ba388 100644 --- a/util/topsql/reporter/reporter.go +++ b/util/topsql/reporter/reporter.go @@ -496,6 +496,8 @@ func (tsr *RemoteTopSQLReporter) takeDataAndSendToReportChan(collectedDataPtr *m select { case tsr.reportCollectedDataChan <- data: default: + // ignore if chan blocked + ignoreReportChannelFullCounter.Inc() } } diff --git a/util/topsql/reporter/subscriber.go b/util/topsql/reporter/subscriber.go index 07757be9f7f6d..c43f0047499da 100644 --- a/util/topsql/reporter/subscriber.go +++ b/util/topsql/reporter/subscriber.go @@ -56,23 +56,16 @@ func (t *TopSQLPublisher) Subscribe( ) error { sc := newSubClient(stream, t.decodePlan) - var wg sync.WaitGroup - wg.Add(1) - go sc.run(&wg) - t.clientRegistry.register(sc) + sc.run() - wg.Wait() return nil } type subClient struct { - stream tipb.TopSQLPubSub_SubscribeServer - sendTask chan struct { - data reportData - timeout time.Duration - } - isDown *atomic.Bool + stream tipb.TopSQLPubSub_SubscribeServer + sendTaskCh chan sendTask + isDown *atomic.Bool decodePlan planBinaryDecodeFunc } @@ -81,36 +74,25 @@ func newSubClient( stream tipb.TopSQLPubSub_SubscribeServer, decodePlan planBinaryDecodeFunc, ) *subClient { - sendTask := make(chan struct { - data reportData - timeout time.Duration - }) return &subClient{ - stream: stream, - sendTask: sendTask, - isDown: atomic.NewBool(false), + stream: stream, + sendTaskCh: make(chan sendTask), + isDown: atomic.NewBool(false), decodePlan: decodePlan, } } -func (s *subClient) run(wg *sync.WaitGroup) { - defer func() { - wg.Done() - s.isDown.Store(true) - }() +func (s *subClient) run() { + defer s.isDown.Store(true) - for task := range s.sendTask { + for task := range s.sendTaskCh { ctx, cancel := context.WithTimeout(context.Background(), task.timeout) start := time.Now() var err error doneCh := make(chan struct{}) - go func( - task struct { - data reportData - timeout time.Duration - }) { + go func(task sendTask) { util.WithRecovery(func() { defer func() { doneCh <- struct{}{} @@ -288,10 +270,7 @@ func (s *subClient) Send(data reportData, timeout time.Duration) { } select { - case s.sendTask <- struct { - data reportData - timeout time.Duration - }{data: data, timeout: timeout}: + case s.sendTaskCh <- sendTask{data: data, timeout: timeout}: // sent successfully default: ignoreReportChannelFullCounter.Inc() @@ -308,5 +287,5 @@ func (s *subClient) IsDown() bool { } func (s *subClient) Close() { - close(s.sendTask) + close(s.sendTaskCh) } From b2afed8e0c3aae18c9b2e439805d725679d3e7b9 Mon Sep 17 00:00:00 2001 From: Zhenchi Date: Mon, 6 Dec 2021 12:30:07 +0800 Subject: [PATCH 20/47] address comment Signed-off-by: Zhenchi --- sessionctx/variable/sysvar.go | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index a69c1739b9504..a30e041f4b684 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -1801,12 +1801,7 @@ var defaultSysVars = []*SysVar{ return nil }}, // variable for top SQL feature. - {Scope: ScopeGlobal, Name: TiDBEnableTopSQL, Value: BoolToOnOff(DefTiDBTopSQLEnable), Type: TypeBool, Hidden: true, AllowEmpty: true, GetGlobal: func(s *SessionVars) (string, error) { - return Off, nil - }, SetGlobal: func(vars *SessionVars, s string) error { - // TODO: call pd to enable topsql globally. Not to affect the current instance directly. - return nil - }, GlobalConfigName: GlobalConfigEnableTopSQL}, + {Scope: ScopeGlobal, Name: TiDBEnableTopSQL, Value: BoolToOnOff(DefTiDBTopSQLEnable), Type: TypeBool, Hidden: true, AllowEmpty: true}, {Scope: ScopeGlobal, Name: TiDBTopSQLPrecisionSeconds, Value: strconv.Itoa(DefTiDBTopSQLPrecisionSeconds), Type: TypeInt, Hidden: true, MinValue: 1, MaxValue: math.MaxInt64, GetGlobal: func(s *SessionVars) (string, error) { return strconv.FormatInt(TopSQLVariable.PrecisionSeconds.Load(), 10), nil }, SetGlobal: func(vars *SessionVars, s string) error { From 4874b8172c33b43d50d8ad7eb24c80c18f42b148 Mon Sep 17 00:00:00 2001 From: Zhenchi Date: Mon, 6 Dec 2021 12:45:31 +0800 Subject: [PATCH 21/47] address comment Signed-off-by: Zhenchi --- util/topsql/reporter/reporter.go | 26 ++++++++++---------------- 1 file changed, 10 insertions(+), 16 deletions(-) diff --git a/util/topsql/reporter/reporter.go b/util/topsql/reporter/reporter.go index f01a7fc8ba388..3bda8d569dbe2 100644 --- a/util/topsql/reporter/reporter.go +++ b/util/topsql/reporter/reporter.go @@ -350,36 +350,30 @@ func (tsr *RemoteTopSQLReporter) collectWorker() { } } -// for simplify slice removal -var tmpSlice []ReportClient - func (tsr *RemoteTopSQLReporter) handleClientRegistry() { tsr.clientRegistry.visitAndClear(func(client ReportClient) { tsr.clients = append(tsr.clients, client) }) - for i := range tsr.clients { - client := tsr.clients[i] + idx := 0 + pendingCnt := 0 + for _, client := range tsr.clients { if client.IsDown() { client.Close() - } else { - tmpSlice = append(tmpSlice, client) + continue + } + if client.IsPending() { + pendingCnt += 1 } + tsr.clients[idx] = client + idx++ } - tsr.clients, tmpSlice = tmpSlice, tsr.clients - tmpSlice = tmpSlice[:0] + tsr.clients = tsr.clients[:idx] if len(tsr.clients) > 256 { logutil.BgLogger().Warn("[top-sql] too many clients", zap.Int("count", len(tsr.clients))) } - pendingCnt := 0 - for i := range tsr.clients { - client := tsr.clients[i] - if client.IsPending() { - pendingCnt += 1 - } - } runningCnt := len(tsr.clients) - pendingCnt variable.TopSQLVariable.InstanceEnable.Store(runningCnt > 0) } From 09a0f33919d8529e96547d97b7fb7d88fbf391cc Mon Sep 17 00:00:00 2001 From: Zhenchi Date: Mon, 6 Dec 2021 13:03:54 +0800 Subject: [PATCH 22/47] fix global var test Signed-off-by: Zhenchi --- domain/globalconfigsync/globalconfig_test.go | 11 ++++++----- sessionctx/variable/sysvar.go | 2 +- 2 files changed, 7 insertions(+), 6 deletions(-) diff --git a/domain/globalconfigsync/globalconfig_test.go b/domain/globalconfigsync/globalconfig_test.go index c7be9137ddf68..3f097189387aa 100644 --- a/domain/globalconfigsync/globalconfig_test.go +++ b/domain/globalconfigsync/globalconfig_test.go @@ -23,6 +23,7 @@ import ( "github.com/pingcap/tidb/domain/globalconfigsync" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/store/mockstore" + "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/util/testbridge" "github.com/stretchr/testify/require" "go.etcd.io/etcd/integration" @@ -67,11 +68,9 @@ func TestStoreGlobalConfig(t *testing.T) { defer cluster.Terminate(t) domain.GetGlobalConfigSyncer().SetEtcdClient(cluster.RandClient()) - se, err := session.CreateSession4Test(store) - require.NoError(t, err) - - _, err = se.Execute(context.Background(), "set @@global.tidb_enable_top_sql=1;") - require.NoError(t, err) + tk := testkit.NewTestKit(t, store) + tk.MustQuery("select @@global.tidb_enable_top_sql;").Check(testkit.Rows("0")) + tk.MustExec("set @@global.tidb_enable_top_sql=1;") for i := 0; i < 20; i++ { resp, err := cluster.RandClient().Get(context.Background(), "/global/config/enable_resource_metering") @@ -87,6 +86,8 @@ func TestStoreGlobalConfig(t *testing.T) { require.Equal(t, len(resp.Kvs), 1) require.Equal(t, resp.Kvs[0].Key, []byte("/global/config/enable_resource_metering")) require.Equal(t, resp.Kvs[0].Value, []byte("true")) + + tk.MustQuery("select @@global.tidb_enable_top_sql;").Check(testkit.Rows("1")) return } diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 42688d974cf1c..1b9ce7165ffaa 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -1801,7 +1801,7 @@ var defaultSysVars = []*SysVar{ return nil }}, // variable for top SQL feature. - {Scope: ScopeGlobal, Name: TiDBEnableTopSQL, Value: BoolToOnOff(DefTiDBTopSQLEnable), Type: TypeBool, Hidden: true, AllowEmpty: true}, + {Scope: ScopeGlobal, Name: TiDBEnableTopSQL, Value: BoolToOnOff(DefTiDBTopSQLEnable), Type: TypeBool, Hidden: true, AllowEmpty: true, GlobalConfigName: GlobalConfigEnableTopSQL}, {Scope: ScopeGlobal, Name: TiDBTopSQLPrecisionSeconds, Value: strconv.Itoa(DefTiDBTopSQLPrecisionSeconds), Type: TypeInt, Hidden: true, MinValue: 1, MaxValue: math.MaxInt64, GetGlobal: func(s *SessionVars) (string, error) { return strconv.FormatInt(TopSQLVariable.PrecisionSeconds.Load(), 10), nil }, SetGlobal: func(vars *SessionVars, s string) error { From cee41fad6d49deb65d3561f1033d6679edaf906b Mon Sep 17 00:00:00 2001 From: Zhenchi Date: Mon, 6 Dec 2021 17:40:56 +0800 Subject: [PATCH 23/47] address comments Signed-off-by: Zhenchi --- util/topsql/reporter/client.go | 34 +++++++++++++++--------------- util/topsql/reporter/reporter.go | 14 +++++++----- util/topsql/reporter/subscriber.go | 2 +- 3 files changed, 27 insertions(+), 23 deletions(-) diff --git a/util/topsql/reporter/client.go b/util/topsql/reporter/client.go index f63f47eff3abf..dda7ec99c8ef2 100644 --- a/util/topsql/reporter/client.go +++ b/util/topsql/reporter/client.go @@ -71,8 +71,8 @@ func (r *ReportClientRegistry) register(client ReportClient) { r.newClients = append(r.newClients, client) } -// GRPCReportClient reports data to grpc servers. -type GRPCReportClient struct { +// ConfigAssignedRemoteClient reports data to grpc servers. +type ConfigAssignedRemoteClient struct { curRPCAddr string conn *grpc.ClientConn sendTaskCh chan sendTask @@ -85,9 +85,9 @@ type sendTask struct { timeout time.Duration } -// NewGRPCReportClient returns a new GRPCReportClient -func NewGRPCReportClient(decodePlan planBinaryDecodeFunc) *GRPCReportClient { - client := &GRPCReportClient{ +// NewGRPCReportClient returns a new ConfigAssignedRemoteClient +func NewGRPCReportClient(decodePlan planBinaryDecodeFunc) *ConfigAssignedRemoteClient { + client := &ConfigAssignedRemoteClient{ decodePlan: decodePlan, sendTaskCh: make(chan sendTask), } @@ -96,7 +96,7 @@ func NewGRPCReportClient(decodePlan planBinaryDecodeFunc) *GRPCReportClient { return client } -func (r *GRPCReportClient) run() { +func (r *ConfigAssignedRemoteClient) run() { for task := range r.sendTaskCh { targetRPCAddr := config.GetGlobalConfig().TopSQL.ReceiverAddress if targetRPCAddr == "" { @@ -116,10 +116,10 @@ func (r *GRPCReportClient) run() { } } -var _ ReportClient = &GRPCReportClient{} +var _ ReportClient = &ConfigAssignedRemoteClient{} // Send implements the ReportClient interface. -func (r *GRPCReportClient) Send(data reportData, timeout time.Duration) { +func (r *ConfigAssignedRemoteClient) Send(data reportData, timeout time.Duration) { select { case r.sendTaskCh <- sendTask{data: data, timeout: timeout}: // sent successfully @@ -130,7 +130,7 @@ func (r *GRPCReportClient) Send(data reportData, timeout time.Duration) { } // Currently the doSend will establish a new connection every time, which is suitable for a per-minute sending period -func (r *GRPCReportClient) doSend(ctx context.Context, addr string, data reportData) (err error) { +func (r *ConfigAssignedRemoteClient) doSend(ctx context.Context, addr string, data reportData) (err error) { err = r.tryEstablishConnection(ctx, addr) if err != nil { return @@ -164,17 +164,17 @@ func (r *GRPCReportClient) doSend(ctx context.Context, addr string, data reportD } // IsPending implements ReportClient interface. -func (r *GRPCReportClient) IsPending() bool { +func (r *ConfigAssignedRemoteClient) IsPending() bool { return len(config.GetGlobalConfig().TopSQL.ReceiverAddress) == 0 } // IsDown implements ReportClient interface. -func (r *GRPCReportClient) IsDown() bool { +func (r *ConfigAssignedRemoteClient) IsDown() bool { return false } // Close uses to close grpc connection. -func (r *GRPCReportClient) Close() { +func (r *ConfigAssignedRemoteClient) Close() { close(r.sendTaskCh) if r.conn == nil { return @@ -187,7 +187,7 @@ func (r *GRPCReportClient) Close() { } // sendBatchCPUTimeRecord sends a batch of TopSQL records by stream. -func (r *GRPCReportClient) sendBatchCPUTimeRecord(ctx context.Context, records []*dataPoints) error { +func (r *ConfigAssignedRemoteClient) sendBatchCPUTimeRecord(ctx context.Context, records []*dataPoints) error { if len(records) == 0 { return nil } @@ -220,7 +220,7 @@ func (r *GRPCReportClient) sendBatchCPUTimeRecord(ctx context.Context, records [ } // sendBatchSQLMeta sends a batch of SQL metas by stream. -func (r *GRPCReportClient) sendBatchSQLMeta(ctx context.Context, sqlMap *sync.Map) error { +func (r *ConfigAssignedRemoteClient) sendBatchSQLMeta(ctx context.Context, sqlMap *sync.Map) error { start := time.Now() client := tipb.NewTopSQLAgentClient(r.conn) stream, err := client.ReportSQLMeta(ctx) @@ -256,7 +256,7 @@ func (r *GRPCReportClient) sendBatchSQLMeta(ctx context.Context, sqlMap *sync.Ma } // sendBatchPlanMeta sends a batch of SQL metas by stream. -func (r *GRPCReportClient) sendBatchPlanMeta(ctx context.Context, planMap *sync.Map) error { +func (r *ConfigAssignedRemoteClient) sendBatchPlanMeta(ctx context.Context, planMap *sync.Map) error { start := time.Now() client := tipb.NewTopSQLAgentClient(r.conn) stream, err := client.ReportPlanMeta(ctx) @@ -295,7 +295,7 @@ func (r *GRPCReportClient) sendBatchPlanMeta(ctx context.Context, planMap *sync. } // tryEstablishConnection establishes the gRPC connection if connection is not established. -func (r *GRPCReportClient) tryEstablishConnection(ctx context.Context, targetRPCAddr string) (err error) { +func (r *ConfigAssignedRemoteClient) tryEstablishConnection(ctx context.Context, targetRPCAddr string) (err error) { if r.curRPCAddr == targetRPCAddr && r.conn != nil { // Address is not changed, skip. return nil @@ -314,7 +314,7 @@ func (r *GRPCReportClient) tryEstablishConnection(ctx context.Context, targetRPC return nil } -func (r *GRPCReportClient) dial(ctx context.Context, targetRPCAddr string) (*grpc.ClientConn, error) { +func (r *ConfigAssignedRemoteClient) dial(ctx context.Context, targetRPCAddr string) (*grpc.ClientConn, error) { dialCtx, cancel := context.WithTimeout(ctx, dialTimeout) defer cancel() return grpc.DialContext( diff --git a/util/topsql/reporter/reporter.go b/util/topsql/reporter/reporter.go index 3bda8d569dbe2..94c9a17bab85f 100644 --- a/util/topsql/reporter/reporter.go +++ b/util/topsql/reporter/reporter.go @@ -355,25 +355,29 @@ func (tsr *RemoteTopSQLReporter) handleClientRegistry() { tsr.clients = append(tsr.clients, client) }) + // Remove all down clients idx := 0 - pendingCnt := 0 for _, client := range tsr.clients { if client.IsDown() { client.Close() continue } - if client.IsPending() { - pendingCnt += 1 - } tsr.clients[idx] = client idx++ } tsr.clients = tsr.clients[:idx] if len(tsr.clients) > 256 { - logutil.BgLogger().Warn("[top-sql] too many clients", zap.Int("count", len(tsr.clients))) + logutil.BgLogger().Warn("[top-sql] too many clients, keep 10 first", zap.Int("count", len(tsr.clients))) + tsr.clients = tsr.clients[:10] } + pendingCnt := 0 + for _, client := range tsr.clients { + if client.IsPending() { + pendingCnt += 1 + } + } runningCnt := len(tsr.clients) - pendingCnt variable.TopSQLVariable.InstanceEnable.Store(runningCnt > 0) } diff --git a/util/topsql/reporter/subscriber.go b/util/topsql/reporter/subscriber.go index c43f0047499da..7e386b2c328ab 100644 --- a/util/topsql/reporter/subscriber.go +++ b/util/topsql/reporter/subscriber.go @@ -116,7 +116,7 @@ func (s *subClient) run() { ) return } - case <-time.After(task.timeout): + case <-ctx.Done(): cancel() logutil.BgLogger().Warn( "[top-sql] client failed to send data to subscriber due to timeout", From b1bb877cbe4c046b44dd08836c6879ca167a58e6 Mon Sep 17 00:00:00 2001 From: Zhenchi Date: Mon, 6 Dec 2021 17:57:31 +0800 Subject: [PATCH 24/47] add keepalive Signed-off-by: Zhenchi --- config/config.go | 26 +++++++++++++++----------- config/config.toml.example | 8 ++++++++ server/rpc_server.go | 7 ++++++- 3 files changed, 29 insertions(+), 12 deletions(-) diff --git a/config/config.go b/config/config.go index 53141fadd093a..4eafcc136e2fa 100644 --- a/config/config.go +++ b/config/config.go @@ -439,12 +439,14 @@ func (s *Security) ClusterSecurity() tikvcfg.Security { // Status is the status section of the config. type Status struct { - StatusHost string `toml:"status-host" json:"status-host"` - MetricsAddr string `toml:"metrics-addr" json:"metrics-addr"` - StatusPort uint `toml:"status-port" json:"status-port"` - MetricsInterval uint `toml:"metrics-interval" json:"metrics-interval"` - ReportStatus bool `toml:"report-status" json:"report-status"` - RecordQPSbyDB bool `toml:"record-db-qps" json:"record-db-qps"` + StatusHost string `toml:"status-host" json:"status-host"` + MetricsAddr string `toml:"metrics-addr" json:"metrics-addr"` + StatusPort uint `toml:"status-port" json:"status-port"` + MetricsInterval uint `toml:"metrics-interval" json:"metrics-interval"` + ReportStatus bool `toml:"report-status" json:"report-status"` + RecordQPSbyDB bool `toml:"record-db-qps" json:"record-db-qps"` + GrpcKeepAliveTime uint `toml:"grpc-keepalive-time" json:"grpc-keepalive-time"` + GrpcKeepAliveTimeout uint `toml:"grpc-keepalive-timeout" json:"grpc-keepalive-timeout"` } // Performance is the performance section of the config. @@ -658,11 +660,13 @@ var defaultConf = Config{ EnableSlowLog: *NewAtomicBool(logutil.DefaultTiDBEnableSlowLog), }, Status: Status{ - ReportStatus: true, - StatusHost: DefStatusHost, - StatusPort: DefStatusPort, - MetricsInterval: 15, - RecordQPSbyDB: false, + ReportStatus: true, + StatusHost: DefStatusHost, + StatusPort: DefStatusPort, + MetricsInterval: 15, + RecordQPSbyDB: false, + GrpcKeepAliveTime: 10, + GrpcKeepAliveTimeout: 3, }, Performance: Performance{ MaxMemory: 0, diff --git a/config/config.toml.example b/config/config.toml.example index 0e91f903748f8..fab225c9a3e53 100644 --- a/config/config.toml.example +++ b/config/config.toml.example @@ -238,6 +238,14 @@ metrics-interval = 15 # Record statements qps by database name if it is enabled. record-db-qps = false +# After a duration of this time in seconds if the server doesn't see any activity it pings +# the client to see if the transport is still alive. +grpc-keepalive-time = 10 + +# After having pinged for keepalive check, the server waits for a duration of Timeout in seconds +# and if no activity is seen even after that the connection is closed. +grpc-keepalive-timeout = 3 + [performance] # Max CPUs to use, 0 use number of CPUs in the machine. max-procs = 0 diff --git a/server/rpc_server.go b/server/rpc_server.go index ff17419a0b69e..9aa27c952b928 100644 --- a/server/rpc_server.go +++ b/server/rpc_server.go @@ -18,6 +18,7 @@ import ( "context" "fmt" "net" + "time" "github.com/pingcap/kvproto/pkg/coprocessor" "github.com/pingcap/kvproto/pkg/diagnosticspb" @@ -35,6 +36,7 @@ import ( "github.com/pingcap/tidb/util/topsql" "go.uber.org/zap" "google.golang.org/grpc" + "google.golang.org/grpc/keepalive" "google.golang.org/grpc/peer" ) @@ -47,7 +49,10 @@ func NewRPCServer(config *config.Config, dom *domain.Domain, sm util.SessionMana } }() - s := grpc.NewServer() + s := grpc.NewServer(grpc.KeepaliveParams(keepalive.ServerParameters{ + Time: time.Duration(config.Status.GrpcKeepAliveTime) * time.Second, + Timeout: time.Duration(config.Status.GrpcKeepAliveTimeout) * time.Second, + })) rpcSrv := &rpcServer{ DiagnosticsServer: sysutil.NewDiagnosticsServer(config.Log.File.Filename), dom: dom, From 13890a57bff2b1d07f181a173e5256111e3e9e14 Mon Sep 17 00:00:00 2001 From: Zhenchi Date: Thu, 9 Dec 2021 14:44:33 +0800 Subject: [PATCH 25/47] add more grpc configs Signed-off-by: Zhenchi --- config/config.go | 36 +++++++++++++++++++++--------------- config/config.toml.example | 11 ++++++++++- server/rpc_server.go | 13 +++++++++---- 3 files changed, 40 insertions(+), 20 deletions(-) diff --git a/config/config.go b/config/config.go index 4eafcc136e2fa..c0f61a914fd66 100644 --- a/config/config.go +++ b/config/config.go @@ -439,14 +439,17 @@ func (s *Security) ClusterSecurity() tikvcfg.Security { // Status is the status section of the config. type Status struct { - StatusHost string `toml:"status-host" json:"status-host"` - MetricsAddr string `toml:"metrics-addr" json:"metrics-addr"` - StatusPort uint `toml:"status-port" json:"status-port"` - MetricsInterval uint `toml:"metrics-interval" json:"metrics-interval"` - ReportStatus bool `toml:"report-status" json:"report-status"` - RecordQPSbyDB bool `toml:"record-db-qps" json:"record-db-qps"` - GrpcKeepAliveTime uint `toml:"grpc-keepalive-time" json:"grpc-keepalive-time"` - GrpcKeepAliveTimeout uint `toml:"grpc-keepalive-timeout" json:"grpc-keepalive-timeout"` + StatusHost string `toml:"status-host" json:"status-host"` + MetricsAddr string `toml:"metrics-addr" json:"metrics-addr"` + StatusPort uint `toml:"status-port" json:"status-port"` + MetricsInterval uint `toml:"metrics-interval" json:"metrics-interval"` + ReportStatus bool `toml:"report-status" json:"report-status"` + RecordQPSbyDB bool `toml:"record-db-qps" json:"record-db-qps"` + GRPCKeepAliveTime uint `toml:"grpc-keepalive-time" json:"grpc-keepalive-time"` + GRPCKeepAliveTimeout uint `toml:"grpc-keepalive-timeout" json:"grpc-keepalive-timeout"` + GRPCConcurrentStreams uint `toml:"grpc-concurrent-streams" json:"grpc-concurrent-streams"` + GRPCInitialWindowSize int `toml:"grpc-initial-window-size" json:"grpc-initial-window-size"` + GRPCMaxSendMsgSize int `toml:"grpc-max-send-msg-size" json:"grpc-max-send-msg-size"` } // Performance is the performance section of the config. @@ -660,13 +663,16 @@ var defaultConf = Config{ EnableSlowLog: *NewAtomicBool(logutil.DefaultTiDBEnableSlowLog), }, Status: Status{ - ReportStatus: true, - StatusHost: DefStatusHost, - StatusPort: DefStatusPort, - MetricsInterval: 15, - RecordQPSbyDB: false, - GrpcKeepAliveTime: 10, - GrpcKeepAliveTimeout: 3, + ReportStatus: true, + StatusHost: DefStatusHost, + StatusPort: DefStatusPort, + MetricsInterval: 15, + RecordQPSbyDB: false, + GRPCKeepAliveTime: 10, + GRPCKeepAliveTimeout: 3, + GRPCConcurrentStreams: 1024, + GRPCInitialWindowSize: 2 * 1024 * 1024, + GRPCMaxSendMsgSize: 10 * 1024 * 1024, }, Performance: Performance{ MaxMemory: 0, diff --git a/config/config.toml.example b/config/config.toml.example index fab225c9a3e53..66e7085084414 100644 --- a/config/config.toml.example +++ b/config/config.toml.example @@ -242,10 +242,19 @@ record-db-qps = false # the client to see if the transport is still alive. grpc-keepalive-time = 10 -# After having pinged for keepalive check, the server waits for a duration of Timeout in seconds +# After having pinged for keepalive check, the server waits for a duration of timeout in seconds # and if no activity is seen even after that the connection is closed. grpc-keepalive-timeout = 3 +## The number of max concurrent streams/requests on a client connection. +# grpc-concurrent-streams = 1024 + +## Sets window size for stream. The default value is 2MB. +# grpc-initial-window-size = 2097152 + +## Set maximum message length in bytes that gRPC can send. `-1` means unlimited. The default value is 10MB. +# grpc-max-send-msg-size = 10485760 + [performance] # Max CPUs to use, 0 use number of CPUs in the machine. max-procs = 0 diff --git a/server/rpc_server.go b/server/rpc_server.go index 9aa27c952b928..3b23539c0bac1 100644 --- a/server/rpc_server.go +++ b/server/rpc_server.go @@ -49,10 +49,15 @@ func NewRPCServer(config *config.Config, dom *domain.Domain, sm util.SessionMana } }() - s := grpc.NewServer(grpc.KeepaliveParams(keepalive.ServerParameters{ - Time: time.Duration(config.Status.GrpcKeepAliveTime) * time.Second, - Timeout: time.Duration(config.Status.GrpcKeepAliveTimeout) * time.Second, - })) + s := grpc.NewServer( + grpc.KeepaliveParams(keepalive.ServerParameters{ + Time: time.Duration(config.Status.GRPCKeepAliveTime) * time.Second, + Timeout: time.Duration(config.Status.GRPCKeepAliveTimeout) * time.Second, + }), + grpc.MaxConcurrentStreams(uint32(config.Status.GRPCConcurrentStreams)), + grpc.InitialWindowSize(int32(config.Status.GRPCInitialWindowSize)), + grpc.MaxSendMsgSize(config.Status.GRPCMaxSendMsgSize), + ) rpcSrv := &rpcServer{ DiagnosticsServer: sysutil.NewDiagnosticsServer(config.Log.File.Filename), dom: dom, From 1c58afdb6cd32e463251ddfdc14d641aa994af31 Mon Sep 17 00:00:00 2001 From: Zhenchi Date: Thu, 9 Dec 2021 16:25:33 +0800 Subject: [PATCH 26/47] client -> datasink Signed-off-by: Zhenchi --- server/tidb_test.go | 2 +- util/topsql/reporter/datasink.go | 34 +++++ util/topsql/reporter/mock/publisher.go | 16 +-- .../reporter/{subscriber.go => pubsub.go} | 121 +++++++++--------- util/topsql/reporter/reporter.go | 99 +++++++++----- util/topsql/reporter/reporter_test.go | 5 +- .../reporter/{client.go => single_target.go} | 94 ++++---------- util/topsql/topsql.go | 10 +- util/topsql/topsql_test.go | 39 +++++- util/topsql/tracecpu/mock/mock.go | 8 ++ 10 files changed, 246 insertions(+), 182 deletions(-) create mode 100644 util/topsql/reporter/datasink.go rename util/topsql/reporter/{subscriber.go => pubsub.go} (73%) rename util/topsql/reporter/{client.go => single_target.go} (70%) diff --git a/server/tidb_test.go b/server/tidb_test.go index cea71ebb55b98..8aa7e8de63f0e 100644 --- a/server/tidb_test.go +++ b/server/tidb_test.go @@ -1559,7 +1559,7 @@ func TestTopSQLReceiver(t *testing.T) { dbt.MustExec("set @@global.tidb_top_sql_report_interval_seconds=2;") dbt.MustExec("set @@global.tidb_top_sql_max_statement_count=5;") - r := reporter.NewRemoteTopSQLReporter(reporter.NewReportClientRegistry(), reporter.NewGRPCReportClient(plancodec.DecodeNormalizedPlan)) + r := reporter.NewRemoteTopSQLReporter(reporter.NewSingleTargetDataSink(plancodec.DecodeNormalizedPlan)) tracecpu.GlobalSQLCPUProfiler.SetCollector(&collectorWrapper{r}) // TODO: change to ensure that the right sql statements are reported, not just counts diff --git a/util/topsql/reporter/datasink.go b/util/topsql/reporter/datasink.go new file mode 100644 index 0000000000000..910bc1c198b5b --- /dev/null +++ b/util/topsql/reporter/datasink.go @@ -0,0 +1,34 @@ +// 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 ( + "time" +) + +// DataSink sends data to the target server. +type DataSink interface { + Send(data reportData, timeout time.Duration) + + // IsPaused indicates that DataSink is not expecting to receive records for now + // and may resume in the future. + IsPaused() bool + + // IsDown indicates that the client has been down and can be cleared. + // Note that: once a DataSink is down, it cannot go back to be up. + IsDown() bool + + Close() +} diff --git a/util/topsql/reporter/mock/publisher.go b/util/topsql/reporter/mock/publisher.go index ebe92eeb98ac1..1ba26c967077b 100644 --- a/util/topsql/reporter/mock/publisher.go +++ b/util/topsql/reporter/mock/publisher.go @@ -24,15 +24,15 @@ import ( "google.golang.org/grpc" ) -type mockPublisherServer struct { +type mockPubSubServer struct { addr string grpcServer *grpc.Server } -// StartMockPublisherServer starts the mock publisher server. -func StartMockPublisherServer( +// StartMockPubSubServer starts the mock publisher server. +func StartMockPubSubServer( service tipb.TopSQLPubSubServer, -) (*mockPublisherServer, error) { +) (*mockPubSubServer, error) { addr := "127.0.0.1:0" lis, err := net.Listen("tcp", addr) if err != nil { @@ -44,21 +44,21 @@ func StartMockPublisherServer( go func() { err := server.Serve(lis) if err != nil { - logutil.BgLogger().Warn("[top-sql] mock publisher server serve failed", zap.Error(err)) + logutil.BgLogger().Warn("[top-sql] mock pubsub server serve failed", zap.Error(err)) } }() - return &mockPublisherServer{ + return &mockPubSubServer{ addr: fmt.Sprintf("127.0.0.1:%d", lis.Addr().(*net.TCPAddr).Port), grpcServer: server, }, nil } -func (svr *mockPublisherServer) Address() string { +func (svr *mockPubSubServer) Address() string { return svr.addr } -func (svr *mockPublisherServer) Stop() { +func (svr *mockPubSubServer) Stop() { if svr.grpcServer != nil { svr.grpcServer.Stop() } diff --git a/util/topsql/reporter/subscriber.go b/util/topsql/reporter/pubsub.go similarity index 73% rename from util/topsql/reporter/subscriber.go rename to util/topsql/reporter/pubsub.go index 7e386b2c328ab..16249ecb81830 100644 --- a/util/topsql/reporter/subscriber.go +++ b/util/topsql/reporter/pubsub.go @@ -26,43 +26,46 @@ import ( "go.uber.org/zap" ) -// TopSQLPublisher implements TopSQLPublisher. +// TopSQLPubSubService implements tipb.TopSQLPubSubServer. // -// If a client subscribes to TopSQL records, the TopSQLPublisher is responsible for registering them to the reporter. -// Then the reporter sends data to the client periodically. -type TopSQLPublisher struct { - decodePlan planBinaryDecodeFunc - clientRegistry *ReportClientRegistry +// If a client subscribes to TopSQL records, the TopSQLPubSubService is responsible +// for registering an associated DataSink to the reporter. Then the reporter sends +// data to the client periodically. +type TopSQLPubSubService struct { + decodePlan planBinaryDecodeFunc + dataSinkRegisterHandle DataSinkRegisterHandle } -// NewTopSQLPublisher creates a new TopSQLPublisher. -func NewTopSQLPublisher( +// NewTopSQLPubSubService creates a new TopSQLPubSubService. +func NewTopSQLPubSubService( decodePlan planBinaryDecodeFunc, - clientRegistry *ReportClientRegistry, -) *TopSQLPublisher { - return &TopSQLPublisher{ - decodePlan: decodePlan, - clientRegistry: clientRegistry, + dataSinkRegisterHandle DataSinkRegisterHandle, +) *TopSQLPubSubService { + return &TopSQLPubSubService{ + decodePlan: decodePlan, + dataSinkRegisterHandle: dataSinkRegisterHandle, } } -var _ tipb.TopSQLPubSubServer = &TopSQLPublisher{} +var _ tipb.TopSQLPubSubServer = &TopSQLPubSubService{} -// Subscribe registers clients to the reporter and redirects data received from reporter -// to subscribers associated with those clients. -func (t *TopSQLPublisher) Subscribe( +// Subscribe registers dataSinks to the reporter and redirects data received from reporter +// to subscribers associated with those dataSinks. +func (t *TopSQLPubSubService) Subscribe( _ *tipb.TopSQLSubRequest, stream tipb.TopSQLPubSub_SubscribeServer, ) error { - sc := newSubClient(stream, t.decodePlan) + sc := newPubSubDataSink(stream, t.decodePlan) - t.clientRegistry.register(sc) - sc.run() + if err := t.dataSinkRegisterHandle.Register(sc); err != nil { + return err + } + sc.run() return nil } -type subClient struct { +type pubSubDataSink struct { stream tipb.TopSQLPubSub_SubscribeServer sendTaskCh chan sendTask isDown *atomic.Bool @@ -70,11 +73,11 @@ type subClient struct { decodePlan planBinaryDecodeFunc } -func newSubClient( +func newPubSubDataSink( stream tipb.TopSQLPubSub_SubscribeServer, decodePlan planBinaryDecodeFunc, -) *subClient { - return &subClient{ +) *pubSubDataSink { + return &pubSubDataSink{ stream: stream, sendTaskCh: make(chan sendTask), isDown: atomic.NewBool(false), @@ -83,7 +86,35 @@ func newSubClient( } } -func (s *subClient) run() { +var _ DataSink = &pubSubDataSink{} + +func (s *pubSubDataSink) Send(data reportData, timeout time.Duration) { + if s.IsDown() { + return + } + + select { + case s.sendTaskCh <- sendTask{data: data, timeout: timeout}: + // sent successfully + default: + ignoreReportChannelFullCounter.Inc() + logutil.BgLogger().Warn("[top-sql] report channel is full") + } +} + +func (s *pubSubDataSink) IsPaused() bool { + return false +} + +func (s *pubSubDataSink) IsDown() bool { + return s.isDown.Load() +} + +func (s *pubSubDataSink) Close() { + close(s.sendTaskCh) +} + +func (s *pubSubDataSink) run() { defer s.isDown.Store(true) for task := range s.sendTaskCh { @@ -111,7 +142,7 @@ func (s *subClient) run() { cancel() if err != nil { logutil.BgLogger().Warn( - "[top-sql] client failed to send data to subscriber", + "[top-sql] pubsub data sink failed to send data to subscriber", zap.Error(err), ) return @@ -119,7 +150,7 @@ func (s *subClient) run() { case <-ctx.Done(): cancel() logutil.BgLogger().Warn( - "[top-sql] client failed to send data to subscriber due to timeout", + "[top-sql] pubsub data sink failed to send data to subscriber due to timeout", zap.Duration("timeout", task.timeout), ) return @@ -127,7 +158,7 @@ func (s *subClient) run() { } } -func (s *subClient) doSend(ctx context.Context, data reportData) error { +func (s *pubSubDataSink) doSend(ctx context.Context, data reportData) error { if err := s.sendCPUTime(ctx, data.collectedData); err != nil { return err } @@ -137,7 +168,7 @@ func (s *subClient) doSend(ctx context.Context, data reportData) error { return s.sendPlanMeta(ctx, data.normalizedPlanMap) } -func (s *subClient) sendCPUTime(ctx context.Context, data []*dataPoints) (err error) { +func (s *pubSubDataSink) sendCPUTime(ctx context.Context, data []*dataPoints) (err error) { start := time.Now() sentCount := 0 defer func() { @@ -175,7 +206,7 @@ func (s *subClient) sendCPUTime(ctx context.Context, data []*dataPoints) (err er return } -func (s *subClient) sendSQLMeta(ctx context.Context, sqlMetaMap *sync.Map) (err error) { +func (s *pubSubDataSink) sendSQLMeta(ctx context.Context, sqlMetaMap *sync.Map) (err error) { start := time.Now() sentCount := 0 defer func() { @@ -217,7 +248,7 @@ func (s *subClient) sendSQLMeta(ctx context.Context, sqlMetaMap *sync.Map) (err return } -func (s *subClient) sendPlanMeta(ctx context.Context, planMetaMap *sync.Map) (err error) { +func (s *pubSubDataSink) sendPlanMeta(ctx context.Context, planMetaMap *sync.Map) (err error) { start := time.Now() sentCount := 0 defer func() { @@ -261,31 +292,3 @@ func (s *subClient) sendPlanMeta(ctx context.Context, planMetaMap *sync.Map) (er return } - -var _ ReportClient = &subClient{} - -func (s *subClient) Send(data reportData, timeout time.Duration) { - if s.IsDown() { - return - } - - select { - case s.sendTaskCh <- sendTask{data: data, timeout: timeout}: - // sent successfully - default: - ignoreReportChannelFullCounter.Inc() - logutil.BgLogger().Warn("[top-sql] report channel is full") - } -} - -func (s *subClient) IsPending() bool { - return false -} - -func (s *subClient) IsDown() bool { - return s.isDown.Load() -} - -func (s *subClient) Close() { - close(s.sendTaskCh) -} diff --git a/util/topsql/reporter/reporter.go b/util/topsql/reporter/reporter.go index 94c9a17bab85f..360e48a1f93ef 100644 --- a/util/topsql/reporter/reporter.go +++ b/util/topsql/reporter/reporter.go @@ -17,6 +17,7 @@ package reporter import ( "bytes" "context" + "errors" "sort" "sync" "sync/atomic" @@ -49,9 +50,15 @@ type TopSQLReporter interface { tracecpu.Collector RegisterSQL(sqlDigest []byte, normalizedSQL string, isInternal bool) RegisterPlan(planDigest []byte, normalizedPlan string) + DataSinkRegisterHandle() DataSinkRegisterHandle Close() } +// DataSinkRegisterHandle registers DataSink +type DataSinkRegisterHandle interface { + Register(dataSink DataSink) error +} + type cpuData struct { timestamp uint64 records []tracecpu.SQLCPUTimeRecord @@ -120,14 +127,14 @@ type RemoteTopSQLReporter struct { ctx context.Context cancel context.CancelFunc - clients []ReportClient - clientRegistry *ReportClientRegistry + dataSinks []DataSink + dataSinkRegCh chan DataSink - // normalizedSQLMap is an map, whose keys are SQL digest strings and values are SQLMeta. + // normalizedSQLMap is a 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**. + // normalizedPlanMap is a 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 @@ -143,17 +150,14 @@ type SQLMeta struct { } // NewRemoteTopSQLReporter creates a new TopSQL reporter -// -// 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 -func NewRemoteTopSQLReporter(clientRegistry *ReportClientRegistry, clients ...ReportClient) *RemoteTopSQLReporter { +func NewRemoteTopSQLReporter(dataSinks ...DataSink) *RemoteTopSQLReporter { ctx, cancel := context.WithCancel(context.Background()) tsr := &RemoteTopSQLReporter{ ctx: ctx, cancel: cancel, - clients: clients, - clientRegistry: clientRegistry, + dataSinks: dataSinks, + dataSinkRegCh: make(chan DataSink), collectCPUDataChan: make(chan cpuData, 1), reportCollectedDataChan: make(chan collectedData, 1), @@ -239,13 +243,19 @@ func (tsr *RemoteTopSQLReporter) Collect(timestamp uint64, records []tracecpu.SQ } } +// DataSinkRegisterHandle returns a DataSinkRegisterHandle for DataSink registration. +func (tsr *RemoteTopSQLReporter) DataSinkRegisterHandle() DataSinkRegisterHandle { + return &RemoteDataSinkRegisterHandle{registerCh: tsr.dataSinkRegCh} +} + // Close uses to close and release the reporter resource. func (tsr *RemoteTopSQLReporter) Close() { tsr.cancel() - for i := range tsr.clients { - tsr.clients[i].Close() + for i := range tsr.dataSinks { + tsr.dataSinks[i].Close() } - tsr.clients = nil + close(tsr.dataSinkRegCh) + tsr.dataSinks = nil } func addEvictedCPUTime(collectTarget map[string]*dataPoints, timestamp uint64, totalCPUTimeMs uint32) { @@ -331,7 +341,7 @@ func (tsr *RemoteTopSQLReporter) collectWorker() { currentReportInterval := variable.TopSQLVariable.ReportIntervalSeconds.Load() reportTicker := time.NewTicker(time.Second * time.Duration(currentReportInterval)) for { - tsr.handleClientRegistry() + tsr.handleDataSinkRegistration() select { case data := <-tsr.collectCPUDataChan: @@ -350,35 +360,41 @@ func (tsr *RemoteTopSQLReporter) collectWorker() { } } -func (tsr *RemoteTopSQLReporter) handleClientRegistry() { - tsr.clientRegistry.visitAndClear(func(client ReportClient) { - tsr.clients = append(tsr.clients, client) - }) +func (tsr *RemoteTopSQLReporter) handleDataSinkRegistration() { +out: + for { + select { + case dataSink := <-tsr.dataSinkRegCh: + tsr.dataSinks = append(tsr.dataSinks, dataSink) + default: + break out + } + } - // Remove all down clients + // Remove all down dataSinks idx := 0 - for _, client := range tsr.clients { - if client.IsDown() { - client.Close() + for _, dataSink := range tsr.dataSinks { + if dataSink.IsDown() { + dataSink.Close() continue } - tsr.clients[idx] = client + tsr.dataSinks[idx] = dataSink idx++ } - tsr.clients = tsr.clients[:idx] + tsr.dataSinks = tsr.dataSinks[:idx] - if len(tsr.clients) > 256 { - logutil.BgLogger().Warn("[top-sql] too many clients, keep 10 first", zap.Int("count", len(tsr.clients))) - tsr.clients = tsr.clients[:10] + if len(tsr.dataSinks) > 10 { + logutil.BgLogger().Warn("[top-sql] too many datasinks, keep 10 first", zap.Int("count", len(tsr.dataSinks))) + tsr.dataSinks = tsr.dataSinks[:10] } pendingCnt := 0 - for _, client := range tsr.clients { - if client.IsPending() { + for _, dataSink := range tsr.dataSinks { + if dataSink.IsPaused() { pendingCnt += 1 } } - runningCnt := len(tsr.clients) - pendingCnt + runningCnt := len(tsr.dataSinks) - pendingCnt variable.TopSQLVariable.InstanceEnable.Store(runningCnt > 0) } @@ -604,7 +620,26 @@ func (tsr *RemoteTopSQLReporter) doReport(data reportData) { } }) - for i := range tsr.clients { - tsr.clients[i].Send(data, timeout) + for i := range tsr.dataSinks { + tsr.dataSinks[i].Send(data, timeout) } } + +var _ DataSinkRegisterHandle = &RemoteDataSinkRegisterHandle{} + +// RemoteDataSinkRegisterHandle is used to receive DataSink registrations. +type RemoteDataSinkRegisterHandle struct { + registerCh chan DataSink +} + +// Register implements DataSinkRegisterHandle interface. +func (r *RemoteDataSinkRegisterHandle) Register(dataSink DataSink) (err error) { + defer func() { + if recover() != nil { + err = errors.New("registration channel is closed") + } + }() + + r.registerCh <- dataSink + return nil +} diff --git a/util/topsql/reporter/reporter_test.go b/util/topsql/reporter/reporter_test.go index 8de2e4bbe912e..6f3c9273fd7cb 100644 --- a/util/topsql/reporter/reporter_test.go +++ b/util/topsql/reporter/reporter_test.go @@ -71,9 +71,8 @@ func setupRemoteTopSQLReporter(maxStatementsNum, interval int, addr string) *Rem conf.TopSQL.ReceiverAddress = addr }) - rc := NewGRPCReportClient(mockPlanBinaryDecoderFunc) - cr := NewReportClientRegistry() - ts := NewRemoteTopSQLReporter(cr, rc) + rc := NewSingleTargetDataSink(mockPlanBinaryDecoderFunc) + ts := NewRemoteTopSQLReporter(rc) return ts } diff --git a/util/topsql/reporter/client.go b/util/topsql/reporter/single_target.go similarity index 70% rename from util/topsql/reporter/client.go rename to util/topsql/reporter/single_target.go index dda7ec99c8ef2..cb9c2bea86896 100644 --- a/util/topsql/reporter/client.go +++ b/util/topsql/reporter/single_target.go @@ -29,50 +29,8 @@ import ( "google.golang.org/grpc/backoff" ) -// ReportClient sends data to the target server. -type ReportClient interface { - Send(data reportData, timeout time.Duration) - - // IsPending indicates that ReportClient is not expecting to receive records for now and may resume in the future. - IsPending() bool - - // IsDown indicates that the client has been down and can be cleared. - // Note that: once a ReportClient is down, it cannot go back to be up. - IsDown() bool - - Close() -} - -// ReportClientRegistry is used to receive ReportClient registrations. -type ReportClientRegistry struct { - sync.Mutex - newClients []ReportClient -} - -// NewReportClientRegistry creates a new ReportClientRegistry. -func NewReportClientRegistry() *ReportClientRegistry { - return &ReportClientRegistry{} -} - -func (r *ReportClientRegistry) visitAndClear(visit func(client ReportClient)) { - r.Lock() - defer r.Unlock() - - for i := range r.newClients { - visit(r.newClients[i]) - } - r.newClients = r.newClients[:0] -} - -func (r *ReportClientRegistry) register(client ReportClient) { - r.Lock() - defer r.Unlock() - - r.newClients = append(r.newClients, client) -} - -// ConfigAssignedRemoteClient reports data to grpc servers. -type ConfigAssignedRemoteClient struct { +// SingleTargetDataSink reports data to grpc servers. +type SingleTargetDataSink struct { curRPCAddr string conn *grpc.ClientConn sendTaskCh chan sendTask @@ -85,18 +43,20 @@ type sendTask struct { timeout time.Duration } -// NewGRPCReportClient returns a new ConfigAssignedRemoteClient -func NewGRPCReportClient(decodePlan planBinaryDecodeFunc) *ConfigAssignedRemoteClient { - client := &ConfigAssignedRemoteClient{ +// NewSingleTargetDataSink returns a new SingleTargetDataSink +// +// planBinaryDecodeFunc is a decoding function which will be called asynchronously to decode the plan binary to string +func NewSingleTargetDataSink(decodePlan planBinaryDecodeFunc) *SingleTargetDataSink { + dataSink := &SingleTargetDataSink{ decodePlan: decodePlan, sendTaskCh: make(chan sendTask), } - go util.WithRecovery(client.run, nil) - return client + go util.WithRecovery(dataSink.run, nil) + return dataSink } -func (r *ConfigAssignedRemoteClient) run() { +func (r *SingleTargetDataSink) run() { for task := range r.sendTaskCh { targetRPCAddr := config.GetGlobalConfig().TopSQL.ReceiverAddress if targetRPCAddr == "" { @@ -108,7 +68,7 @@ func (r *ConfigAssignedRemoteClient) run() { err := r.doSend(ctx, targetRPCAddr, task.data) cancel() if err != nil { - logutil.BgLogger().Warn("[top-sql] client failed to send data to receiver", zap.Error(err)) + logutil.BgLogger().Warn("[top-sql] single target data sink failed to send data to receiver", zap.Error(err)) reportAllDurationFailedHistogram.Observe(time.Since(start).Seconds()) } else { reportAllDurationSuccHistogram.Observe(time.Since(start).Seconds()) @@ -116,10 +76,10 @@ func (r *ConfigAssignedRemoteClient) run() { } } -var _ ReportClient = &ConfigAssignedRemoteClient{} +var _ DataSink = &SingleTargetDataSink{} -// Send implements the ReportClient interface. -func (r *ConfigAssignedRemoteClient) Send(data reportData, timeout time.Duration) { +// Send implements the DataSink interface. +func (r *SingleTargetDataSink) Send(data reportData, timeout time.Duration) { select { case r.sendTaskCh <- sendTask{data: data, timeout: timeout}: // sent successfully @@ -130,7 +90,7 @@ func (r *ConfigAssignedRemoteClient) Send(data reportData, timeout time.Duration } // Currently the doSend will establish a new connection every time, which is suitable for a per-minute sending period -func (r *ConfigAssignedRemoteClient) doSend(ctx context.Context, addr string, data reportData) (err error) { +func (r *SingleTargetDataSink) doSend(ctx context.Context, addr string, data reportData) (err error) { err = r.tryEstablishConnection(ctx, addr) if err != nil { return @@ -163,31 +123,31 @@ func (r *ConfigAssignedRemoteClient) doSend(ctx context.Context, addr string, da return } -// IsPending implements ReportClient interface. -func (r *ConfigAssignedRemoteClient) IsPending() bool { +// IsPaused implements DataSink interface. +func (r *SingleTargetDataSink) IsPaused() bool { return len(config.GetGlobalConfig().TopSQL.ReceiverAddress) == 0 } -// IsDown implements ReportClient interface. -func (r *ConfigAssignedRemoteClient) IsDown() bool { +// IsDown implements DataSink interface. +func (r *SingleTargetDataSink) IsDown() bool { return false } // Close uses to close grpc connection. -func (r *ConfigAssignedRemoteClient) Close() { +func (r *SingleTargetDataSink) Close() { close(r.sendTaskCh) if r.conn == nil { return } err := r.conn.Close() if err != nil { - logutil.BgLogger().Warn("[top-sql] grpc client close connection failed", zap.Error(err)) + logutil.BgLogger().Warn("[top-sql] single target data sink failed to close connection", zap.Error(err)) } r.conn = nil } // sendBatchCPUTimeRecord sends a batch of TopSQL records by stream. -func (r *ConfigAssignedRemoteClient) sendBatchCPUTimeRecord(ctx context.Context, records []*dataPoints) error { +func (r *SingleTargetDataSink) sendBatchCPUTimeRecord(ctx context.Context, records []*dataPoints) error { if len(records) == 0 { return nil } @@ -220,7 +180,7 @@ func (r *ConfigAssignedRemoteClient) sendBatchCPUTimeRecord(ctx context.Context, } // sendBatchSQLMeta sends a batch of SQL metas by stream. -func (r *ConfigAssignedRemoteClient) sendBatchSQLMeta(ctx context.Context, sqlMap *sync.Map) error { +func (r *SingleTargetDataSink) sendBatchSQLMeta(ctx context.Context, sqlMap *sync.Map) error { start := time.Now() client := tipb.NewTopSQLAgentClient(r.conn) stream, err := client.ReportSQLMeta(ctx) @@ -256,7 +216,7 @@ func (r *ConfigAssignedRemoteClient) sendBatchSQLMeta(ctx context.Context, sqlMa } // sendBatchPlanMeta sends a batch of SQL metas by stream. -func (r *ConfigAssignedRemoteClient) sendBatchPlanMeta(ctx context.Context, planMap *sync.Map) error { +func (r *SingleTargetDataSink) sendBatchPlanMeta(ctx context.Context, planMap *sync.Map) error { start := time.Now() client := tipb.NewTopSQLAgentClient(r.conn) stream, err := client.ReportPlanMeta(ctx) @@ -295,7 +255,7 @@ func (r *ConfigAssignedRemoteClient) sendBatchPlanMeta(ctx context.Context, plan } // tryEstablishConnection establishes the gRPC connection if connection is not established. -func (r *ConfigAssignedRemoteClient) tryEstablishConnection(ctx context.Context, targetRPCAddr string) (err error) { +func (r *SingleTargetDataSink) tryEstablishConnection(ctx context.Context, targetRPCAddr string) (err error) { if r.curRPCAddr == targetRPCAddr && r.conn != nil { // Address is not changed, skip. return nil @@ -303,7 +263,7 @@ func (r *ConfigAssignedRemoteClient) tryEstablishConnection(ctx context.Context, if r.conn != nil { err := r.conn.Close() - logutil.BgLogger().Warn("[top-sql] grpc client close connection failed", zap.Error(err)) + logutil.BgLogger().Warn("[top-sql] single target data sink failed to close connection", zap.Error(err)) } r.conn, err = r.dial(ctx, targetRPCAddr) @@ -314,7 +274,7 @@ func (r *ConfigAssignedRemoteClient) tryEstablishConnection(ctx context.Context, return nil } -func (r *ConfigAssignedRemoteClient) dial(ctx context.Context, targetRPCAddr string) (*grpc.ClientConn, error) { +func (r *SingleTargetDataSink) dial(ctx context.Context, targetRPCAddr string) (*grpc.ClientConn, error) { dialCtx, cancel := context.WithTimeout(ctx, dialTimeout) defer cancel() return grpc.DialContext( diff --git a/util/topsql/topsql.go b/util/topsql/topsql.go index fda7ea41f4942..250dabf66795b 100644 --- a/util/topsql/topsql.go +++ b/util/topsql/topsql.go @@ -47,14 +47,14 @@ var ( // SetupTopSQL sets up the top-sql worker. func SetupTopSQL() { - rc := reporter.NewGRPCReportClient(plancodec.DecodeNormalizedPlan) - cr := reporter.NewReportClientRegistry() - globalTopSQLReport = reporter.NewRemoteTopSQLReporter(cr, rc) + rc := reporter.NewSingleTargetDataSink(plancodec.DecodeNormalizedPlan) + globalTopSQLReport = reporter.NewRemoteTopSQLReporter(rc) tracecpu.GlobalSQLCPUProfiler.SetCollector(globalTopSQLReport) tracecpu.GlobalSQLCPUProfiler.Run() - publisher := reporter.NewTopSQLPublisher(plancodec.DecodeNormalizedPlan, cr) - globalTopSQLPubSubService = publisher + dataSinkRegHandle := globalTopSQLReport.DataSinkRegisterHandle() + pubsub := reporter.NewTopSQLPubSubService(plancodec.DecodeNormalizedPlan, dataSinkRegHandle) + globalTopSQLPubSubService = pubsub if len(config.GetGlobalConfig().TopSQL.ReceiverAddress) != 0 { variable.TopSQLVariable.InstanceEnable.Store(true) diff --git a/util/topsql/topsql_test.go b/util/topsql/topsql_test.go index b924bca1017d4..90140ae9d4da5 100644 --- a/util/topsql/topsql_test.go +++ b/util/topsql/topsql_test.go @@ -120,9 +120,8 @@ func TestTopSQLReporter(t *testing.T) { conf.TopSQL.ReceiverAddress = server.Address() }) - client := reporter.NewGRPCReportClient(mockPlanBinaryDecoderFunc) - cr := reporter.NewReportClientRegistry() - report := reporter.NewRemoteTopSQLReporter(cr, client) + dataSink := reporter.NewSingleTargetDataSink(mockPlanBinaryDecoderFunc) + report := reporter.NewRemoteTopSQLReporter(dataSink) defer report.Close() tracecpu.GlobalSQLCPUProfiler.SetCollector(&collectorWrapper{report}) @@ -186,12 +185,11 @@ func TestTopSQLPubSub(t *testing.T) { variable.TopSQLVariable.MaxStatementCount.Store(200) variable.TopSQLVariable.ReportIntervalSeconds.Store(1) - cr := reporter.NewReportClientRegistry() - report := reporter.NewRemoteTopSQLReporter(cr) + report := reporter.NewRemoteTopSQLReporter() defer report.Close() - publisherServer := reporter.NewTopSQLPublisher(mockPlanBinaryDecoderFunc, cr) - server, err := mockServer.StartMockPublisherServer(publisherServer) + pubsub := reporter.NewTopSQLPubSubService(mockPlanBinaryDecoderFunc, report.DataSinkRegisterHandle()) + server, err := mockServer.StartMockPubSubServer(pubsub) require.NoError(t, err) defer server.Stop() @@ -296,6 +294,33 @@ func TestTopSQLPubSub(t *testing.T) { require.Equal(t, len(checkSQLPlanMap), 2) } +func TestTopSQLPubSubReporterStopBeforePubSub(t *testing.T) { + report := reporter.NewRemoteTopSQLReporter() + + pubsub := reporter.NewTopSQLPubSubService(mockPlanBinaryDecoderFunc, report.DataSinkRegisterHandle()) + server, err := mockServer.StartMockPubSubServer(pubsub) + defer server.Stop() + require.NoError(t, err) + + // close reporter first + report.Close() + + // try to subscribe + tracecpu.GlobalSQLCPUProfiler.SetCollector(&collectorWrapper{report}) + conn, err := grpc.Dial(server.Address(), grpc.WithBlock(), grpc.WithInsecure()) + require.NoError(t, err) + defer conn.Close() + + ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) + defer cancel() + client := tipb.NewTopSQLPubSubClient(conn) + stream, err := client.Subscribe(ctx, &tipb.TopSQLSubRequest{}) + require.NoError(t, err) + + _, err = stream.Recv() + require.NotNil(t, err) +} + func TestMaxSQLAndPlanTest(t *testing.T) { collector := mock.NewTopSQLCollector() tracecpu.GlobalSQLCPUProfiler.SetCollector(&collectorWrapper{collector}) diff --git a/util/topsql/tracecpu/mock/mock.go b/util/topsql/tracecpu/mock/mock.go index 62125d202f67c..0c843a12cac83 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/reporter" "github.com/pingcap/tidb/util/topsql/tracecpu" "go.uber.org/atomic" "go.uber.org/zap" @@ -48,6 +49,8 @@ func NewTopSQLCollector() *TopSQLCollector { } } +var _ reporter.TopSQLReporter = &TopSQLCollector{} + // Collect uses for testing. func (c *TopSQLCollector) Collect(ts uint64, stats []tracecpu.SQLCPUTimeRecord) { defer c.collectCnt.Inc() @@ -168,6 +171,11 @@ func (c *TopSQLCollector) WaitCollectCnt(count int64) { } } +// DataSinkRegisterHandle implements the interface. +func (c *TopSQLCollector) DataSinkRegisterHandle() reporter.DataSinkRegisterHandle { + return nil +} + // Close implements the interface. func (c *TopSQLCollector) Close() {} From 2ed9c6ebb86dbd3c9f3c2609f72de5fdd17c8c51 Mon Sep 17 00:00:00 2001 From: Zhenchi Date: Thu, 9 Dec 2021 18:23:51 +0800 Subject: [PATCH 27/47] share tipb report data construction Signed-off-by: Zhenchi --- server/tidb_test.go | 4 +- util/topsql/reporter/pubsub.go | 128 +++++++++-------------- util/topsql/reporter/reporter.go | 111 +++++++++++++------- util/topsql/reporter/reporter_test.go | 22 ++-- util/topsql/reporter/single_target.go | 142 ++++++++++++-------------- util/topsql/topsql.go | 14 ++- util/topsql/topsql_test.go | 15 +-- util/topsql/tracecpu/mock/mock.go | 4 +- 8 files changed, 225 insertions(+), 215 deletions(-) diff --git a/server/tidb_test.go b/server/tidb_test.go index 8aa7e8de63f0e..490a35565e4f0 100644 --- a/server/tidb_test.go +++ b/server/tidb_test.go @@ -1559,7 +1559,9 @@ func TestTopSQLReceiver(t *testing.T) { dbt.MustExec("set @@global.tidb_top_sql_report_interval_seconds=2;") dbt.MustExec("set @@global.tidb_top_sql_max_statement_count=5;") - r := reporter.NewRemoteTopSQLReporter(reporter.NewSingleTargetDataSink(plancodec.DecodeNormalizedPlan)) + r := reporter.NewRemoteTopSQLReporter(plancodec.DecodeNormalizedPlan) + err = r.DataSinkRegHandle().Register(reporter.NewSingleTargetDataSink()) + require.NoError(t, err) tracecpu.GlobalSQLCPUProfiler.SetCollector(&collectorWrapper{r}) // TODO: change to ensure that the right sql statements are reported, not just counts diff --git a/util/topsql/reporter/pubsub.go b/util/topsql/reporter/pubsub.go index 16249ecb81830..ff573985e481c 100644 --- a/util/topsql/reporter/pubsub.go +++ b/util/topsql/reporter/pubsub.go @@ -16,7 +16,6 @@ package reporter import ( "context" - "sync" "time" "github.com/pingcap/tidb/util" @@ -32,18 +31,15 @@ import ( // for registering an associated DataSink to the reporter. Then the reporter sends // data to the client periodically. type TopSQLPubSubService struct { - decodePlan planBinaryDecodeFunc - dataSinkRegisterHandle DataSinkRegisterHandle + dataSinkRegHandle DataSinkRegHandle } // NewTopSQLPubSubService creates a new TopSQLPubSubService. func NewTopSQLPubSubService( - decodePlan planBinaryDecodeFunc, - dataSinkRegisterHandle DataSinkRegisterHandle, + dataSinkRegHandle DataSinkRegHandle, ) *TopSQLPubSubService { return &TopSQLPubSubService{ - decodePlan: decodePlan, - dataSinkRegisterHandle: dataSinkRegisterHandle, + dataSinkRegHandle: dataSinkRegHandle, } } @@ -55,9 +51,9 @@ func (t *TopSQLPubSubService) Subscribe( _ *tipb.TopSQLSubRequest, stream tipb.TopSQLPubSub_SubscribeServer, ) error { - sc := newPubSubDataSink(stream, t.decodePlan) + sc := newPubSubDataSink(stream) - if err := t.dataSinkRegisterHandle.Register(sc); err != nil { + if err := t.dataSinkRegHandle.Register(sc); err != nil { return err } @@ -69,20 +65,15 @@ type pubSubDataSink struct { stream tipb.TopSQLPubSub_SubscribeServer sendTaskCh chan sendTask isDown *atomic.Bool - - decodePlan planBinaryDecodeFunc } func newPubSubDataSink( stream tipb.TopSQLPubSub_SubscribeServer, - decodePlan planBinaryDecodeFunc, ) *pubSubDataSink { return &pubSubDataSink{ stream: stream, sendTaskCh: make(chan sendTask), isDown: atomic.NewBool(false), - - decodePlan: decodePlan, } } @@ -123,23 +114,21 @@ func (s *pubSubDataSink) run() { var err error doneCh := make(chan struct{}) - go func(task sendTask) { - util.WithRecovery(func() { - defer func() { - doneCh <- struct{}{} - }() - err = s.doSend(ctx, task.data) - if err != nil { - reportAllDurationFailedHistogram.Observe(time.Since(start).Seconds()) - } else { - reportAllDurationSuccHistogram.Observe(time.Since(start).Seconds()) - } - }, nil) - }(task) + go util.WithRecovery(func() { + defer func() { + doneCh <- struct{}{} + cancel() + }() + err = s.doSend(ctx, task.data) + if err != nil { + reportAllDurationFailedHistogram.Observe(time.Since(start).Seconds()) + } else { + reportAllDurationSuccHistogram.Observe(time.Since(start).Seconds()) + } + }, nil) select { case <-doneCh: - cancel() if err != nil { logutil.BgLogger().Warn( "[top-sql] pubsub data sink failed to send data to subscriber", @@ -148,7 +137,6 @@ func (s *pubSubDataSink) run() { return } case <-ctx.Done(): - cancel() logutil.BgLogger().Warn( "[top-sql] pubsub data sink failed to send data to subscriber due to timeout", zap.Duration("timeout", task.timeout), @@ -159,16 +147,20 @@ func (s *pubSubDataSink) run() { } func (s *pubSubDataSink) doSend(ctx context.Context, data reportData) error { - if err := s.sendCPUTime(ctx, data.collectedData); err != nil { + if err := s.sendCPUTime(ctx, data.cpuTimeRecords); err != nil { return err } - if err := s.sendSQLMeta(ctx, data.normalizedSQLMap); err != nil { + if err := s.sendSQLMeta(ctx, data.sqlMetas); err != nil { return err } - return s.sendPlanMeta(ctx, data.normalizedPlanMap) + return s.sendPlanMeta(ctx, data.planMetas) } -func (s *pubSubDataSink) sendCPUTime(ctx context.Context, data []*dataPoints) (err error) { +func (s *pubSubDataSink) sendCPUTime(ctx context.Context, records []*tipb.CPUTimeRecord) (err error) { + if len(records) == 0 { + return + } + start := time.Now() sentCount := 0 defer func() { @@ -180,16 +172,11 @@ func (s *pubSubDataSink) sendCPUTime(ctx context.Context, data []*dataPoints) (e } }() - r := &tipb.TopSQLSubResponse{} - record := &tipb.CPUTimeRecord{} - r.RespOneof = &tipb.TopSQLSubResponse_Record{Record: record} + cpuRecord := &tipb.TopSQLSubResponse_Record{} + r := &tipb.TopSQLSubResponse{RespOneof: cpuRecord} - for i := range data { - point := data[i] - record.SqlDigest = point.SQLDigest - record.PlanDigest = point.PlanDigest - record.RecordListCpuTimeMs = point.CPUTimeMsList - record.RecordListTimestampSec = point.TimestampList + for i := range records { + cpuRecord.Record = records[i] if err = s.stream.Send(r); err != nil { return } @@ -206,7 +193,11 @@ func (s *pubSubDataSink) sendCPUTime(ctx context.Context, data []*dataPoints) (e return } -func (s *pubSubDataSink) sendSQLMeta(ctx context.Context, sqlMetaMap *sync.Map) (err error) { +func (s *pubSubDataSink) sendSQLMeta(ctx context.Context, sqlMetas []*tipb.SQLMeta) (err error) { + if len(sqlMetas) == 0 { + return + } + start := time.Now() sentCount := 0 defer func() { @@ -218,37 +209,32 @@ func (s *pubSubDataSink) sendSQLMeta(ctx context.Context, sqlMetaMap *sync.Map) } }() - r := &tipb.TopSQLSubResponse{} - sqlMeta := &tipb.SQLMeta{} - r.RespOneof = &tipb.TopSQLSubResponse_SqlMeta{SqlMeta: sqlMeta} + sqlMeta := &tipb.TopSQLSubResponse_SqlMeta{} + r := &tipb.TopSQLSubResponse{RespOneof: sqlMeta} - sqlMetaMap.Range(func(key, value interface{}) bool { - meta := value.(SQLMeta) - sqlMeta.SqlDigest = []byte(key.(string)) - sqlMeta.NormalizedSql = meta.normalizedSQL - sqlMeta.IsInternalSql = meta.isInternal + for i := range sqlMetas { + sqlMeta.SqlMeta = sqlMetas[i] if err = s.stream.Send(r); err != nil { - return false + return } sentCount += 1 select { case <-ctx.Done(): err = ctx.Err() - return false + return default: } - - return true - }) - if err != nil { - return err } return } -func (s *pubSubDataSink) sendPlanMeta(ctx context.Context, planMetaMap *sync.Map) (err error) { +func (s *pubSubDataSink) sendPlanMeta(ctx context.Context, planMetas []*tipb.PlanMeta) (err error) { + if len(planMetas) == 0 { + return + } + start := time.Now() sentCount := 0 defer func() { @@ -260,34 +246,22 @@ func (s *pubSubDataSink) sendPlanMeta(ctx context.Context, planMetaMap *sync.Map } }() - r := &tipb.TopSQLSubResponse{} - planMeta := &tipb.PlanMeta{} - r.RespOneof = &tipb.TopSQLSubResponse_PlanMeta{PlanMeta: planMeta} - planMetaMap.Range(func(key, value interface{}) bool { - planDecoded, err1 := s.decodePlan(value.(string)) - if err1 != nil { - logutil.BgLogger().Warn("[top-sql] decode plan failed", zap.Error(err1)) - return true - } + planMeta := &tipb.TopSQLSubResponse_PlanMeta{} + r := &tipb.TopSQLSubResponse{RespOneof: planMeta} - planMeta.PlanDigest = []byte(key.(string)) - planMeta.NormalizedPlan = planDecoded + for i := range planMetas { + planMeta.PlanMeta = planMetas[i] if err = s.stream.Send(r); err != nil { - return false + return } sentCount += 1 select { case <-ctx.Done(): err = ctx.Err() - return false + return default: } - - return true - }) - if err != nil { - return err } return diff --git a/util/topsql/reporter/reporter.go b/util/topsql/reporter/reporter.go index 360e48a1f93ef..30d260477a382 100644 --- a/util/topsql/reporter/reporter.go +++ b/util/topsql/reporter/reporter.go @@ -29,6 +29,7 @@ import ( "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/logutil" "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" @@ -50,12 +51,12 @@ type TopSQLReporter interface { tracecpu.Collector RegisterSQL(sqlDigest []byte, normalizedSQL string, isInternal bool) RegisterPlan(planDigest []byte, normalizedPlan string) - DataSinkRegisterHandle() DataSinkRegisterHandle + DataSinkRegHandle() DataSinkRegHandle Close() } -// DataSinkRegisterHandle registers DataSink -type DataSinkRegisterHandle interface { +// DataSinkRegHandle registers DataSink +type DataSinkRegHandle interface { Register(dataSink DataSink) error } @@ -139,6 +140,9 @@ type RemoteTopSQLReporter struct { normalizedPlanMap atomic.Value // sync.Map planMapLength atomic2.Int64 + // calling decodePlan this can take a while, so should not block critical paths + decodePlan planBinaryDecodeFunc + collectCPUDataChan chan cpuData reportCollectedDataChan chan collectedData } @@ -150,15 +154,18 @@ type SQLMeta struct { } // NewRemoteTopSQLReporter creates a new TopSQL reporter -func NewRemoteTopSQLReporter(dataSinks ...DataSink) *RemoteTopSQLReporter { +// +// planBinaryDecodeFunc is a decoding function which will be called asynchronously to decode the plan binary to string +func NewRemoteTopSQLReporter(planBinaryDecodeFunc planBinaryDecodeFunc) *RemoteTopSQLReporter { ctx, cancel := context.WithCancel(context.Background()) tsr := &RemoteTopSQLReporter{ ctx: ctx, cancel: cancel, - dataSinks: dataSinks, dataSinkRegCh: make(chan DataSink), + decodePlan: planBinaryDecodeFunc, + collectCPUDataChan: make(chan cpuData, 1), reportCollectedDataChan: make(chan collectedData, 1), } @@ -243,9 +250,9 @@ func (tsr *RemoteTopSQLReporter) Collect(timestamp uint64, records []tracecpu.SQ } } -// DataSinkRegisterHandle returns a DataSinkRegisterHandle for DataSink registration. -func (tsr *RemoteTopSQLReporter) DataSinkRegisterHandle() DataSinkRegisterHandle { - return &RemoteDataSinkRegisterHandle{registerCh: tsr.dataSinkRegCh} +// DataSinkRegHandle returns a DataSinkRegHandle for DataSink registration. +func (tsr *RemoteTopSQLReporter) DataSinkRegHandle() DataSinkRegHandle { + return &RemoteDataSinkRegHandle{registerCh: tsr.dataSinkRegCh} } // Close uses to close and release the reporter resource. @@ -523,29 +530,14 @@ type collectedData struct { // reportData contains data that reporter sends to the agent type reportData struct { - // collectedData contains the topN collected records and the `others` record which aggregation all records that is out of Top N. - collectedData []*dataPoints - normalizedSQLMap *sync.Map - normalizedPlanMap *sync.Map + // 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 } func (d *reportData) hasData() bool { - if len(d.collectedData) > 0 { - return true - } - cnt := 0 - d.normalizedSQLMap.Range(func(key, value interface{}) bool { - cnt++ - return false - }) - if cnt > 0 { - return true - } - d.normalizedPlanMap.Range(func(key, value interface{}) bool { - cnt++ - return false - }) - return cnt > 0 + return len(d.cpuTimeRecords) != 0 || len(d.sqlMetas) != 0 || len(d.planMetas) != 0 } // reportWorker sends data to the gRPC endpoint from the `reportCollectedDataChan` one by one. @@ -570,10 +562,19 @@ func (tsr *RemoteTopSQLReporter) reportWorker() { // 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, sqlMap, planMap := getTopN(collected) + return buildReportData(records, sqlMap, planMap, tsr.decodePlan) +} + +func getTopN(collected collectedData) (records []*dataPoints, sqlMap *sync.Map, planMap *sync.Map) { // Fetch TopN dataPoints. others := collected.records[keyOthers] delete(collected.records, keyOthers) - records := make([]*dataPoints, 0, len(collected.records)) + + records = make([]*dataPoints, 0, len(collected.records)) + sqlMap = collected.normalizedSQLMap + planMap = collected.normalizedPlanMap + for _, v := range collected.records { records = append(records, v) } @@ -596,11 +597,43 @@ func (tsr *RemoteTopSQLReporter) getReportData(collected collectedData) reportDa records = append(records, others) } - return reportData{ - collectedData: records, - normalizedSQLMap: collected.normalizedSQLMap, - normalizedPlanMap: collected.normalizedPlanMap, - } + return +} + +func buildReportData(records []*dataPoints, sqlMap *sync.Map, planMap *sync.Map, decodePlan planBinaryDecodeFunc) (res reportData) { + for _, record := range records { + res.cpuTimeRecords = append(res.cpuTimeRecords, &tipb.CPUTimeRecord{ + RecordListTimestampSec: record.TimestampList, + RecordListCpuTimeMs: record.CPUTimeMsList, + 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 := 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 } func (tsr *RemoteTopSQLReporter) doReport(data reportData) { @@ -625,15 +658,15 @@ func (tsr *RemoteTopSQLReporter) doReport(data reportData) { } } -var _ DataSinkRegisterHandle = &RemoteDataSinkRegisterHandle{} +var _ DataSinkRegHandle = &RemoteDataSinkRegHandle{} -// RemoteDataSinkRegisterHandle is used to receive DataSink registrations. -type RemoteDataSinkRegisterHandle struct { +// RemoteDataSinkRegHandle is used to receive DataSink registrations. +type RemoteDataSinkRegHandle struct { registerCh chan DataSink } -// Register implements DataSinkRegisterHandle interface. -func (r *RemoteDataSinkRegisterHandle) Register(dataSink DataSink) (err error) { +// Register implements DataSinkRegHandle interface. +func (r *RemoteDataSinkRegHandle) Register(dataSink DataSink) (err error) { defer func() { if recover() != nil { err = errors.New("registration channel is closed") diff --git a/util/topsql/reporter/reporter_test.go b/util/topsql/reporter/reporter_test.go index 6f3c9273fd7cb..9712010c92131 100644 --- a/util/topsql/reporter/reporter_test.go +++ b/util/topsql/reporter/reporter_test.go @@ -64,20 +64,24 @@ func mockPlanBinaryDecoderFunc(plan string) (string, error) { return plan, nil } -func setupRemoteTopSQLReporter(maxStatementsNum, interval int, addr string) *RemoteTopSQLReporter { +func setupRemoteTopSQLReporter(t *testing.T, maxStatementsNum, interval int, addr string) *RemoteTopSQLReporter { variable.TopSQLVariable.MaxStatementCount.Store(int64(maxStatementsNum)) variable.TopSQLVariable.ReportIntervalSeconds.Store(int64(interval)) config.UpdateGlobal(func(conf *config.Config) { conf.TopSQL.ReceiverAddress = addr }) - rc := NewSingleTargetDataSink(mockPlanBinaryDecoderFunc) - ts := NewRemoteTopSQLReporter(rc) + ts := NewRemoteTopSQLReporter(mockPlanBinaryDecoderFunc) + ds := NewSingleTargetDataSink() + err := ts.DataSinkRegHandle().Register(ds) + if t != nil { + require.NoError(t, err) + } return ts } func initializeCache(maxStatementsNum, interval int, addr string) *RemoteTopSQLReporter { - ts := setupRemoteTopSQLReporter(maxStatementsNum, interval, addr) + ts := setupRemoteTopSQLReporter(nil, maxStatementsNum, interval, addr) populateCache(ts, 0, maxStatementsNum, 1) return ts } @@ -87,7 +91,7 @@ func TestCollectAndSendBatch(t *testing.T) { require.NoError(t, err) defer agentServer.Stop() - tsr := setupRemoteTopSQLReporter(maxSQLNum, 1, agentServer.Address()) + tsr := setupRemoteTopSQLReporter(t, maxSQLNum, 1, agentServer.Address()) defer tsr.Close() populateCache(tsr, 0, maxSQLNum, 1) @@ -126,7 +130,7 @@ func TestCollectAndEvicted(t *testing.T) { require.NoError(t, err) defer agentServer.Stop() - tsr := setupRemoteTopSQLReporter(maxSQLNum, 1, agentServer.Address()) + tsr := setupRemoteTopSQLReporter(t, maxSQLNum, 1, agentServer.Address()) defer tsr.Close() populateCache(tsr, 0, maxSQLNum*2, 2) @@ -191,7 +195,7 @@ func TestCollectAndTopN(t *testing.T) { require.NoError(t, err) defer agentServer.Stop() - tsr := setupRemoteTopSQLReporter(2, 1, agentServer.Address()) + tsr := setupRemoteTopSQLReporter(t, 2, 1, agentServer.Address()) defer tsr.Close() records := []tracecpu.SQLCPUTimeRecord{ @@ -252,7 +256,7 @@ func TestCollectAndTopN(t *testing.T) { } func TestCollectCapacity(t *testing.T) { - tsr := setupRemoteTopSQLReporter(maxSQLNum, 60, "") + tsr := setupRemoteTopSQLReporter(t, maxSQLNum, 60, "") defer tsr.Close() registerSQL := func(n int) { @@ -393,7 +397,7 @@ func TestCollectInternal(t *testing.T) { require.NoError(t, err) defer agentServer.Stop() - tsr := setupRemoteTopSQLReporter(3000, 1, agentServer.Address()) + tsr := setupRemoteTopSQLReporter(t, 3000, 1, agentServer.Address()) defer tsr.Close() records := []tracecpu.SQLCPUTimeRecord{ diff --git a/util/topsql/reporter/single_target.go b/util/topsql/reporter/single_target.go index cb9c2bea86896..2a05869c0bc97 100644 --- a/util/topsql/reporter/single_target.go +++ b/util/topsql/reporter/single_target.go @@ -34,8 +34,6 @@ type SingleTargetDataSink struct { curRPCAddr string conn *grpc.ClientConn sendTaskCh chan sendTask - // calling decodePlan this can take a while, so should not block critical paths - decodePlan planBinaryDecodeFunc } type sendTask struct { @@ -45,10 +43,8 @@ type sendTask struct { // NewSingleTargetDataSink returns a new SingleTargetDataSink // -// planBinaryDecodeFunc is a decoding function which will be called asynchronously to decode the plan binary to string -func NewSingleTargetDataSink(decodePlan planBinaryDecodeFunc) *SingleTargetDataSink { +func NewSingleTargetDataSink() *SingleTargetDataSink { dataSink := &SingleTargetDataSink{ - decodePlan: decodePlan, sendTaskCh: make(chan sendTask), } @@ -102,15 +98,15 @@ func (r *SingleTargetDataSink) doSend(ctx context.Context, addr string, data rep go func() { defer wg.Done() - errCh <- r.sendBatchSQLMeta(ctx, data.normalizedSQLMap) + errCh <- r.sendBatchSQLMeta(ctx, data.sqlMetas) }() go func() { defer wg.Done() - errCh <- r.sendBatchPlanMeta(ctx, data.normalizedPlanMap) + errCh <- r.sendBatchPlanMeta(ctx, data.planMetas) }() go func() { defer wg.Done() - errCh <- r.sendBatchCPUTimeRecord(ctx, data.collectedData) + errCh <- r.sendBatchCPUTimeRecord(ctx, data.cpuTimeRecords) }() wg.Wait() close(errCh) @@ -147,111 +143,105 @@ func (r *SingleTargetDataSink) Close() { } // sendBatchCPUTimeRecord sends a batch of TopSQL records by stream. -func (r *SingleTargetDataSink) sendBatchCPUTimeRecord(ctx context.Context, records []*dataPoints) error { +func (r *SingleTargetDataSink) sendBatchCPUTimeRecord(ctx context.Context, records []*tipb.CPUTimeRecord) (err error) { if len(records) == 0 { - return nil + return } + start := time.Now() + sentCount := 0 + defer func() { + topSQLReportRecordCounterHistogram.Observe(float64(sentCount)) + if err != nil { + reportRecordDurationFailedHistogram.Observe(time.Since(start).Seconds()) + } + reportRecordDurationSuccHistogram.Observe(time.Since(start).Seconds()) + }() + client := tipb.NewTopSQLAgentClient(r.conn) stream, err := client.ReportCPUTimeRecords(ctx) if err != nil { return err } for _, record := range records { - record := &tipb.CPUTimeRecord{ - RecordListTimestampSec: record.TimestampList, - RecordListCpuTimeMs: record.CPUTimeMsList, - SqlDigest: record.SQLDigest, - PlanDigest: record.PlanDigest, - } - if err := stream.Send(record); err != nil { - return err + if err = stream.Send(record); err != nil { + return } + sentCount += 1 } - topSQLReportRecordCounterHistogram.Observe(float64(len(records))) + // See https://pkg.go.dev/google.golang.org/grpc#ClientConn.NewStream for how to avoid leaking the stream _, err = stream.CloseAndRecv() - if err != nil { - reportRecordDurationFailedHistogram.Observe(time.Since(start).Seconds()) - return err - } - reportRecordDurationSuccHistogram.Observe(time.Since(start).Seconds()) - return nil + return } // sendBatchSQLMeta sends a batch of SQL metas by stream. -func (r *SingleTargetDataSink) sendBatchSQLMeta(ctx context.Context, sqlMap *sync.Map) error { +func (r *SingleTargetDataSink) sendBatchSQLMeta(ctx context.Context, sqlMetas []*tipb.SQLMeta) (err error) { + if len(sqlMetas) == 0 { + return + } + start := time.Now() + sentCount := 0 + defer func() { + topSQLReportSQLCountHistogram.Observe(float64(sentCount)) + + if err != nil { + reportSQLDurationFailedHistogram.Observe(time.Since(start).Seconds()) + } + reportSQLDurationSuccHistogram.Observe(time.Since(start).Seconds()) + }() + client := tipb.NewTopSQLAgentClient(r.conn) stream, err := client.ReportSQLMeta(ctx) if err != nil { return err } - cnt := 0 - sqlMap.Range(func(key, value interface{}) bool { - cnt++ - meta := value.(SQLMeta) - sqlMeta := &tipb.SQLMeta{ - SqlDigest: []byte(key.(string)), - NormalizedSql: meta.normalizedSQL, - IsInternalSql: meta.isInternal, - } - if err = stream.Send(sqlMeta); err != nil { - return false + + for _, meta := range sqlMetas { + if err = stream.Send(meta); err != nil { + return } - return true - }) - // stream.Send return error - if err != nil { - return err + sentCount += 1 } - topSQLReportSQLCountHistogram.Observe(float64(cnt)) + + // See https://pkg.go.dev/google.golang.org/grpc#ClientConn.NewStream for how to avoid leaking the stream _, err = stream.CloseAndRecv() - if err != nil { - reportSQLDurationFailedHistogram.Observe(time.Since(start).Seconds()) - return err - } - reportSQLDurationSuccHistogram.Observe(time.Since(start).Seconds()) - return nil + return } // sendBatchPlanMeta sends a batch of SQL metas by stream. -func (r *SingleTargetDataSink) sendBatchPlanMeta(ctx context.Context, planMap *sync.Map) error { +func (r *SingleTargetDataSink) sendBatchPlanMeta(ctx context.Context, planMetas []*tipb.PlanMeta) (err error) { + if len(planMetas) == 0 { + return nil + } + start := time.Now() + sentCount := 0 + defer func() { + topSQLReportPlanCountHistogram.Observe(float64(sentCount)) + if err != nil { + reportPlanDurationFailedHistogram.Observe(time.Since(start).Seconds()) + } + reportPlanDurationSuccHistogram.Observe(time.Since(start).Seconds()) + }() + client := tipb.NewTopSQLAgentClient(r.conn) stream, err := client.ReportPlanMeta(ctx) if err != nil { return err } - cnt := 0 - planMap.Range(func(key, value interface{}) bool { - planDecoded, errDecode := r.decodePlan(value.(string)) - if errDecode != nil { - logutil.BgLogger().Warn("[top-sql] decode plan failed", zap.Error(errDecode)) - return true - } - cnt++ - planMeta := &tipb.PlanMeta{ - PlanDigest: []byte(key.(string)), - NormalizedPlan: planDecoded, - } - if err = stream.Send(planMeta); err != nil { - return false + + for _, meta := range planMetas { + if err = stream.Send(meta); err != nil { + return err } - return true - }) - // stream.Send return error - if err != nil { - return err + sentCount += 1 } - topSQLReportPlanCountHistogram.Observe(float64(cnt)) + + // See https://pkg.go.dev/google.golang.org/grpc#ClientConn.NewStream for how to avoid leaking the stream _, err = stream.CloseAndRecv() - if err != nil { - reportPlanDurationFailedHistogram.Observe(time.Since(start).Seconds()) - return err - } - reportPlanDurationSuccHistogram.Observe(time.Since(start).Seconds()) - return err + return } // tryEstablishConnection establishes the gRPC connection if connection is not established. diff --git a/util/topsql/topsql.go b/util/topsql/topsql.go index 250dabf66795b..209aaf7ad7fd8 100644 --- a/util/topsql/topsql.go +++ b/util/topsql/topsql.go @@ -23,6 +23,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/parser" + "github.com/pingcap/tidb/parser/terror" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/plancodec" @@ -47,14 +48,17 @@ var ( // SetupTopSQL sets up the top-sql worker. func SetupTopSQL() { - rc := reporter.NewSingleTargetDataSink(plancodec.DecodeNormalizedPlan) - globalTopSQLReport = reporter.NewRemoteTopSQLReporter(rc) + globalTopSQLReport = reporter.NewRemoteTopSQLReporter(plancodec.DecodeNormalizedPlan) + tracecpu.GlobalSQLCPUProfiler.SetCollector(globalTopSQLReport) tracecpu.GlobalSQLCPUProfiler.Run() - dataSinkRegHandle := globalTopSQLReport.DataSinkRegisterHandle() - pubsub := reporter.NewTopSQLPubSubService(plancodec.DecodeNormalizedPlan, dataSinkRegHandle) - globalTopSQLPubSubService = pubsub + // register single target datasink to reporter + singleTargetDataSink := reporter.NewSingleTargetDataSink() + err := globalTopSQLReport.DataSinkRegHandle().Register(singleTargetDataSink) + terror.MustNil(err) + + globalTopSQLPubSubService = reporter.NewTopSQLPubSubService(globalTopSQLReport.DataSinkRegHandle()) if len(config.GetGlobalConfig().TopSQL.ReceiverAddress) != 0 { variable.TopSQLVariable.InstanceEnable.Store(true) diff --git a/util/topsql/topsql_test.go b/util/topsql/topsql_test.go index 90140ae9d4da5..1b7692a802d40 100644 --- a/util/topsql/topsql_test.go +++ b/util/topsql/topsql_test.go @@ -120,10 +120,13 @@ func TestTopSQLReporter(t *testing.T) { conf.TopSQL.ReceiverAddress = server.Address() }) - dataSink := reporter.NewSingleTargetDataSink(mockPlanBinaryDecoderFunc) - report := reporter.NewRemoteTopSQLReporter(dataSink) + report := reporter.NewRemoteTopSQLReporter(mockPlanBinaryDecoderFunc) defer report.Close() + dataSink := reporter.NewSingleTargetDataSink() + err = report.DataSinkRegHandle().Register(dataSink) + require.NoError(t, err) + tracecpu.GlobalSQLCPUProfiler.SetCollector(&collectorWrapper{report}) reqs := []struct { sql string @@ -185,10 +188,10 @@ func TestTopSQLPubSub(t *testing.T) { variable.TopSQLVariable.MaxStatementCount.Store(200) variable.TopSQLVariable.ReportIntervalSeconds.Store(1) - report := reporter.NewRemoteTopSQLReporter() + report := reporter.NewRemoteTopSQLReporter(mockPlanBinaryDecoderFunc) defer report.Close() - pubsub := reporter.NewTopSQLPubSubService(mockPlanBinaryDecoderFunc, report.DataSinkRegisterHandle()) + pubsub := reporter.NewTopSQLPubSubService(report.DataSinkRegHandle()) server, err := mockServer.StartMockPubSubServer(pubsub) require.NoError(t, err) defer server.Stop() @@ -295,9 +298,9 @@ func TestTopSQLPubSub(t *testing.T) { } func TestTopSQLPubSubReporterStopBeforePubSub(t *testing.T) { - report := reporter.NewRemoteTopSQLReporter() + report := reporter.NewRemoteTopSQLReporter(mockPlanBinaryDecoderFunc) - pubsub := reporter.NewTopSQLPubSubService(mockPlanBinaryDecoderFunc, report.DataSinkRegisterHandle()) + pubsub := reporter.NewTopSQLPubSubService(report.DataSinkRegHandle()) server, err := mockServer.StartMockPubSubServer(pubsub) defer server.Stop() require.NoError(t, err) diff --git a/util/topsql/tracecpu/mock/mock.go b/util/topsql/tracecpu/mock/mock.go index 0c843a12cac83..e9b95bae93740 100644 --- a/util/topsql/tracecpu/mock/mock.go +++ b/util/topsql/tracecpu/mock/mock.go @@ -171,8 +171,8 @@ func (c *TopSQLCollector) WaitCollectCnt(count int64) { } } -// DataSinkRegisterHandle implements the interface. -func (c *TopSQLCollector) DataSinkRegisterHandle() reporter.DataSinkRegisterHandle { +// DataSinkRegHandle implements the interface. +func (c *TopSQLCollector) DataSinkRegHandle() reporter.DataSinkRegHandle { return nil } From c3e92da2217f8dcbb7a8b3c775f6b81ce1e34b49 Mon Sep 17 00:00:00 2001 From: Zhenchi Date: Thu, 9 Dec 2021 18:33:21 +0800 Subject: [PATCH 28/47] polish Signed-off-by: Zhenchi --- util/topsql/reporter/datasink.go | 4 ++-- util/topsql/reporter/pubsub.go | 10 +++++----- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/util/topsql/reporter/datasink.go b/util/topsql/reporter/datasink.go index 910bc1c198b5b..ebc6c31ec38c0 100644 --- a/util/topsql/reporter/datasink.go +++ b/util/topsql/reporter/datasink.go @@ -22,11 +22,11 @@ import ( type DataSink interface { Send(data reportData, timeout time.Duration) - // IsPaused indicates that DataSink is not expecting to receive records for now + // IsPaused indicates that the DataSink is not expecting to receive records for now // and may resume in the future. IsPaused() bool - // IsDown indicates that the client has been down and can be cleared. + // IsDown indicates that the DataSink has been down and can be cleared. // Note that: once a DataSink is down, it cannot go back to be up. IsDown() bool diff --git a/util/topsql/reporter/pubsub.go b/util/topsql/reporter/pubsub.go index ff573985e481c..20466347f1853 100644 --- a/util/topsql/reporter/pubsub.go +++ b/util/topsql/reporter/pubsub.go @@ -51,13 +51,13 @@ func (t *TopSQLPubSubService) Subscribe( _ *tipb.TopSQLSubRequest, stream tipb.TopSQLPubSub_SubscribeServer, ) error { - sc := newPubSubDataSink(stream) + ds := newPubSubDataSink(stream) - if err := t.dataSinkRegHandle.Register(sc); err != nil { + if err := t.dataSinkRegHandle.Register(ds); err != nil { return err } - sc.run() + ds.run() return nil } @@ -131,14 +131,14 @@ func (s *pubSubDataSink) run() { case <-doneCh: if err != nil { logutil.BgLogger().Warn( - "[top-sql] pubsub data sink failed to send data to subscriber", + "[top-sql] pubsub datasink failed to send data to subscriber", zap.Error(err), ) return } case <-ctx.Done(): logutil.BgLogger().Warn( - "[top-sql] pubsub data sink failed to send data to subscriber due to timeout", + "[top-sql] pubsub datasink failed to send data to subscriber due to timeout", zap.Duration("timeout", task.timeout), ) return From 0d50283196eda26f8eb63c51f9f2c064994fbac3 Mon Sep 17 00:00:00 2001 From: Zhenchi Date: Thu, 9 Dec 2021 19:03:38 +0800 Subject: [PATCH 29/47] remove the global service & timeout -> deadline Signed-off-by: Zhenchi --- util/topsql/reporter/datasink.go | 2 +- util/topsql/reporter/pubsub.go | 8 +-- util/topsql/reporter/reporter.go | 79 ++++++++++++++------------- util/topsql/reporter/single_target.go | 10 ++-- util/topsql/topsql.go | 10 ++-- 5 files changed, 54 insertions(+), 55 deletions(-) diff --git a/util/topsql/reporter/datasink.go b/util/topsql/reporter/datasink.go index ebc6c31ec38c0..7b824a2bca352 100644 --- a/util/topsql/reporter/datasink.go +++ b/util/topsql/reporter/datasink.go @@ -20,7 +20,7 @@ import ( // DataSink sends data to the target server. type DataSink interface { - Send(data reportData, timeout time.Duration) + Send(data reportData, deadline time.Time) // IsPaused indicates that the DataSink is not expecting to receive records for now // and may resume in the future. diff --git a/util/topsql/reporter/pubsub.go b/util/topsql/reporter/pubsub.go index 20466347f1853..5aa3fd71e5d9b 100644 --- a/util/topsql/reporter/pubsub.go +++ b/util/topsql/reporter/pubsub.go @@ -79,13 +79,13 @@ func newPubSubDataSink( var _ DataSink = &pubSubDataSink{} -func (s *pubSubDataSink) Send(data reportData, timeout time.Duration) { +func (s *pubSubDataSink) Send(data reportData, deadline time.Time) { if s.IsDown() { return } select { - case s.sendTaskCh <- sendTask{data: data, timeout: timeout}: + case s.sendTaskCh <- sendTask{data: data, deadline: deadline}: // sent successfully default: ignoreReportChannelFullCounter.Inc() @@ -109,7 +109,7 @@ func (s *pubSubDataSink) run() { defer s.isDown.Store(true) for task := range s.sendTaskCh { - ctx, cancel := context.WithTimeout(context.Background(), task.timeout) + ctx, cancel := context.WithDeadline(context.Background(), task.deadline) start := time.Now() var err error @@ -139,7 +139,7 @@ func (s *pubSubDataSink) run() { case <-ctx.Done(): logutil.BgLogger().Warn( "[top-sql] pubsub datasink failed to send data to subscriber due to timeout", - zap.Duration("timeout", task.timeout), + zap.Time("deadline", task.deadline), ) return } diff --git a/util/topsql/reporter/reporter.go b/util/topsql/reporter/reporter.go index 30d260477a382..c00c155b55ad7 100644 --- a/util/topsql/reporter/reporter.go +++ b/util/topsql/reporter/reporter.go @@ -284,8 +284,8 @@ func addEvictedCPUTime(collectTarget map[string]*dataPoints, timestamp uint64, t others.CPUTimeMsTotal += uint64(totalCPUTimeMs) } -// addEvictedIntoSortedDataPoints adds the evict dataPoints into others. -// Attention, this function depend on others dataPoints is sorted, and this function will modify the evict dataPoints +// 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 { @@ -540,6 +540,42 @@ func (d *reportData) hasData() bool { return len(d.cpuTimeRecords) != 0 || len(d.sqlMetas) != 0 || len(d.planMetas) != 0 } +func buildReportData(records []*dataPoints, sqlMap *sync.Map, planMap *sync.Map, decodePlan planBinaryDecodeFunc) (res reportData) { + for _, record := range records { + res.cpuTimeRecords = append(res.cpuTimeRecords, &tipb.CPUTimeRecord{ + RecordListTimestampSec: record.TimestampList, + RecordListCpuTimeMs: record.CPUTimeMsList, + 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 := 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 +} + // 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) @@ -600,42 +636,6 @@ func getTopN(collected collectedData) (records []*dataPoints, sqlMap *sync.Map, return } -func buildReportData(records []*dataPoints, sqlMap *sync.Map, planMap *sync.Map, decodePlan planBinaryDecodeFunc) (res reportData) { - for _, record := range records { - res.cpuTimeRecords = append(res.cpuTimeRecords, &tipb.CPUTimeRecord{ - RecordListTimestampSec: record.TimestampList, - RecordListCpuTimeMs: record.CPUTimeMsList, - 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 := 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 -} - func (tsr *RemoteTopSQLReporter) doReport(data reportData) { defer util.Recover("top-sql", "doReport", nil, false) @@ -653,8 +653,9 @@ func (tsr *RemoteTopSQLReporter) doReport(data reportData) { } }) + deadline := time.Now().Add(timeout) for i := range tsr.dataSinks { - tsr.dataSinks[i].Send(data, timeout) + tsr.dataSinks[i].Send(data, deadline) } } diff --git a/util/topsql/reporter/single_target.go b/util/topsql/reporter/single_target.go index 2a05869c0bc97..1fa31f08a858f 100644 --- a/util/topsql/reporter/single_target.go +++ b/util/topsql/reporter/single_target.go @@ -37,8 +37,8 @@ type SingleTargetDataSink struct { } type sendTask struct { - data reportData - timeout time.Duration + data reportData + deadline time.Time } // NewSingleTargetDataSink returns a new SingleTargetDataSink @@ -59,7 +59,7 @@ func (r *SingleTargetDataSink) run() { continue } - ctx, cancel := context.WithTimeout(context.Background(), task.timeout) + ctx, cancel := context.WithDeadline(context.Background(), task.deadline) start := time.Now() err := r.doSend(ctx, targetRPCAddr, task.data) cancel() @@ -75,9 +75,9 @@ func (r *SingleTargetDataSink) run() { var _ DataSink = &SingleTargetDataSink{} // Send implements the DataSink interface. -func (r *SingleTargetDataSink) Send(data reportData, timeout time.Duration) { +func (r *SingleTargetDataSink) Send(data reportData, deadline time.Time) { select { - case r.sendTaskCh <- sendTask{data: data, timeout: timeout}: + case r.sendTaskCh <- sendTask{data: data, deadline: deadline}: // sent successfully default: ignoreReportChannelFullCounter.Inc() diff --git a/util/topsql/topsql.go b/util/topsql/topsql.go index 209aaf7ad7fd8..30e2923c9b406 100644 --- a/util/topsql/topsql.go +++ b/util/topsql/topsql.go @@ -42,8 +42,7 @@ const ( ) var ( - globalTopSQLReport reporter.TopSQLReporter - globalTopSQLPubSubService tipb.TopSQLPubSubServer + globalTopSQLReport reporter.TopSQLReporter ) // SetupTopSQL sets up the top-sql worker. @@ -58,8 +57,6 @@ func SetupTopSQL() { err := globalTopSQLReport.DataSinkRegHandle().Register(singleTargetDataSink) terror.MustNil(err) - globalTopSQLPubSubService = reporter.NewTopSQLPubSubService(globalTopSQLReport.DataSinkRegHandle()) - if len(config.GetGlobalConfig().TopSQL.ReceiverAddress) != 0 { variable.TopSQLVariable.InstanceEnable.Store(true) } @@ -67,8 +64,9 @@ func SetupTopSQL() { // RegisterPubSubServer registers TopSQLPubSubService to the given gRPC server. func RegisterPubSubServer(s *grpc.Server) { - if globalTopSQLPubSubService != nil { - tipb.RegisterTopSQLPubSubServer(s, globalTopSQLPubSubService) + if globalTopSQLReport != nil { + service := reporter.NewTopSQLPubSubService(globalTopSQLReport.DataSinkRegHandle()) + tipb.RegisterTopSQLPubSubServer(s, service) } } From bca0e17f4de52df16f44468b11e095a4beebef2b Mon Sep 17 00:00:00 2001 From: Zhenchi Date: Thu, 9 Dec 2021 19:41:12 +0800 Subject: [PATCH 30/47] remove session var Signed-off-by: Zhenchi --- ddl/ddl_worker.go | 4 ++-- distsql/request_builder.go | 3 ++- executor/adapter.go | 2 +- executor/executor.go | 4 ++-- executor/prepared.go | 3 +-- executor/update.go | 4 ++-- server/conn.go | 3 ++- server/conn_stmt.go | 5 ++--- session/session.go | 10 +++++----- sessionctx/variable/sysvar.go | 2 +- sessionctx/variable/tidb_vars.go | 8 -------- util/topsql/reporter/reporter.go | 2 +- util/topsql/topsql.go | 10 ++++++---- util/topsql/topsql_test.go | 16 ++++++++-------- util/topsql/tracecpu/main_test.go | 5 ++--- util/topsql/tracecpu/profile.go | 23 +++++++++++++++++------ 16 files changed, 54 insertions(+), 50 deletions(-) diff --git a/ddl/ddl_worker.go b/ddl/ddl_worker.go index cd7cc44c75c39..91d3de8e688ce 100644 --- a/ddl/ddl_worker.go +++ b/ddl/ddl_worker.go @@ -17,7 +17,7 @@ package ddl import ( "context" "fmt" - "strconv" + "strconv" "sync" "sync/atomic" "time" @@ -477,7 +477,7 @@ func newMetaWithQueueTp(txn kv.Transaction, tp workerType) *meta.Meta { } func (w *worker) setDDLLabelForTopSQL(job *model.Job) { - if !variable.TopSQLInstanceEnabled() || job == nil { + if !topsql.InstanceEnabled() || job == nil { return } diff --git a/distsql/request_builder.go b/distsql/request_builder.go index 15deada49484f..d826bfc28fe58 100644 --- a/distsql/request_builder.go +++ b/distsql/request_builder.go @@ -35,6 +35,7 @@ import ( "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/memory" "github.com/pingcap/tidb/util/ranger" + "github.com/pingcap/tidb/util/topsql" "github.com/pingcap/tipb/go-tipb" ) @@ -291,7 +292,7 @@ func (builder *RequestBuilder) SetFromInfoSchema(pis interface{}) *RequestBuilde // SetResourceGroupTagger sets the request resource group tagger. func (builder *RequestBuilder) SetResourceGroupTagger(sc *stmtctx.StatementContext) *RequestBuilder { - if variable.TopSQLInstanceEnabled() { + if topsql.InstanceEnabled() { builder.Request.ResourceGroupTagger = sc.GetResourceGroupTagger() } return builder diff --git a/executor/adapter.go b/executor/adapter.go index 93f99dec2c809..40929db7b23ce 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -314,7 +314,7 @@ func (a *ExecStmt) RebuildPlan(ctx context.Context) (int64, error) { } func (a *ExecStmt) setPlanLabelForTopSQL(ctx context.Context) context.Context { - if a.Plan == nil || !variable.TopSQLInstanceEnabled() { + if a.Plan == nil || !topsql.InstanceEnabled() { return ctx } vars := a.Ctx.GetSessionVars() diff --git a/executor/executor.go b/executor/executor.go index 6c6626e3374b3..f7ab1869685d8 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -1730,7 +1730,7 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) { goCtx = pprof.WithLabels(goCtx, pprof.Labels("sql", util.QueryStrForLog(prepareStmt.NormalizedSQL))) pprof.SetGoroutineLabels(goCtx) } - if variable.TopSQLInstanceEnabled() && prepareStmt.SQLDigest != nil { + if topsql.InstanceEnabled() && prepareStmt.SQLDigest != nil { topsql.AttachSQLInfo(goCtx, prepareStmt.NormalizedSQL, prepareStmt.SQLDigest, "", nil, vars.InRestrictedSQL) } } @@ -1910,7 +1910,7 @@ func FillVirtualColumnValue(virtualRetTypes []*types.FieldType, virtualColumnInd } func setResourceGroupTaggerForTxn(sc *stmtctx.StatementContext, snapshot kv.Snapshot) { - if snapshot != nil && variable.TopSQLInstanceEnabled() { + if snapshot != nil && topsql.InstanceEnabled() { snapshot.SetOption(kv.ResourceGroupTagger, sc.GetResourceGroupTagger()) } } diff --git a/executor/prepared.go b/executor/prepared.go index c35f93d530aff..8c91b02a411eb 100644 --- a/executor/prepared.go +++ b/executor/prepared.go @@ -30,7 +30,6 @@ import ( "github.com/pingcap/tidb/planner" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/types" driver "github.com/pingcap/tidb/types/parser_driver" "github.com/pingcap/tidb/util" @@ -194,7 +193,7 @@ func (e *PrepareExec) Next(ctx context.Context, req *chunk.Chunk) error { SchemaVersion: ret.InfoSchema.SchemaMetaVersion(), } normalizedSQL, digest := parser.NormalizeDigest(prepared.Stmt.Text()) - if variable.TopSQLInstanceEnabled() { + if topsql.InstanceEnabled() { ctx = topsql.AttachSQLInfo(ctx, normalizedSQL, digest, "", nil, vars.InRestrictedSQL) } diff --git a/executor/update.go b/executor/update.go index 44840d9aa14a2..ea5230632283a 100644 --- a/executor/update.go +++ b/executor/update.go @@ -26,12 +26,12 @@ import ( "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" plannercore "github.com/pingcap/tidb/planner/core" - "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/execdetails" "github.com/pingcap/tidb/util/memory" + "github.com/pingcap/tidb/util/topsql" "github.com/tikv/client-go/v2/txnkv/txnsnapshot" ) @@ -271,7 +271,7 @@ func (e *UpdateExec) updateRows(ctx context.Context) (int, error) { txn.GetSnapshot().SetOption(kv.CollectRuntimeStats, e.stats.SnapshotRuntimeStats) } } - if variable.TopSQLInstanceEnabled() { + if topsql.InstanceEnabled() { txn, err := e.ctx.Txn(true) if err == nil { txn.SetOption(kv.ResourceGroupTagger, e.ctx.GetSessionVars().StmtCtx.GetResourceGroupTagger()) diff --git a/server/conn.go b/server/conn.go index d2ea9e16c658e..71a3dba53390b 100644 --- a/server/conn.go +++ b/server/conn.go @@ -85,6 +85,7 @@ import ( "github.com/pingcap/tidb/util/hack" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/memory" + "github.com/pingcap/tidb/util/topsql" "github.com/prometheus/client_golang/prometheus" "github.com/tikv/client-go/v2/util" "go.uber.org/zap" @@ -1244,7 +1245,7 @@ func (cc *clientConn) dispatch(ctx context.Context, data []byte) error { cc.lastPacket = data cmd := data[0] data = data[1:] - if variable.TopSQLInstanceEnabled() { + if topsql.InstanceEnabled() { defer pprof.SetGoroutineLabels(ctx) } if variable.EnablePProfSQLCPU.Load() { diff --git a/server/conn_stmt.go b/server/conn_stmt.go index 203cbc5f2da79..b45799d73b8dd 100644 --- a/server/conn_stmt.go +++ b/server/conn_stmt.go @@ -50,7 +50,6 @@ import ( "github.com/pingcap/tidb/parser/terror" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx/stmtctx" - "github.com/pingcap/tidb/sessionctx/variable" storeerr "github.com/pingcap/tidb/store/driver/error" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/execdetails" @@ -127,7 +126,7 @@ func (cc *clientConn) handleStmtExecute(ctx context.Context, data []byte) (err e stmtID := binary.LittleEndian.Uint32(data[0:4]) pos += 4 - if variable.TopSQLInstanceEnabled() { + if topsql.InstanceEnabled() { preparedStmt, _ := cc.preparedStmtID2CachePreparedStmt(stmtID) if preparedStmt != nil && preparedStmt.SQLDigest != nil { ctx = topsql.AttachSQLInfo(ctx, preparedStmt.NormalizedSQL, preparedStmt.SQLDigest, "", nil, false) @@ -273,7 +272,7 @@ func (cc *clientConn) handleStmtFetch(ctx context.Context, data []byte) (err err return errors.Annotate(mysql.NewErr(mysql.ErrUnknownStmtHandler, strconv.FormatUint(uint64(stmtID), 10), "stmt_fetch"), cc.preparedStmt2String(stmtID)) } - if variable.TopSQLInstanceEnabled() { + if topsql.InstanceEnabled() { prepareObj, _ := cc.preparedStmtID2CachePreparedStmt(stmtID) if prepareObj != nil && prepareObj.SQLDigest != nil { ctx = topsql.AttachSQLInfo(ctx, prepareObj.NormalizedSQL, prepareObj.SQLDigest, "", nil, false) diff --git a/session/session.go b/session/session.go index 4b35b9ea97d63..c2766607dac65 100644 --- a/session/session.go +++ b/session/session.go @@ -24,7 +24,7 @@ import ( "crypto/tls" "encoding/json" "fmt" - "runtime/pprof" + "runtime/pprof" "runtime/trace" "strconv" "strings" @@ -1272,7 +1272,7 @@ func (s *session) ExecuteInternal(ctx context.Context, sql string, args ...inter s.sessionVars.InRestrictedSQL = true defer func() { s.sessionVars.InRestrictedSQL = origin - if variable.TopSQLInstanceEnabled() { + if topsql.InstanceEnabled() { // Restore the goroutine label by using the original ctx after execution is finished. pprof.SetGoroutineLabels(ctx) } @@ -1412,7 +1412,7 @@ func (s *session) ParseWithParams(ctx context.Context, sql string, args ...inter for _, warn := range warns { s.sessionVars.StmtCtx.AppendWarning(util.SyntaxWarn(warn)) } - if variable.TopSQLInstanceEnabled() { + if topsql.InstanceEnabled() { normalized, digest := parser.NormalizeDigest(sql) if digest != nil { // Reset the goroutine label when internal sql execute finish. @@ -1426,7 +1426,7 @@ func (s *session) ParseWithParams(ctx context.Context, sql string, args ...inter // ExecRestrictedStmt implements RestrictedSQLExecutor interface. func (s *session) ExecRestrictedStmt(ctx context.Context, stmtNode ast.StmtNode, opts ...sqlexec.OptionFuncAlias) ( []chunk.Row, []*ast.ResultField, error) { - if variable.TopSQLInstanceEnabled() { + if topsql.InstanceEnabled() { defer pprof.SetGoroutineLabels(ctx) } var execOption sqlexec.ExecOption @@ -1533,7 +1533,7 @@ func (s *session) ExecuteStmt(ctx context.Context, stmtNode ast.StmtNode) (sqlex return nil, err } normalizedSQL, digest := s.sessionVars.StmtCtx.SQLDigest() - if variable.TopSQLInstanceEnabled() { + if topsql.InstanceEnabled() { ctx = topsql.AttachSQLInfo(ctx, normalizedSQL, digest, "", nil, s.sessionVars.InRestrictedSQL) } diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 4c7ed30860258..b4062515425bf 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -1250,7 +1250,7 @@ var defaultSysVars = []*SysVar{ return nil }}, // variable for top SQL feature. - {Scope: ScopeGlobal, Name: TiDBEnableTopSQL, Value: BoolToOnOff(DefTiDBTopSQLEnable), Type: TypeBool, Hidden: true, AllowEmpty: true, GlobalConfigName: GlobalConfigEnableTopSQL}, + {Scope: ScopeGlobal, Name: TiDBEnableTopSQL, Type: TypeBool, Hidden: true, AllowEmpty: true, GlobalConfigName: GlobalConfigEnableTopSQL}, {Scope: ScopeGlobal, Name: TiDBTopSQLPrecisionSeconds, Value: strconv.Itoa(DefTiDBTopSQLPrecisionSeconds), Type: TypeInt, Hidden: true, MinValue: 1, MaxValue: math.MaxInt64, GetGlobal: func(s *SessionVars) (string, error) { return strconv.FormatInt(TopSQLVariable.PrecisionSeconds.Load(), 10), nil }, SetGlobal: func(vars *SessionVars, s string) error { diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 3c83a5838576d..b69777ebb2279 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -763,7 +763,6 @@ const ( DefTiDBTrackAggregateMemoryUsage = true DefTiDBEnableExchangePartition = false DefCTEMaxRecursionDepth = 1000 - DefTiDBTopSQLEnable = false DefTiDBTopSQLPrecisionSeconds = 1 DefTiDBTopSQLMaxStatementCount = 200 DefTiDBTopSQLMaxCollect = 10000 @@ -802,7 +801,6 @@ var ( DefExecutorConcurrency = 5 MemoryUsageAlarmRatio = atomic.NewFloat64(config.GetGlobalConfig().Performance.MemoryUsageAlarmRatio) TopSQLVariable = TopSQL{ - InstanceEnable: atomic.NewBool(DefTiDBTopSQLEnable), PrecisionSeconds: atomic.NewInt64(DefTiDBTopSQLPrecisionSeconds), MaxStatementCount: atomic.NewInt64(DefTiDBTopSQLMaxStatementCount), MaxCollect: atomic.NewInt64(DefTiDBTopSQLMaxCollect), @@ -816,8 +814,6 @@ var ( // TopSQL is the variable for control top sql feature. type TopSQL struct { - // InstanceEnable represents if TopSQL is enabled on the current instance or not. - InstanceEnable *atomic.Bool // The refresh interval of top-sql. PrecisionSeconds *atomic.Int64 // The maximum number of statements kept in memory. @@ -828,7 +824,3 @@ type TopSQL struct { ReportIntervalSeconds *atomic.Int64 } -// TopSQLInstanceEnabled is used to check if TopSQL is enabled on the current instance. -func TopSQLInstanceEnabled() bool { - return TopSQLVariable.InstanceEnable.Load() -} diff --git a/util/topsql/reporter/reporter.go b/util/topsql/reporter/reporter.go index c00c155b55ad7..3031a616c011b 100644 --- a/util/topsql/reporter/reporter.go +++ b/util/topsql/reporter/reporter.go @@ -402,7 +402,7 @@ out: } } runningCnt := len(tsr.dataSinks) - pendingCnt - variable.TopSQLVariable.InstanceEnable.Store(runningCnt > 0) + tracecpu.GlobalSQLCPUProfiler.SetTopSQLEnabled(runningCnt > 0) } func encodeKey(buf *bytes.Buffer, sqlDigest, planDigest []byte) string { diff --git a/util/topsql/topsql.go b/util/topsql/topsql.go index 30e2923c9b406..80990e5350d4e 100644 --- a/util/topsql/topsql.go +++ b/util/topsql/topsql.go @@ -24,7 +24,6 @@ import ( "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/terror" - "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/plancodec" "github.com/pingcap/tidb/util/topsql/reporter" @@ -57,9 +56,12 @@ func SetupTopSQL() { err := globalTopSQLReport.DataSinkRegHandle().Register(singleTargetDataSink) terror.MustNil(err) - if len(config.GetGlobalConfig().TopSQL.ReceiverAddress) != 0 { - variable.TopSQLVariable.InstanceEnable.Store(true) - } + enable := len(config.GetGlobalConfig().TopSQL.ReceiverAddress) != 0 + tracecpu.GlobalSQLCPUProfiler.SetTopSQLEnabled(enable) +} + +func InstanceEnabled() bool { + return tracecpu.GlobalSQLCPUProfiler.GetTopSQLEnabled() } // RegisterPubSubServer registers TopSQLPubSubService to the given gRPC server. diff --git a/util/topsql/topsql_test.go b/util/topsql/topsql_test.go index 1b7692a802d40..3819630fc2507 100644 --- a/util/topsql/topsql_test.go +++ b/util/topsql/topsql_test.go @@ -39,8 +39,8 @@ type collectorWrapper struct { } func TestTopSQLCPUProfile(t *testing.T) { - variable.TopSQLVariable.InstanceEnable.Store(true) - defer variable.TopSQLVariable.InstanceEnable.Store(false) + setTopSQLEnable(true) + defer setTopSQLEnable(false) collector := mock.NewTopSQLCollector() tracecpu.GlobalSQLCPUProfiler.SetCollector(&collectorWrapper{collector}) @@ -90,21 +90,21 @@ func TestTopSQLCPUProfile(t *testing.T) { func TestIsEnabled(t *testing.T) { setTopSQLEnable(false) - require.False(t, tracecpu.GlobalSQLCPUProfiler.IsEnabled()) + require.False(t, tracecpu.GlobalSQLCPUProfiler.ShouldProfile()) setTopSQLEnable(true) err := tracecpu.StartCPUProfile(bytes.NewBuffer(nil)) require.NoError(t, err) - require.True(t, tracecpu.GlobalSQLCPUProfiler.IsEnabled()) + require.True(t, tracecpu.GlobalSQLCPUProfiler.ShouldProfile()) setTopSQLEnable(false) - require.True(t, tracecpu.GlobalSQLCPUProfiler.IsEnabled()) + require.True(t, tracecpu.GlobalSQLCPUProfiler.ShouldProfile()) err = tracecpu.StopCPUProfile() require.NoError(t, err) setTopSQLEnable(false) - require.False(t, tracecpu.GlobalSQLCPUProfiler.IsEnabled()) + require.False(t, tracecpu.GlobalSQLCPUProfiler.ShouldProfile()) setTopSQLEnable(true) - require.True(t, tracecpu.GlobalSQLCPUProfiler.IsEnabled()) + require.True(t, tracecpu.GlobalSQLCPUProfiler.ShouldProfile()) } func mockPlanBinaryDecoderFunc(plan string) (string, error) { @@ -358,7 +358,7 @@ func TestMaxSQLAndPlanTest(t *testing.T) { } func setTopSQLEnable(enabled bool) { - variable.TopSQLVariable.InstanceEnable.Store(enabled) + tracecpu.GlobalSQLCPUProfiler.SetTopSQLEnabled(enabled) } func mockExecuteSQL(sql, plan string) { diff --git a/util/topsql/tracecpu/main_test.go b/util/topsql/tracecpu/main_test.go index dfbe2aca633df..c93e76eacf011 100644 --- a/util/topsql/tracecpu/main_test.go +++ b/util/topsql/tracecpu/main_test.go @@ -30,7 +30,6 @@ import ( func TestMain(m *testing.M) { testbridge.WorkaroundGoCheckFlags() - variable.TopSQLVariable.InstanceEnable.Store(false) config.UpdateGlobal(func(conf *config.Config) { conf.TopSQL.ReceiverAddress = "mock" }) @@ -52,10 +51,10 @@ func TestPProfCPUProfile(t *testing.T) { err := tracecpu.StartCPUProfile(buf) require.NoError(t, err) // enable top sql. - variable.TopSQLVariable.InstanceEnable.Store(true) + tracecpu.GlobalSQLCPUProfiler.SetTopSQLEnabled(true) err = tracecpu.StopCPUProfile() require.NoError(t, err) _, err = profile.Parse(buf) require.NoError(t, err) - variable.TopSQLVariable.InstanceEnable.Store(false) + tracecpu.GlobalSQLCPUProfiler.SetTopSQLEnabled(false) } diff --git a/util/topsql/tracecpu/profile.go b/util/topsql/tracecpu/profile.go index 2d5d294d143d0..029a3c7753363 100644 --- a/util/topsql/tracecpu/profile.go +++ b/util/topsql/tracecpu/profile.go @@ -33,6 +33,7 @@ import ( "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/hack" "github.com/pingcap/tidb/util/logutil" + atomic2 "go.uber.org/atomic" "go.uber.org/zap" ) @@ -70,7 +71,9 @@ type sqlCPUProfiler struct { sync.Mutex ept *exportProfileTask } - collector atomic.Value + + topSQLEnabled atomic2.Bool + collector atomic.Value } var ( @@ -107,10 +110,18 @@ func (sp *sqlCPUProfiler) GetCollector() Collector { return c } +func (sp *sqlCPUProfiler) SetTopSQLEnabled(v bool) { + sp.topSQLEnabled.Store(v) +} + +func (sp *sqlCPUProfiler) GetTopSQLEnabled() bool { + return sp.topSQLEnabled.Load() +} + func (sp *sqlCPUProfiler) startCPUProfileWorker() { defer util.Recover("top-sql", "profileWorker", nil, false) for { - if sp.IsEnabled() { + if sp.ShouldProfile() { sp.doCPUProfile() } else { time.Sleep(time.Second) @@ -278,16 +289,16 @@ func (sp *sqlCPUProfiler) hasExportProfileTask() bool { return has } -// IsEnabled return true if it is(should be) enabled. It exports for tests. -func (sp *sqlCPUProfiler) IsEnabled() bool { - return variable.TopSQLInstanceEnabled() || sp.hasExportProfileTask() +// ShouldProfile return true if it's required to profile. It exports for tests. +func (sp *sqlCPUProfiler) ShouldProfile() bool { + return sp.topSQLEnabled.Load() || sp.hasExportProfileTask() } // StartCPUProfile same like pprof.StartCPUProfile. // Because the GlobalSQLCPUProfiler keep calling pprof.StartCPUProfile to fetch SQL cpu stats, other place (such pprof profile HTTP API handler) call pprof.StartCPUProfile will be failed, // other place should call tracecpu.StartCPUProfile instead of pprof.StartCPUProfile. func StartCPUProfile(w io.Writer) error { - if GlobalSQLCPUProfiler.IsEnabled() { + if GlobalSQLCPUProfiler.ShouldProfile() { return GlobalSQLCPUProfiler.startExportCPUProfile(w) } return pprof.StartCPUProfile(w) From 5f09f4be3b02c137ec185a8e145ac1bde087a55c Mon Sep 17 00:00:00 2001 From: Zhenchi Date: Thu, 9 Dec 2021 20:20:29 +0800 Subject: [PATCH 31/47] fix Signed-off-by: Zhenchi --- ddl/ddl_worker.go | 2 +- executor/executor_test.go | 8 +++----- server/tidb_test.go | 9 ++------- session/session.go | 2 +- sessionctx/variable/tidb_vars.go | 1 - util/topsql/topsql.go | 1 + 6 files changed, 8 insertions(+), 15 deletions(-) diff --git a/ddl/ddl_worker.go b/ddl/ddl_worker.go index 91d3de8e688ce..c1ea7303c531d 100644 --- a/ddl/ddl_worker.go +++ b/ddl/ddl_worker.go @@ -17,7 +17,7 @@ package ddl import ( "context" "fmt" - "strconv" + "strconv" "sync" "sync/atomic" "time" diff --git a/executor/executor_test.go b/executor/executor_test.go index 8702615a14d82..0736c6bf09909 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -82,6 +82,7 @@ import ( "github.com/pingcap/tidb/util/testkit" "github.com/pingcap/tidb/util/testutil" "github.com/pingcap/tidb/util/timeutil" + "github.com/pingcap/tidb/util/topsql/tracecpu" "github.com/pingcap/tipb/go-tipb" "github.com/stretchr/testify/require" "github.com/tikv/client-go/v2/oracle" @@ -8718,11 +8719,8 @@ func (s *testResourceTagSuite) TestResourceGroupTag(c *C) { tk.MustExec("create table t(a int, b int, unique index idx(a));") tbInfo := testGetTableByName(c, tk.Se, "test", "t") - variable.TopSQLVariable.InstanceEnable.Store(true) - defer variable.TopSQLVariable.InstanceEnable.Store(false) - - c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/mockstore/unistore/unistoreRPCClientSendHook", `return(true)`), IsNil) - defer failpoint.Disable("github.com/pingcap/tidb/store/mockstore/unistore/unistoreRPCClientSendHook") + tracecpu.GlobalSQLCPUProfiler.SetTopSQLEnabled(true) + defer tracecpu.GlobalSQLCPUProfiler.SetTopSQLEnabled(false) var sqlDigest, planDigest *parser.Digest var tagLabel tipb.ResourceGroupTagLabel diff --git a/server/tidb_test.go b/server/tidb_test.go index f59cb0930e9e2..70d466853c76a 100644 --- a/server/tidb_test.go +++ b/server/tidb_test.go @@ -44,7 +44,6 @@ import ( "github.com/pingcap/tidb/parser" tmysql "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/session" - "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/util" @@ -1315,10 +1314,10 @@ func TestTopSQLCPUProfile(t *testing.T) { require.NoError(t, err) }() - variable.TopSQLVariable.InstanceEnable.Store(true) - defer variable.TopSQLVariable.InstanceEnable.Store(false) collector := mockTopSQLTraceCPU.NewTopSQLCollector() tracecpu.GlobalSQLCPUProfiler.SetCollector(&collectorWrapper{collector}) + tracecpu.GlobalSQLCPUProfiler.SetTopSQLEnabled(true) + defer tracecpu.GlobalSQLCPUProfiler.SetTopSQLEnabled(false) dbt := testkit.NewDBTestKit(t, db) dbt.MustExec("drop database if exists topsql") @@ -1327,10 +1326,6 @@ func TestTopSQLCPUProfile(t *testing.T) { dbt.MustExec("create table t (a int auto_increment, b int, unique index idx(a));") dbt.MustExec("create table t1 (a int auto_increment, b int, unique index idx(a));") dbt.MustExec("create table t2 (a int auto_increment, b int, unique index idx(a));") - config.UpdateGlobal(func(conf *config.Config) { - conf.TopSQL.ReceiverAddress = "127.0.0.1:4001" - }) - dbt.MustExec("set @@global.tidb_top_sql_precision_seconds=1;") dbt.MustExec("set @@global.tidb_txn_mode = 'pessimistic'") // Test case 1: DML query: insert/update/replace/delete/select diff --git a/session/session.go b/session/session.go index c2766607dac65..d033bef25260b 100644 --- a/session/session.go +++ b/session/session.go @@ -24,7 +24,7 @@ import ( "crypto/tls" "encoding/json" "fmt" - "runtime/pprof" + "runtime/pprof" "runtime/trace" "strconv" "strings" diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index b69777ebb2279..14e5d8314f17a 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -823,4 +823,3 @@ type TopSQL struct { // The report data interval of top-sql. ReportIntervalSeconds *atomic.Int64 } - diff --git a/util/topsql/topsql.go b/util/topsql/topsql.go index 80990e5350d4e..ed97a91387092 100644 --- a/util/topsql/topsql.go +++ b/util/topsql/topsql.go @@ -60,6 +60,7 @@ func SetupTopSQL() { tracecpu.GlobalSQLCPUProfiler.SetTopSQLEnabled(enable) } +// InstanceEnabled is used to check if TopSQL is enabled on the current instance. func InstanceEnabled() bool { return tracecpu.GlobalSQLCPUProfiler.GetTopSQLEnabled() } From 2909a57668398d7fefdb454df5022bf16af5a7c0 Mon Sep 17 00:00:00 2001 From: Zhenchi Date: Thu, 9 Dec 2021 20:40:49 +0800 Subject: [PATCH 32/47] fix Signed-off-by: Zhenchi --- domain/globalconfigsync/globalconfig_test.go | 3 --- sessionctx/variable/sysvar.go | 5 ++++- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/domain/globalconfigsync/globalconfig_test.go b/domain/globalconfigsync/globalconfig_test.go index 3f097189387aa..3a81b1a38453a 100644 --- a/domain/globalconfigsync/globalconfig_test.go +++ b/domain/globalconfigsync/globalconfig_test.go @@ -69,7 +69,6 @@ func TestStoreGlobalConfig(t *testing.T) { domain.GetGlobalConfigSyncer().SetEtcdClient(cluster.RandClient()) tk := testkit.NewTestKit(t, store) - tk.MustQuery("select @@global.tidb_enable_top_sql;").Check(testkit.Rows("0")) tk.MustExec("set @@global.tidb_enable_top_sql=1;") for i := 0; i < 20; i++ { @@ -86,8 +85,6 @@ func TestStoreGlobalConfig(t *testing.T) { require.Equal(t, len(resp.Kvs), 1) require.Equal(t, resp.Kvs[0].Key, []byte("/global/config/enable_resource_metering")) require.Equal(t, resp.Kvs[0].Value, []byte("true")) - - tk.MustQuery("select @@global.tidb_enable_top_sql;").Check(testkit.Rows("1")) return } diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index b4062515425bf..af70bb2461504 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -1250,7 +1250,10 @@ var defaultSysVars = []*SysVar{ return nil }}, // variable for top SQL feature. - {Scope: ScopeGlobal, Name: TiDBEnableTopSQL, Type: TypeBool, Hidden: true, AllowEmpty: true, GlobalConfigName: GlobalConfigEnableTopSQL}, + {Scope: ScopeGlobal, Name: TiDBEnableTopSQL, Value: Off, Type: TypeBool, Hidden: true, GetGlobal: func(sessionVars *SessionVars) (string, error) { + // TODO(zhongzc): implement read from global config + return Off, nil + }, GlobalConfigName: GlobalConfigEnableTopSQL}, {Scope: ScopeGlobal, Name: TiDBTopSQLPrecisionSeconds, Value: strconv.Itoa(DefTiDBTopSQLPrecisionSeconds), Type: TypeInt, Hidden: true, MinValue: 1, MaxValue: math.MaxInt64, GetGlobal: func(s *SessionVars) (string, error) { return strconv.FormatInt(TopSQLVariable.PrecisionSeconds.Load(), 10), nil }, SetGlobal: func(vars *SessionVars, s string) error { From c53a043e3c78d2553b113e6fd71542c4a348d639 Mon Sep 17 00:00:00 2001 From: Zhenchi Date: Fri, 10 Dec 2021 09:43:15 +0800 Subject: [PATCH 33/47] comment DataSink Signed-off-by: Zhenchi --- util/topsql/reporter/datasink.go | 7 +++++-- util/topsql/reporter/pubsub.go | 10 ++++----- util/topsql/reporter/reporter.go | 30 +++++++++++++-------------- util/topsql/reporter/single_target.go | 12 +++++------ 4 files changed, 31 insertions(+), 28 deletions(-) diff --git a/util/topsql/reporter/datasink.go b/util/topsql/reporter/datasink.go index 7b824a2bca352..3058a30b2903c 100644 --- a/util/topsql/reporter/datasink.go +++ b/util/topsql/reporter/datasink.go @@ -18,9 +18,12 @@ import ( "time" ) -// DataSink sends data to the target server. +// DataSink collects and sends data to a target. type DataSink interface { - Send(data reportData, deadline time.Time) + // Send pushes a report data into the sink, which will later be sent to a target by the sink. A deadline can be + // specified to control how late it should be sent. If the sink is kept full and cannot schedule a send within + // the specified deadline, the data will be silently dropped. + Send(data ReportData, deadline time.Time) // IsPaused indicates that the DataSink is not expecting to receive records for now // and may resume in the future. diff --git a/util/topsql/reporter/pubsub.go b/util/topsql/reporter/pubsub.go index 5aa3fd71e5d9b..88b4be15fb329 100644 --- a/util/topsql/reporter/pubsub.go +++ b/util/topsql/reporter/pubsub.go @@ -79,7 +79,7 @@ func newPubSubDataSink( var _ DataSink = &pubSubDataSink{} -func (s *pubSubDataSink) Send(data reportData, deadline time.Time) { +func (s *pubSubDataSink) Send(data ReportData, deadline time.Time) { if s.IsDown() { return } @@ -146,14 +146,14 @@ func (s *pubSubDataSink) run() { } } -func (s *pubSubDataSink) doSend(ctx context.Context, data reportData) error { - if err := s.sendCPUTime(ctx, data.cpuTimeRecords); err != nil { +func (s *pubSubDataSink) doSend(ctx context.Context, data ReportData) error { + if err := s.sendCPUTime(ctx, data.CPUTimeRecords); err != nil { return err } - if err := s.sendSQLMeta(ctx, data.sqlMetas); err != nil { + if err := s.sendSQLMeta(ctx, data.SQLMetas); err != nil { return err } - return s.sendPlanMeta(ctx, data.planMetas) + return s.sendPlanMeta(ctx, data.PlanMetas) } func (s *pubSubDataSink) sendCPUTime(ctx context.Context, records []*tipb.CPUTimeRecord) (err error) { diff --git a/util/topsql/reporter/reporter.go b/util/topsql/reporter/reporter.go index 3031a616c011b..cbe0928cf6ec0 100644 --- a/util/topsql/reporter/reporter.go +++ b/util/topsql/reporter/reporter.go @@ -528,21 +528,21 @@ type collectedData struct { normalizedPlanMap *sync.Map } -// 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 +// 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 } -func (d *reportData) hasData() bool { - return len(d.cpuTimeRecords) != 0 || len(d.sqlMetas) != 0 || len(d.planMetas) != 0 +func (d *ReportData) hasData() bool { + return len(d.CPUTimeRecords) != 0 || len(d.SQLMetas) != 0 || len(d.PlanMetas) != 0 } -func buildReportData(records []*dataPoints, sqlMap *sync.Map, planMap *sync.Map, decodePlan planBinaryDecodeFunc) (res reportData) { +func buildReportData(records []*dataPoints, sqlMap *sync.Map, planMap *sync.Map, decodePlan planBinaryDecodeFunc) (res ReportData) { for _, record := range records { - res.cpuTimeRecords = append(res.cpuTimeRecords, &tipb.CPUTimeRecord{ + res.CPUTimeRecords = append(res.CPUTimeRecords, &tipb.CPUTimeRecord{ RecordListTimestampSec: record.TimestampList, RecordListCpuTimeMs: record.CPUTimeMsList, SqlDigest: record.SQLDigest, @@ -552,7 +552,7 @@ func buildReportData(records []*dataPoints, sqlMap *sync.Map, planMap *sync.Map, sqlMap.Range(func(key, value interface{}) bool { meta := value.(SQLMeta) - res.sqlMetas = append(res.sqlMetas, &tipb.SQLMeta{ + res.SQLMetas = append(res.SQLMetas, &tipb.SQLMeta{ SqlDigest: []byte(key.(string)), NormalizedSql: meta.normalizedSQL, IsInternalSql: meta.isInternal, @@ -566,7 +566,7 @@ func buildReportData(records []*dataPoints, sqlMap *sync.Map, planMap *sync.Map, logutil.BgLogger().Warn("[top-sql] decode plan failed", zap.Error(errDecode)) return true } - res.planMetas = append(res.planMetas, &tipb.PlanMeta{ + res.PlanMetas = append(res.PlanMetas, &tipb.PlanMeta{ PlanDigest: []byte(key.(string)), NormalizedPlan: planDecoded, }) @@ -595,9 +595,9 @@ func (tsr *RemoteTopSQLReporter) reportWorker() { } } -// getReportData gets reportData from the collectedData. +// 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 { +func (tsr *RemoteTopSQLReporter) getReportData(collected collectedData) ReportData { records, sqlMap, planMap := getTopN(collected) return buildReportData(records, sqlMap, planMap, tsr.decodePlan) } @@ -636,7 +636,7 @@ func getTopN(collected collectedData) (records []*dataPoints, sqlMap *sync.Map, return } -func (tsr *RemoteTopSQLReporter) doReport(data reportData) { +func (tsr *RemoteTopSQLReporter) doReport(data ReportData) { defer util.Recover("top-sql", "doReport", nil, false) if !data.hasData() { diff --git a/util/topsql/reporter/single_target.go b/util/topsql/reporter/single_target.go index 1fa31f08a858f..27ed687947689 100644 --- a/util/topsql/reporter/single_target.go +++ b/util/topsql/reporter/single_target.go @@ -37,7 +37,7 @@ type SingleTargetDataSink struct { } type sendTask struct { - data reportData + data ReportData deadline time.Time } @@ -75,7 +75,7 @@ func (r *SingleTargetDataSink) run() { var _ DataSink = &SingleTargetDataSink{} // Send implements the DataSink interface. -func (r *SingleTargetDataSink) Send(data reportData, deadline time.Time) { +func (r *SingleTargetDataSink) Send(data ReportData, deadline time.Time) { select { case r.sendTaskCh <- sendTask{data: data, deadline: deadline}: // sent successfully @@ -86,7 +86,7 @@ func (r *SingleTargetDataSink) Send(data reportData, deadline time.Time) { } // Currently the doSend will establish a new connection every time, which is suitable for a per-minute sending period -func (r *SingleTargetDataSink) doSend(ctx context.Context, addr string, data reportData) (err error) { +func (r *SingleTargetDataSink) doSend(ctx context.Context, addr string, data ReportData) (err error) { err = r.tryEstablishConnection(ctx, addr) if err != nil { return @@ -98,15 +98,15 @@ func (r *SingleTargetDataSink) doSend(ctx context.Context, addr string, data rep go func() { defer wg.Done() - errCh <- r.sendBatchSQLMeta(ctx, data.sqlMetas) + errCh <- r.sendBatchSQLMeta(ctx, data.SQLMetas) }() go func() { defer wg.Done() - errCh <- r.sendBatchPlanMeta(ctx, data.planMetas) + errCh <- r.sendBatchPlanMeta(ctx, data.PlanMetas) }() go func() { defer wg.Done() - errCh <- r.sendBatchCPUTimeRecord(ctx, data.cpuTimeRecords) + errCh <- r.sendBatchCPUTimeRecord(ctx, data.CPUTimeRecords) }() wg.Wait() close(errCh) From a3cf4d6e3426141775593697852ed1684116d378 Mon Sep 17 00:00:00 2001 From: Zhenchi Date: Fri, 10 Dec 2021 11:14:10 +0800 Subject: [PATCH 34/47] not to close receiver Signed-off-by: Zhenchi --- executor/executor_test.go | 3 +++ util/topsql/reporter/pubsub.go | 27 ++++++++++----------------- util/topsql/reporter/reporter.go | 26 +++++++++++++------------- 3 files changed, 26 insertions(+), 30 deletions(-) diff --git a/executor/executor_test.go b/executor/executor_test.go index 0736c6bf09909..ed9396f4d7529 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -8722,6 +8722,9 @@ func (s *testResourceTagSuite) TestResourceGroupTag(c *C) { tracecpu.GlobalSQLCPUProfiler.SetTopSQLEnabled(true) defer tracecpu.GlobalSQLCPUProfiler.SetTopSQLEnabled(false) + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/mockstore/unistore/unistoreRPCClientSendHook", `return(true)`), IsNil) + defer failpoint.Disable("github.com/pingcap/tidb/store/mockstore/unistore/unistoreRPCClientSendHook") + var sqlDigest, planDigest *parser.Digest var tagLabel tipb.ResourceGroupTagLabel checkFn := func() {} diff --git a/util/topsql/reporter/pubsub.go b/util/topsql/reporter/pubsub.go index 88b4be15fb329..e6204717a768a 100644 --- a/util/topsql/reporter/pubsub.go +++ b/util/topsql/reporter/pubsub.go @@ -16,6 +16,7 @@ package reporter import ( "context" + "errors" "time" "github.com/pingcap/tidb/util" @@ -113,12 +114,8 @@ func (s *pubSubDataSink) run() { start := time.Now() var err error - doneCh := make(chan struct{}) go util.WithRecovery(func() { - defer func() { - doneCh <- struct{}{} - cancel() - }() + defer cancel() err = s.doSend(ctx, task.data) if err != nil { reportAllDurationFailedHistogram.Observe(time.Since(start).Seconds()) @@ -127,21 +124,17 @@ func (s *pubSubDataSink) run() { } }, nil) - select { - case <-doneCh: - if err != nil { - logutil.BgLogger().Warn( - "[top-sql] pubsub datasink failed to send data to subscriber", - zap.Error(err), - ) - return - } - case <-ctx.Done(): + <-ctx.Done() + if errors.Is(ctx.Err(), context.DeadlineExceeded) { logutil.BgLogger().Warn( - "[top-sql] pubsub datasink failed to send data to subscriber due to timeout", + "[top-sql] pubsub datasink failed to send data to subscriber due to deadline exceeded", zap.Time("deadline", task.deadline), ) - return + } else if err != nil { + logutil.BgLogger().Warn( + "[top-sql] pubsub datasink failed to send data to subscriber", + zap.Error(err), + ) } } } diff --git a/util/topsql/reporter/reporter.go b/util/topsql/reporter/reporter.go index cbe0928cf6ec0..19cc139a3f4fb 100644 --- a/util/topsql/reporter/reporter.go +++ b/util/topsql/reporter/reporter.go @@ -128,8 +128,9 @@ type RemoteTopSQLReporter struct { ctx context.Context cancel context.CancelFunc - dataSinks []DataSink - dataSinkRegCh chan DataSink + dataSinks []DataSink + dataSinkRegCh chan DataSink + dataSinkRegCloseCh chan struct{} // normalizedSQLMap is a map, whose keys are SQL digest strings and values are SQLMeta. normalizedSQLMap atomic.Value // sync.Map @@ -252,7 +253,7 @@ func (tsr *RemoteTopSQLReporter) Collect(timestamp uint64, records []tracecpu.SQ // DataSinkRegHandle returns a DataSinkRegHandle for DataSink registration. func (tsr *RemoteTopSQLReporter) DataSinkRegHandle() DataSinkRegHandle { - return &RemoteDataSinkRegHandle{registerCh: tsr.dataSinkRegCh} + return &RemoteDataSinkRegHandle{registerCh: tsr.dataSinkRegCh, closeCh: tsr.dataSinkRegCloseCh} } // Close uses to close and release the reporter resource. @@ -261,7 +262,7 @@ func (tsr *RemoteTopSQLReporter) Close() { for i := range tsr.dataSinks { tsr.dataSinks[i].Close() } - close(tsr.dataSinkRegCh) + close(tsr.dataSinkRegCloseCh) tsr.dataSinks = nil } @@ -664,16 +665,15 @@ var _ DataSinkRegHandle = &RemoteDataSinkRegHandle{} // RemoteDataSinkRegHandle is used to receive DataSink registrations. type RemoteDataSinkRegHandle struct { registerCh chan DataSink + closeCh chan struct{} } // Register implements DataSinkRegHandle interface. -func (r *RemoteDataSinkRegHandle) Register(dataSink DataSink) (err error) { - defer func() { - if recover() != nil { - err = errors.New("registration channel is closed") - } - }() - - r.registerCh <- dataSink - return nil +func (r *RemoteDataSinkRegHandle) Register(dataSink DataSink) error { + select { + case r.registerCh <- dataSink: + return nil + case <-r.closeCh: + return errors.New("registration channel is closed") + } } From 26c5812b4c7aaf05dd91a4c2a9aa3637487178bc Mon Sep 17 00:00:00 2001 From: Zhenchi Date: Fri, 10 Dec 2021 11:48:37 +0800 Subject: [PATCH 35/47] split handleDataSinkRegistration Signed-off-by: Zhenchi --- server/tidb_test.go | 3 +- util/topsql/reporter/pubsub.go | 6 +--- util/topsql/reporter/reporter.go | 48 ++++++++++++++------------- util/topsql/reporter/reporter_test.go | 20 +++++------ util/topsql/topsql.go | 4 +-- util/topsql/topsql_test.go | 3 +- 6 files changed, 38 insertions(+), 46 deletions(-) diff --git a/server/tidb_test.go b/server/tidb_test.go index 70d466853c76a..3f5584716bc76 100644 --- a/server/tidb_test.go +++ b/server/tidb_test.go @@ -1566,8 +1566,7 @@ func TestTopSQLReceiver(t *testing.T) { dbt.MustExec("set @@global.tidb_top_sql_max_statement_count=5;") r := reporter.NewRemoteTopSQLReporter(plancodec.DecodeNormalizedPlan) - err = r.DataSinkRegHandle().Register(reporter.NewSingleTargetDataSink()) - require.NoError(t, err) + r.DataSinkRegHandle().Register(reporter.NewSingleTargetDataSink()) tracecpu.GlobalSQLCPUProfiler.SetCollector(&collectorWrapper{r}) // TODO: change to ensure that the right sql statements are reported, not just counts diff --git a/util/topsql/reporter/pubsub.go b/util/topsql/reporter/pubsub.go index e6204717a768a..cfaa5ad47bcdf 100644 --- a/util/topsql/reporter/pubsub.go +++ b/util/topsql/reporter/pubsub.go @@ -53,11 +53,7 @@ func (t *TopSQLPubSubService) Subscribe( stream tipb.TopSQLPubSub_SubscribeServer, ) error { ds := newPubSubDataSink(stream) - - if err := t.dataSinkRegHandle.Register(ds); err != nil { - return err - } - + t.dataSinkRegHandle.Register(ds) ds.run() return nil } diff --git a/util/topsql/reporter/reporter.go b/util/topsql/reporter/reporter.go index 19cc139a3f4fb..fb38149efeea0 100644 --- a/util/topsql/reporter/reporter.go +++ b/util/topsql/reporter/reporter.go @@ -17,7 +17,6 @@ package reporter import ( "bytes" "context" - "errors" "sort" "sync" "sync/atomic" @@ -57,7 +56,7 @@ type TopSQLReporter interface { // DataSinkRegHandle registers DataSink type DataSinkRegHandle interface { - Register(dataSink DataSink) error + Register(dataSink DataSink) } type cpuData struct { @@ -128,9 +127,8 @@ type RemoteTopSQLReporter struct { ctx context.Context cancel context.CancelFunc - dataSinks []DataSink - dataSinkRegCh chan DataSink - dataSinkRegCloseCh chan struct{} + dataSinks []DataSink + dataSinkRegCh chan DataSink // normalizedSQLMap is a map, whose keys are SQL digest strings and values are SQLMeta. normalizedSQLMap atomic.Value // sync.Map @@ -253,7 +251,7 @@ func (tsr *RemoteTopSQLReporter) Collect(timestamp uint64, records []tracecpu.SQ // DataSinkRegHandle returns a DataSinkRegHandle for DataSink registration. func (tsr *RemoteTopSQLReporter) DataSinkRegHandle() DataSinkRegHandle { - return &RemoteDataSinkRegHandle{registerCh: tsr.dataSinkRegCh, closeCh: tsr.dataSinkRegCloseCh} + return &RemoteDataSinkRegHandle{reporterDoneCh: tsr.ctx.Done(), registerCh: tsr.dataSinkRegCh} } // Close uses to close and release the reporter resource. @@ -262,7 +260,6 @@ func (tsr *RemoteTopSQLReporter) Close() { for i := range tsr.dataSinks { tsr.dataSinks[i].Close() } - close(tsr.dataSinkRegCloseCh) tsr.dataSinks = nil } @@ -349,7 +346,13 @@ func (tsr *RemoteTopSQLReporter) collectWorker() { currentReportInterval := variable.TopSQLVariable.ReportIntervalSeconds.Load() reportTicker := time.NewTicker(time.Second * time.Duration(currentReportInterval)) for { - tsr.handleDataSinkRegistration() + tsr.acceptDataSinkRegs() + tsr.removeDownDataSinks() + if len(tsr.dataSinks) > 10 { + logutil.BgLogger().Warn("[top-sql] too many datasinks, keep 10 first", zap.Int("count", len(tsr.dataSinks))) + tsr.dataSinks = tsr.dataSinks[:10] + } + tracecpu.GlobalSQLCPUProfiler.SetTopSQLEnabled(tsr.activeDataSinkCnt() > 0) select { case data := <-tsr.collectCPUDataChan: @@ -368,7 +371,8 @@ func (tsr *RemoteTopSQLReporter) collectWorker() { } } -func (tsr *RemoteTopSQLReporter) handleDataSinkRegistration() { +// acceptDataSinkRegs accepts datasink registrations +func (tsr *RemoteTopSQLReporter) acceptDataSinkRegs() { out: for { select { @@ -378,8 +382,10 @@ out: break out } } +} - // Remove all down dataSinks +// removeDownDataSinks removes all down dataSinks +func (tsr *RemoteTopSQLReporter) removeDownDataSinks() { idx := 0 for _, dataSink := range tsr.dataSinks { if dataSink.IsDown() { @@ -390,20 +396,17 @@ out: idx++ } tsr.dataSinks = tsr.dataSinks[:idx] +} - if len(tsr.dataSinks) > 10 { - logutil.BgLogger().Warn("[top-sql] too many datasinks, keep 10 first", zap.Int("count", len(tsr.dataSinks))) - tsr.dataSinks = tsr.dataSinks[:10] - } - +// activeDataSinkCnt gets the count of active datasinks +func (tsr *RemoteTopSQLReporter) activeDataSinkCnt() int { pendingCnt := 0 for _, dataSink := range tsr.dataSinks { if dataSink.IsPaused() { pendingCnt += 1 } } - runningCnt := len(tsr.dataSinks) - pendingCnt - tracecpu.GlobalSQLCPUProfiler.SetTopSQLEnabled(runningCnt > 0) + return len(tsr.dataSinks) - pendingCnt } func encodeKey(buf *bytes.Buffer, sqlDigest, planDigest []byte) string { @@ -664,16 +667,15 @@ var _ DataSinkRegHandle = &RemoteDataSinkRegHandle{} // RemoteDataSinkRegHandle is used to receive DataSink registrations. type RemoteDataSinkRegHandle struct { - registerCh chan DataSink - closeCh chan struct{} + registerCh chan DataSink + reporterDoneCh <-chan struct{} } // Register implements DataSinkRegHandle interface. -func (r *RemoteDataSinkRegHandle) Register(dataSink DataSink) error { +func (r *RemoteDataSinkRegHandle) Register(dataSink DataSink) { select { case r.registerCh <- dataSink: - return nil - case <-r.closeCh: - return errors.New("registration channel is closed") + case <-r.reporterDoneCh: + logutil.BgLogger().Warn("[top-sql] failed to register datasink due to the reporter is down") } } diff --git a/util/topsql/reporter/reporter_test.go b/util/topsql/reporter/reporter_test.go index 9712010c92131..3980c1c81e0e2 100644 --- a/util/topsql/reporter/reporter_test.go +++ b/util/topsql/reporter/reporter_test.go @@ -64,7 +64,7 @@ func mockPlanBinaryDecoderFunc(plan string) (string, error) { return plan, nil } -func setupRemoteTopSQLReporter(t *testing.T, maxStatementsNum, interval int, addr string) *RemoteTopSQLReporter { +func setupRemoteTopSQLReporter(maxStatementsNum, interval int, addr string) *RemoteTopSQLReporter { variable.TopSQLVariable.MaxStatementCount.Store(int64(maxStatementsNum)) variable.TopSQLVariable.ReportIntervalSeconds.Store(int64(interval)) config.UpdateGlobal(func(conf *config.Config) { @@ -73,15 +73,13 @@ func setupRemoteTopSQLReporter(t *testing.T, maxStatementsNum, interval int, add ts := NewRemoteTopSQLReporter(mockPlanBinaryDecoderFunc) ds := NewSingleTargetDataSink() - err := ts.DataSinkRegHandle().Register(ds) - if t != nil { - require.NoError(t, err) - } + ts.DataSinkRegHandle().Register(ds) + return ts } func initializeCache(maxStatementsNum, interval int, addr string) *RemoteTopSQLReporter { - ts := setupRemoteTopSQLReporter(nil, maxStatementsNum, interval, addr) + ts := setupRemoteTopSQLReporter(maxStatementsNum, interval, addr) populateCache(ts, 0, maxStatementsNum, 1) return ts } @@ -91,7 +89,7 @@ func TestCollectAndSendBatch(t *testing.T) { require.NoError(t, err) defer agentServer.Stop() - tsr := setupRemoteTopSQLReporter(t, maxSQLNum, 1, agentServer.Address()) + tsr := setupRemoteTopSQLReporter(maxSQLNum, 1, agentServer.Address()) defer tsr.Close() populateCache(tsr, 0, maxSQLNum, 1) @@ -130,7 +128,7 @@ func TestCollectAndEvicted(t *testing.T) { require.NoError(t, err) defer agentServer.Stop() - tsr := setupRemoteTopSQLReporter(t, maxSQLNum, 1, agentServer.Address()) + tsr := setupRemoteTopSQLReporter(maxSQLNum, 1, agentServer.Address()) defer tsr.Close() populateCache(tsr, 0, maxSQLNum*2, 2) @@ -195,7 +193,7 @@ func TestCollectAndTopN(t *testing.T) { require.NoError(t, err) defer agentServer.Stop() - tsr := setupRemoteTopSQLReporter(t, 2, 1, agentServer.Address()) + tsr := setupRemoteTopSQLReporter(2, 1, agentServer.Address()) defer tsr.Close() records := []tracecpu.SQLCPUTimeRecord{ @@ -256,7 +254,7 @@ func TestCollectAndTopN(t *testing.T) { } func TestCollectCapacity(t *testing.T) { - tsr := setupRemoteTopSQLReporter(t, maxSQLNum, 60, "") + tsr := setupRemoteTopSQLReporter(maxSQLNum, 60, "") defer tsr.Close() registerSQL := func(n int) { @@ -397,7 +395,7 @@ func TestCollectInternal(t *testing.T) { require.NoError(t, err) defer agentServer.Stop() - tsr := setupRemoteTopSQLReporter(t, 3000, 1, agentServer.Address()) + tsr := setupRemoteTopSQLReporter(3000, 1, agentServer.Address()) defer tsr.Close() records := []tracecpu.SQLCPUTimeRecord{ diff --git a/util/topsql/topsql.go b/util/topsql/topsql.go index ed97a91387092..bb4944476d542 100644 --- a/util/topsql/topsql.go +++ b/util/topsql/topsql.go @@ -23,7 +23,6 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/parser" - "github.com/pingcap/tidb/parser/terror" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/plancodec" "github.com/pingcap/tidb/util/topsql/reporter" @@ -53,8 +52,7 @@ func SetupTopSQL() { // register single target datasink to reporter singleTargetDataSink := reporter.NewSingleTargetDataSink() - err := globalTopSQLReport.DataSinkRegHandle().Register(singleTargetDataSink) - terror.MustNil(err) + globalTopSQLReport.DataSinkRegHandle().Register(singleTargetDataSink) enable := len(config.GetGlobalConfig().TopSQL.ReceiverAddress) != 0 tracecpu.GlobalSQLCPUProfiler.SetTopSQLEnabled(enable) diff --git a/util/topsql/topsql_test.go b/util/topsql/topsql_test.go index 3819630fc2507..40111130649bf 100644 --- a/util/topsql/topsql_test.go +++ b/util/topsql/topsql_test.go @@ -124,8 +124,7 @@ func TestTopSQLReporter(t *testing.T) { defer report.Close() dataSink := reporter.NewSingleTargetDataSink() - err = report.DataSinkRegHandle().Register(dataSink) - require.NoError(t, err) + report.DataSinkRegHandle().Register(dataSink) tracecpu.GlobalSQLCPUProfiler.SetCollector(&collectorWrapper{report}) reqs := []struct { From a7474d76ff4a64c943dc75f2aa3545a4fa378f9b Mon Sep 17 00:00:00 2001 From: Zhenchi Date: Fri, 10 Dec 2021 13:02:53 +0800 Subject: [PATCH 36/47] fix leak goroutine in test Signed-off-by: Zhenchi --- util/topsql/reporter/reporter.go | 1 + util/topsql/topsql_test.go | 11 ++++++++++- 2 files changed, 11 insertions(+), 1 deletion(-) diff --git a/util/topsql/reporter/reporter.go b/util/topsql/reporter/reporter.go index fb38149efeea0..74003e6edfa1f 100644 --- a/util/topsql/reporter/reporter.go +++ b/util/topsql/reporter/reporter.go @@ -677,5 +677,6 @@ func (r *RemoteDataSinkRegHandle) Register(dataSink DataSink) { case r.registerCh <- dataSink: case <-r.reporterDoneCh: logutil.BgLogger().Warn("[top-sql] failed to register datasink due to the reporter is down") + dataSink.Close() } } diff --git a/util/topsql/topsql_test.go b/util/topsql/topsql_test.go index 40111130649bf..ecec47882ebb7 100644 --- a/util/topsql/topsql_test.go +++ b/util/topsql/topsql_test.go @@ -17,6 +17,7 @@ package topsql_test import ( "bytes" "context" + "google.golang.org/grpc/keepalive" "testing" "time" @@ -196,7 +197,15 @@ func TestTopSQLPubSub(t *testing.T) { defer server.Stop() tracecpu.GlobalSQLCPUProfiler.SetCollector(&collectorWrapper{report}) - conn, err := grpc.Dial(server.Address(), grpc.WithBlock(), grpc.WithInsecure()) + conn, err := grpc.Dial( + server.Address(), + grpc.WithBlock(), + grpc.WithInsecure(), + grpc.WithKeepaliveParams(keepalive.ClientParameters{ + Time: 10 * time.Second, + Timeout: 3 * time.Second, + }), + ) require.NoError(t, err) defer conn.Close() From 3628b3e0826f02fa39b2a0fddf74fc0d8f8b04b2 Mon Sep 17 00:00:00 2001 From: Zhenchi Date: Fri, 10 Dec 2021 14:56:51 +0800 Subject: [PATCH 37/47] fix imports Signed-off-by: Zhenchi --- util/topsql/topsql_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/util/topsql/topsql_test.go b/util/topsql/topsql_test.go index ecec47882ebb7..a365d1fefa0a9 100644 --- a/util/topsql/topsql_test.go +++ b/util/topsql/topsql_test.go @@ -17,7 +17,6 @@ package topsql_test import ( "bytes" "context" - "google.golang.org/grpc/keepalive" "testing" "time" @@ -33,6 +32,7 @@ import ( "github.com/pingcap/tipb/go-tipb" "github.com/stretchr/testify/require" "google.golang.org/grpc" + "google.golang.org/grpc/keepalive" ) type collectorWrapper struct { From 1ecea775c6c663c9656f1176f4ae65c2707349c2 Mon Sep 17 00:00:00 2001 From: Zhenchi Date: Fri, 10 Dec 2021 17:19:06 +0800 Subject: [PATCH 38/47] enabled Signed-off-by: Zhenchi --- domain/sysvar_cache.go | 10 +++-- executor/executor_test.go | 6 ++- executor/set_test.go | 18 ++++---- server/tidb_test.go | 2 - sessionctx/variable/sysvar.go | 36 ++++++++------- sessionctx/variable/tidb_vars.go | 30 ++----------- util/topsql/main_test.go | 3 +- util/topsql/reporter/reporter.go | 48 +++++++++++++++----- util/topsql/reporter/reporter_test.go | 11 +++-- util/topsql/topsql.go | 9 ++-- util/topsql/topsql_test.go | 29 +++++------- util/topsql/tracecpu/main_test.go | 9 ++-- util/topsql/tracecpu/mock/mock.go | 30 +++++++++++++ util/topsql/tracecpu/profile.go | 64 +++++++++++++++++---------- 14 files changed, 179 insertions(+), 126 deletions(-) diff --git a/domain/sysvar_cache.go b/domain/sysvar_cache.go index 0ebd4e176a155..131085de2026a 100644 --- a/domain/sysvar_cache.go +++ b/domain/sysvar_cache.go @@ -26,6 +26,8 @@ import ( "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/sqlexec" "github.com/pingcap/tidb/util/stmtsummary" + "github.com/pingcap/tidb/util/topsql/reporter" + "github.com/pingcap/tidb/util/topsql/tracecpu" storekv "github.com/tikv/client-go/v2/kv" pd "github.com/tikv/pd/client" "go.uber.org/zap" @@ -207,28 +209,28 @@ func (do *Domain) checkEnableServerGlobalVar(name, sVal string) { if err != nil { break } - variable.TopSQLVariable.PrecisionSeconds.Store(val) + tracecpu.PrecisionSeconds.Store(val) case variable.TiDBTopSQLMaxStatementCount: var val int64 val, err = strconv.ParseInt(sVal, 10, 64) if err != nil { break } - variable.TopSQLVariable.MaxStatementCount.Store(val) + reporter.MaxStatementCount.Store(val) case variable.TiDBTopSQLMaxCollect: var val int64 val, err = strconv.ParseInt(sVal, 10, 64) if err != nil { break } - variable.TopSQLVariable.MaxCollect.Store(val) + reporter.MaxCollect.Store(val) case variable.TiDBTopSQLReportIntervalSeconds: var val int64 val, err = strconv.ParseInt(sVal, 10, 64) if err != nil { break } - variable.TopSQLVariable.ReportIntervalSeconds.Store(val) + reporter.ReportIntervalSeconds.Store(val) case variable.TiDBRestrictedReadOnly: variable.RestrictedReadOnly.Store(variable.TiDBOptOn(sVal)) case variable.TiDBStoreLimit: diff --git a/executor/executor_test.go b/executor/executor_test.go index ed9396f4d7529..e8148accdc9b3 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -83,6 +83,7 @@ import ( "github.com/pingcap/tidb/util/testutil" "github.com/pingcap/tidb/util/timeutil" "github.com/pingcap/tidb/util/topsql/tracecpu" + tracecpumock "github.com/pingcap/tidb/util/topsql/tracecpu/mock" "github.com/pingcap/tipb/go-tipb" "github.com/stretchr/testify/require" "github.com/tikv/client-go/v2/oracle" @@ -8719,8 +8720,9 @@ func (s *testResourceTagSuite) TestResourceGroupTag(c *C) { tk.MustExec("create table t(a int, b int, unique index idx(a));") tbInfo := testGetTableByName(c, tk.Se, "test", "t") - tracecpu.GlobalSQLCPUProfiler.SetTopSQLEnabled(true) - defer tracecpu.GlobalSQLCPUProfiler.SetTopSQLEnabled(false) + ctl := tracecpumock.NewProfileController(true) + tracecpu.GlobalSQLCPUProfiler.SetCollector(ctl) + defer tracecpu.GlobalSQLCPUProfiler.SetCollector(nil) c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/mockstore/unistore/unistoreRPCClientSendHook", `return(true)`), IsNil) defer failpoint.Disable("github.com/pingcap/tidb/store/mockstore/unistore/unistoreRPCClientSendHook") diff --git a/executor/set_test.go b/executor/set_test.go index 8bbb4d8ea730e..fd1da23e4ad9f 100644 --- a/executor/set_test.go +++ b/executor/set_test.go @@ -38,6 +38,8 @@ import ( "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/testkit" "github.com/pingcap/tidb/util/testutil" + "github.com/pingcap/tidb/util/topsql/reporter" + "github.com/pingcap/tidb/util/topsql/tracecpu" ) func (s *testSerialSuite1) TestSetVar(c *C) { @@ -1500,18 +1502,18 @@ func (s *testSerialSuite) TestSetTopSQLVariables(c *C) { tk.MustExec("set @@global.tidb_top_sql_precision_seconds=2;") tk.MustQuery("select @@global.tidb_top_sql_precision_seconds;").Check(testkit.Rows("2")) - c.Assert(variable.TopSQLVariable.PrecisionSeconds.Load(), Equals, int64(2)) + c.Assert(tracecpu.PrecisionSeconds.Load(), Equals, int64(2)) _, err := tk.Exec("set @@global.tidb_top_sql_precision_seconds='abc';") c.Assert(err.Error(), Equals, "[variable:1232]Incorrect argument type to variable 'tidb_top_sql_precision_seconds'") tk.MustExec("set @@global.tidb_top_sql_precision_seconds='-1';") tk.MustQuery("select @@global.tidb_top_sql_precision_seconds;").Check(testkit.Rows("1")) tk.MustExec("set @@global.tidb_top_sql_precision_seconds=2;") tk.MustQuery("select @@global.tidb_top_sql_precision_seconds;").Check(testkit.Rows("2")) - c.Assert(variable.TopSQLVariable.PrecisionSeconds.Load(), Equals, int64(2)) + c.Assert(tracecpu.PrecisionSeconds.Load(), Equals, int64(2)) tk.MustExec("set @@global.tidb_top_sql_max_statement_count=20;") tk.MustQuery("select @@global.tidb_top_sql_max_statement_count;").Check(testkit.Rows("20")) - c.Assert(variable.TopSQLVariable.MaxStatementCount.Load(), Equals, int64(20)) + c.Assert(reporter.MaxStatementCount.Load(), Equals, int64(20)) _, err = tk.Exec("set @@global.tidb_top_sql_max_statement_count='abc';") c.Assert(err.Error(), Equals, "[variable:1232]Incorrect argument type to variable 'tidb_top_sql_max_statement_count'") tk.MustExec("set @@global.tidb_top_sql_max_statement_count='-1';") @@ -1522,11 +1524,11 @@ func (s *testSerialSuite) TestSetTopSQLVariables(c *C) { tk.MustExec("set @@global.tidb_top_sql_max_statement_count=20;") tk.MustQuery("select @@global.tidb_top_sql_max_statement_count;").Check(testkit.Rows("20")) - c.Assert(variable.TopSQLVariable.MaxStatementCount.Load(), Equals, int64(20)) + c.Assert(reporter.MaxStatementCount.Load(), Equals, int64(20)) tk.MustExec("set @@global.tidb_top_sql_max_collect=20000;") tk.MustQuery("select @@global.tidb_top_sql_max_collect;").Check(testkit.Rows("20000")) - c.Assert(variable.TopSQLVariable.MaxCollect.Load(), Equals, int64(20000)) + c.Assert(reporter.MaxCollect.Load(), Equals, int64(20000)) _, err = tk.Exec("set @@global.tidb_top_sql_max_collect='abc';") c.Assert(err.Error(), Equals, "[variable:1232]Incorrect argument type to variable 'tidb_top_sql_max_collect'") tk.MustExec("set @@global.tidb_top_sql_max_collect='-1';") @@ -1539,11 +1541,11 @@ func (s *testSerialSuite) TestSetTopSQLVariables(c *C) { tk.MustExec("set @@global.tidb_top_sql_max_collect=20000;") tk.MustQuery("select @@global.tidb_top_sql_max_collect;").Check(testkit.Rows("20000")) - c.Assert(variable.TopSQLVariable.MaxCollect.Load(), Equals, int64(20000)) + c.Assert(reporter.MaxCollect.Load(), Equals, int64(20000)) tk.MustExec("set @@global.tidb_top_sql_report_interval_seconds=120;") tk.MustQuery("select @@global.tidb_top_sql_report_interval_seconds;").Check(testkit.Rows("120")) - c.Assert(variable.TopSQLVariable.ReportIntervalSeconds.Load(), Equals, int64(120)) + c.Assert(reporter.ReportIntervalSeconds.Load(), Equals, int64(120)) _, err = tk.Exec("set @@global.tidb_top_sql_report_interval_seconds='abc';") c.Assert(err.Error(), Equals, "[variable:1232]Incorrect argument type to variable 'tidb_top_sql_report_interval_seconds'") tk.MustExec("set @@global.tidb_top_sql_report_interval_seconds='5000';") @@ -1552,7 +1554,7 @@ func (s *testSerialSuite) TestSetTopSQLVariables(c *C) { tk.MustExec("set @@global.tidb_top_sql_report_interval_seconds=120;") tk.MustQuery("select @@global.tidb_top_sql_report_interval_seconds;").Check(testkit.Rows("120")) - c.Assert(variable.TopSQLVariable.ReportIntervalSeconds.Load(), Equals, int64(120)) + c.Assert(reporter.ReportIntervalSeconds.Load(), Equals, int64(120)) // Test for hide top sql variable in show variable. tk.MustQuery("show variables like '%top_sql%'").Check(testkit.Rows()) diff --git a/server/tidb_test.go b/server/tidb_test.go index 3f5584716bc76..9bab2e03dc0a2 100644 --- a/server/tidb_test.go +++ b/server/tidb_test.go @@ -1316,8 +1316,6 @@ func TestTopSQLCPUProfile(t *testing.T) { collector := mockTopSQLTraceCPU.NewTopSQLCollector() tracecpu.GlobalSQLCPUProfiler.SetCollector(&collectorWrapper{collector}) - tracecpu.GlobalSQLCPUProfiler.SetTopSQLEnabled(true) - defer tracecpu.GlobalSQLCPUProfiler.SetTopSQLEnabled(false) dbt := testkit.NewDBTestKit(t, db) dbt.MustExec("drop database if exists topsql") diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index af70bb2461504..b792e60ba67f6 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -35,6 +35,9 @@ import ( "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/stmtsummary" + "github.com/pingcap/tidb/util/topsql" + "github.com/pingcap/tidb/util/topsql/reporter" + "github.com/pingcap/tidb/util/topsql/tracecpu" "github.com/pingcap/tidb/util/versioninfo" tikvstore "github.com/tikv/client-go/v2/kv" atomic2 "go.uber.org/atomic" @@ -799,7 +802,9 @@ var defaultSysVars = []*SysVar{ return strconv.FormatInt(int64(GlobalLogMaxDays.Load()), 10), nil }}, {Scope: ScopeSession, Name: TiDBPProfSQLCPU, Value: strconv.Itoa(DefTiDBPProfSQLCPU), Type: TypeInt, skipInit: true, MinValue: 0, MaxValue: 1, SetSession: func(s *SessionVars, val string) error { - EnablePProfSQLCPU.Store(uint32(tidbOptPositiveInt32(val, DefTiDBPProfSQLCPU)) > 0) + enable := uint32(tidbOptPositiveInt32(val, DefTiDBPProfSQLCPU)) > 0 + EnablePProfSQLCPU.Store(enable) + tracecpu.EnablePProfSQLCPU.Store(enable) return nil }, GetSession: func(s *SessionVars) (string, error) { val := "0" @@ -1250,48 +1255,47 @@ var defaultSysVars = []*SysVar{ return nil }}, // variable for top SQL feature. - {Scope: ScopeGlobal, Name: TiDBEnableTopSQL, Value: Off, Type: TypeBool, Hidden: true, GetGlobal: func(sessionVars *SessionVars) (string, error) { - // TODO(zhongzc): implement read from global config - return Off, nil + {Scope: ScopeGlobal, Name: TiDBEnableTopSQL, Value: BoolToOnOff(topsql.InstanceEnabled()), Type: TypeBool, Hidden: true, GetGlobal: func(sessionVars *SessionVars) (string, error) { + return BoolToOnOff(topsql.InstanceEnabled()), nil }, GlobalConfigName: GlobalConfigEnableTopSQL}, - {Scope: ScopeGlobal, Name: TiDBTopSQLPrecisionSeconds, Value: strconv.Itoa(DefTiDBTopSQLPrecisionSeconds), Type: TypeInt, Hidden: true, MinValue: 1, MaxValue: math.MaxInt64, GetGlobal: func(s *SessionVars) (string, error) { - return strconv.FormatInt(TopSQLVariable.PrecisionSeconds.Load(), 10), nil + {Scope: ScopeGlobal, Name: TiDBTopSQLPrecisionSeconds, Value: strconv.Itoa(tracecpu.DefPrecisionSeconds), Type: TypeInt, Hidden: true, MinValue: 1, MaxValue: math.MaxInt64, GetGlobal: func(s *SessionVars) (string, error) { + return strconv.FormatInt(tracecpu.PrecisionSeconds.Load(), 10), nil }, SetGlobal: func(vars *SessionVars, s string) error { val, err := strconv.ParseInt(s, 10, 64) if err != nil { return err } - TopSQLVariable.PrecisionSeconds.Store(val) + tracecpu.PrecisionSeconds.Store(val) return nil }}, - {Scope: ScopeGlobal, Name: TiDBTopSQLMaxStatementCount, Value: strconv.Itoa(DefTiDBTopSQLMaxStatementCount), Type: TypeInt, Hidden: true, MinValue: 0, MaxValue: 5000, GetGlobal: func(s *SessionVars) (string, error) { - return strconv.FormatInt(TopSQLVariable.MaxStatementCount.Load(), 10), nil + {Scope: ScopeGlobal, Name: TiDBTopSQLMaxStatementCount, Value: strconv.Itoa(reporter.DefMaxStatementCount), Type: TypeInt, Hidden: true, MinValue: 0, MaxValue: 5000, GetGlobal: func(s *SessionVars) (string, error) { + return strconv.FormatInt(reporter.MaxStatementCount.Load(), 10), nil }, SetGlobal: func(vars *SessionVars, s string) error { val, err := strconv.ParseInt(s, 10, 64) if err != nil { return err } - TopSQLVariable.MaxStatementCount.Store(val) + reporter.MaxStatementCount.Store(val) return nil }}, - {Scope: ScopeGlobal, Name: TiDBTopSQLMaxCollect, Value: strconv.Itoa(DefTiDBTopSQLMaxCollect), Type: TypeInt, Hidden: true, MinValue: 1, MaxValue: 500000, GetGlobal: func(s *SessionVars) (string, error) { - return strconv.FormatInt(TopSQLVariable.MaxCollect.Load(), 10), nil + {Scope: ScopeGlobal, Name: TiDBTopSQLMaxCollect, Value: strconv.Itoa(reporter.DefMaxCollect), Type: TypeInt, Hidden: true, MinValue: 1, MaxValue: 500000, GetGlobal: func(s *SessionVars) (string, error) { + return strconv.FormatInt(reporter.MaxCollect.Load(), 10), nil }, SetGlobal: func(vars *SessionVars, s string) error { val, err := strconv.ParseInt(s, 10, 64) if err != nil { return err } - TopSQLVariable.MaxCollect.Store(val) + reporter.MaxCollect.Store(val) return nil }}, - {Scope: ScopeGlobal, Name: TiDBTopSQLReportIntervalSeconds, Value: strconv.Itoa(DefTiDBTopSQLReportIntervalSeconds), Type: TypeInt, Hidden: true, MinValue: 1, MaxValue: 1 * 60 * 60, GetGlobal: func(s *SessionVars) (string, error) { - return strconv.FormatInt(TopSQLVariable.ReportIntervalSeconds.Load(), 10), nil + {Scope: ScopeGlobal, Name: TiDBTopSQLReportIntervalSeconds, Value: strconv.Itoa(reporter.DefReportIntervalSeconds), Type: TypeInt, Hidden: true, MinValue: 1, MaxValue: 1 * 60 * 60, GetGlobal: func(s *SessionVars) (string, error) { + return strconv.FormatInt(reporter.ReportIntervalSeconds.Load(), 10), nil }, SetGlobal: func(vars *SessionVars, s string) error { val, err := strconv.ParseInt(s, 10, 64) if err != nil { return err } - TopSQLVariable.ReportIntervalSeconds.Store(val) + reporter.ReportIntervalSeconds.Store(val) return nil }}, {Scope: ScopeGlobal, Name: SkipNameResolve, Value: Off, Type: TypeBool}, diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 14e5d8314f17a..ece2ee45e4c39 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -763,10 +763,6 @@ const ( DefTiDBTrackAggregateMemoryUsage = true DefTiDBEnableExchangePartition = false DefCTEMaxRecursionDepth = 1000 - DefTiDBTopSQLPrecisionSeconds = 1 - DefTiDBTopSQLMaxStatementCount = 200 - DefTiDBTopSQLMaxCollect = 10000 - DefTiDBTopSQLReportIntervalSeconds = 60 DefTiDBTmpTableMaxSize = 64 << 20 // 64MB. DefTiDBEnableLocalTxn = false DefTiDBTSOClientBatchMaxWaitTime = 0.0 // 0ms @@ -800,26 +796,8 @@ var ( CapturePlanBaseline = serverGlobalVariable{globalVal: Off} DefExecutorConcurrency = 5 MemoryUsageAlarmRatio = atomic.NewFloat64(config.GetGlobalConfig().Performance.MemoryUsageAlarmRatio) - TopSQLVariable = TopSQL{ - PrecisionSeconds: atomic.NewInt64(DefTiDBTopSQLPrecisionSeconds), - MaxStatementCount: atomic.NewInt64(DefTiDBTopSQLMaxStatementCount), - MaxCollect: atomic.NewInt64(DefTiDBTopSQLMaxCollect), - ReportIntervalSeconds: atomic.NewInt64(DefTiDBTopSQLReportIntervalSeconds), - } - EnableLocalTxn = atomic.NewBool(DefTiDBEnableLocalTxn) - MaxTSOBatchWaitInterval = atomic.NewFloat64(DefTiDBTSOClientBatchMaxWaitTime) - EnableTSOFollowerProxy = atomic.NewBool(DefTiDBEnableTSOFollowerProxy) - RestrictedReadOnly = atomic.NewBool(DefTiDBRestrictedReadOnly) + EnableLocalTxn = atomic.NewBool(DefTiDBEnableLocalTxn) + MaxTSOBatchWaitInterval = atomic.NewFloat64(DefTiDBTSOClientBatchMaxWaitTime) + EnableTSOFollowerProxy = atomic.NewBool(DefTiDBEnableTSOFollowerProxy) + RestrictedReadOnly = atomic.NewBool(DefTiDBRestrictedReadOnly) ) - -// TopSQL is the variable for control top sql feature. -type TopSQL struct { - // The refresh interval of top-sql. - PrecisionSeconds *atomic.Int64 - // The maximum number of statements kept in memory. - MaxStatementCount *atomic.Int64 - // The maximum capacity of the collect map. - MaxCollect *atomic.Int64 - // The report data interval of top-sql. - ReportIntervalSeconds *atomic.Int64 -} diff --git a/util/topsql/main_test.go b/util/topsql/main_test.go index 4e156ba4b8d36..2000c98f5e794 100644 --- a/util/topsql/main_test.go +++ b/util/topsql/main_test.go @@ -18,7 +18,6 @@ import ( "testing" "github.com/pingcap/tidb/config" - "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/util/testbridge" "github.com/pingcap/tidb/util/topsql/tracecpu" "go.uber.org/goleak" @@ -31,7 +30,7 @@ func TestMain(m *testing.M) { config.UpdateGlobal(func(conf *config.Config) { conf.TopSQL.ReceiverAddress = "mock" }) - variable.TopSQLVariable.PrecisionSeconds.Store(1) + tracecpu.PrecisionSeconds.Store(1) tracecpu.GlobalSQLCPUProfiler.Run() opts := []goleak.Option{ diff --git a/util/topsql/reporter/reporter.go b/util/topsql/reporter/reporter.go index 74003e6edfa1f..44aefb865bbe9 100644 --- a/util/topsql/reporter/reporter.go +++ b/util/topsql/reporter/reporter.go @@ -24,7 +24,6 @@ import ( "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/tracecpu" @@ -41,6 +40,27 @@ const ( grpcInitialConnWindowSize = 1 << 30 // keyOthers is the key to store the aggregation of all records that is out of Top N. keyOthers = "" + + // DefMaxStatementCount indicates that the default MaxStatementCount is 200. + DefMaxStatementCount = 200 + + // DefMaxCollect indicates that the default MaxCollect is 10000. + DefMaxCollect = 10000 + + // DefReportIntervalSeconds indicates that the default ReportIntervalSeconds is 60. + DefReportIntervalSeconds = 60 +) + +// Variables for control top sql feature. +var ( + // MaxStatementCount is the maximum number of statements kept in memory. + MaxStatementCount = atomic2.NewInt64(DefMaxStatementCount) + + // MaxCollect is the maximum capacity of the collect map. + MaxCollect = atomic2.NewInt64(DefMaxCollect) + + // ReportIntervalSeconds is he report data interval of top-sql. + ReportIntervalSeconds = atomic2.NewInt64(DefReportIntervalSeconds) ) var _ TopSQLReporter = &RemoteTopSQLReporter{} @@ -130,6 +150,9 @@ type RemoteTopSQLReporter struct { dataSinks []DataSink dataSinkRegCh chan DataSink + // paused means no active datasinks + paused atomic2.Bool + // normalizedSQLMap is a map, whose keys are SQL digest strings and values are SQLMeta. normalizedSQLMap atomic.Value // sync.Map sqlMapLength atomic2.Int64 @@ -202,7 +225,7 @@ var ( // 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() { + if tsr.sqlMapLength.Load() >= MaxCollect.Load() { ignoreExceedSQLCounter.Inc() return } @@ -220,7 +243,7 @@ func (tsr *RemoteTopSQLReporter) RegisterSQL(sqlDigest []byte, normalizedSQL str // 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() { + if tsr.planMapLength.Load() >= MaxCollect.Load() { ignoreExceedPlanCounter.Inc() return } @@ -249,6 +272,11 @@ func (tsr *RemoteTopSQLReporter) Collect(timestamp uint64, records []tracecpu.SQ } } +// IsPaused returns whether RemoteTopSQLReporter is paused. The value also means whether TopSQL is enabled. +func (tsr *RemoteTopSQLReporter) IsPaused() bool { + return tsr.paused.Load() +} + // DataSinkRegHandle returns a DataSinkRegHandle for DataSink registration. func (tsr *RemoteTopSQLReporter) DataSinkRegHandle() DataSinkRegHandle { return &RemoteDataSinkRegHandle{reporterDoneCh: tsr.ctx.Done(), registerCh: tsr.dataSinkRegCh} @@ -343,7 +371,7 @@ func (tsr *RemoteTopSQLReporter) collectWorker() { defer util.Recover("top-sql", "collectWorker", nil, false) collectedData := make(map[string]*dataPoints) - currentReportInterval := variable.TopSQLVariable.ReportIntervalSeconds.Load() + currentReportInterval := ReportIntervalSeconds.Load() reportTicker := time.NewTicker(time.Second * time.Duration(currentReportInterval)) for { tsr.acceptDataSinkRegs() @@ -352,7 +380,7 @@ func (tsr *RemoteTopSQLReporter) collectWorker() { logutil.BgLogger().Warn("[top-sql] too many datasinks, keep 10 first", zap.Int("count", len(tsr.dataSinks))) tsr.dataSinks = tsr.dataSinks[:10] } - tracecpu.GlobalSQLCPUProfiler.SetTopSQLEnabled(tsr.activeDataSinkCnt() > 0) + tsr.paused.Store(tsr.activeDataSinkCnt() == 0) select { case data := <-tsr.collectCPUDataChan: @@ -361,7 +389,7 @@ func (tsr *RemoteTopSQLReporter) collectWorker() { case <-reportTicker.C: tsr.takeDataAndSendToReportChan(&collectedData) // Update `reportTicker` if report interval changed. - if newInterval := variable.TopSQLVariable.ReportIntervalSeconds.Load(); newInterval != currentReportInterval { + if newInterval := ReportIntervalSeconds.Load(); newInterval != currentReportInterval { currentReportInterval = newInterval reportTicker.Reset(time.Second * time.Duration(currentReportInterval)) } @@ -417,7 +445,7 @@ func encodeKey(buf *bytes.Buffer, sqlDigest, planDigest []byte) string { } func getTopNRecords(records []tracecpu.SQLCPUTimeRecord) (topN, shouldEvict []tracecpu.SQLCPUTimeRecord) { - maxStmt := int(variable.TopSQLVariable.MaxStatementCount.Load()) + maxStmt := int(MaxStatementCount.Load()) if len(records) <= maxStmt { return records, nil } @@ -429,7 +457,7 @@ func getTopNRecords(records []tracecpu.SQLCPUTimeRecord) (topN, shouldEvict []tr } func getTopNDataPoints(records []*dataPoints) (topN, shouldEvict []*dataPoints) { - maxStmt := int(variable.TopSQLVariable.MaxStatementCount.Load()) + maxStmt := int(MaxStatementCount.Load()) if len(records) <= maxStmt { return records, nil } @@ -451,7 +479,7 @@ 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) + listCapacity := int(ReportIntervalSeconds.Load()/tracecpu.PrecisionSeconds.Load() + 1) if listCapacity < 1 { listCapacity = 1 } @@ -650,7 +678,7 @@ func (tsr *RemoteTopSQLReporter) doReport(data ReportData) { timeout := reportTimeout failpoint.Inject("resetTimeoutForTest", func(val failpoint.Value) { if val.(bool) { - interval := time.Duration(variable.TopSQLVariable.ReportIntervalSeconds.Load()) * time.Second + interval := time.Duration(ReportIntervalSeconds.Load()) * time.Second if interval < timeout { timeout = interval } diff --git a/util/topsql/reporter/reporter_test.go b/util/topsql/reporter/reporter_test.go index 3980c1c81e0e2..5a2e2dd5b9997 100644 --- a/util/topsql/reporter/reporter_test.go +++ b/util/topsql/reporter/reporter_test.go @@ -22,7 +22,6 @@ import ( "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/tracecpu" "github.com/pingcap/tipb/go-tipb" @@ -65,8 +64,8 @@ func mockPlanBinaryDecoderFunc(plan string) (string, error) { } func setupRemoteTopSQLReporter(maxStatementsNum, interval int, addr string) *RemoteTopSQLReporter { - variable.TopSQLVariable.MaxStatementCount.Store(int64(maxStatementsNum)) - variable.TopSQLVariable.ReportIntervalSeconds.Store(int64(interval)) + MaxStatementCount.Store(int64(maxStatementsNum)) + ReportIntervalSeconds.Store(int64(interval)) config.UpdateGlobal(func(conf *config.Config) { conf.TopSQL.ReceiverAddress = addr }) @@ -283,7 +282,7 @@ func TestCollectCapacity(t *testing.T) { return records } - variable.TopSQLVariable.MaxCollect.Store(10000) + MaxCollect.Store(10000) registerSQL(5000) require.Equal(t, int64(5000), tsr.sqlMapLength.Load()) registerPlan(1000) @@ -294,13 +293,13 @@ func TestCollectCapacity(t *testing.T) { registerPlan(20000) require.Equal(t, int64(10000), tsr.planMapLength.Load()) - variable.TopSQLVariable.MaxCollect.Store(20000) + MaxCollect.Store(20000) registerSQL(50000) require.Equal(t, int64(20000), tsr.sqlMapLength.Load()) registerPlan(50000) require.Equal(t, int64(20000), tsr.planMapLength.Load()) - variable.TopSQLVariable.MaxStatementCount.Store(5000) + MaxStatementCount.Store(5000) collectedData := make(map[string]*dataPoints) tsr.doCollect(collectedData, 1, genRecord(20000)) require.Equal(t, 5001, len(collectedData)) diff --git a/util/topsql/topsql.go b/util/topsql/topsql.go index bb4944476d542..cbe041c1327c5 100644 --- a/util/topsql/topsql.go +++ b/util/topsql/topsql.go @@ -21,7 +21,6 @@ import ( "time" "github.com/pingcap/failpoint" - "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/plancodec" @@ -53,14 +52,14 @@ func SetupTopSQL() { // register single target datasink to reporter singleTargetDataSink := reporter.NewSingleTargetDataSink() globalTopSQLReport.DataSinkRegHandle().Register(singleTargetDataSink) - - enable := len(config.GetGlobalConfig().TopSQL.ReceiverAddress) != 0 - tracecpu.GlobalSQLCPUProfiler.SetTopSQLEnabled(enable) } // InstanceEnabled is used to check if TopSQL is enabled on the current instance. func InstanceEnabled() bool { - return tracecpu.GlobalSQLCPUProfiler.GetTopSQLEnabled() + if globalTopSQLReport == nil { + return false + } + return !globalTopSQLReport.IsPaused() } // RegisterPubSubServer registers TopSQLPubSubService to the given gRPC server. diff --git a/util/topsql/topsql_test.go b/util/topsql/topsql_test.go index a365d1fefa0a9..aa93b719b7fe8 100644 --- a/util/topsql/topsql_test.go +++ b/util/topsql/topsql_test.go @@ -23,7 +23,6 @@ import ( "github.com/google/pprof/profile" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/parser" - "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/util/topsql" "github.com/pingcap/tidb/util/topsql/reporter" mockServer "github.com/pingcap/tidb/util/topsql/reporter/mock" @@ -40,9 +39,6 @@ type collectorWrapper struct { } func TestTopSQLCPUProfile(t *testing.T) { - setTopSQLEnable(true) - defer setTopSQLEnable(false) - collector := mock.NewTopSQLCollector() tracecpu.GlobalSQLCPUProfiler.SetCollector(&collectorWrapper{collector}) reqs := []struct { @@ -90,21 +86,24 @@ func TestTopSQLCPUProfile(t *testing.T) { } func TestIsEnabled(t *testing.T) { - setTopSQLEnable(false) + controller := mock.NewProfileController(false) + tracecpu.GlobalSQLCPUProfiler.SetCollector(controller) + defer tracecpu.GlobalSQLCPUProfiler.SetCollector(nil) + require.False(t, tracecpu.GlobalSQLCPUProfiler.ShouldProfile()) - setTopSQLEnable(true) + controller.SetEnabled(true) err := tracecpu.StartCPUProfile(bytes.NewBuffer(nil)) require.NoError(t, err) require.True(t, tracecpu.GlobalSQLCPUProfiler.ShouldProfile()) - setTopSQLEnable(false) + controller.SetEnabled(false) require.True(t, tracecpu.GlobalSQLCPUProfiler.ShouldProfile()) err = tracecpu.StopCPUProfile() require.NoError(t, err) - setTopSQLEnable(false) + controller.SetEnabled(false) require.False(t, tracecpu.GlobalSQLCPUProfiler.ShouldProfile()) - setTopSQLEnable(true) + controller.SetEnabled(true) require.True(t, tracecpu.GlobalSQLCPUProfiler.ShouldProfile()) } @@ -115,8 +114,8 @@ func mockPlanBinaryDecoderFunc(plan string) (string, error) { func TestTopSQLReporter(t *testing.T) { server, err := mockServer.StartMockReceiverServer() require.NoError(t, err) - variable.TopSQLVariable.MaxStatementCount.Store(200) - variable.TopSQLVariable.ReportIntervalSeconds.Store(1) + reporter.MaxStatementCount.Store(200) + reporter.ReportIntervalSeconds.Store(1) config.UpdateGlobal(func(conf *config.Config) { conf.TopSQL.ReceiverAddress = server.Address() }) @@ -185,8 +184,8 @@ func TestTopSQLReporter(t *testing.T) { } func TestTopSQLPubSub(t *testing.T) { - variable.TopSQLVariable.MaxStatementCount.Store(200) - variable.TopSQLVariable.ReportIntervalSeconds.Store(1) + reporter.MaxStatementCount.Store(200) + reporter.ReportIntervalSeconds.Store(1) report := reporter.NewRemoteTopSQLReporter(mockPlanBinaryDecoderFunc) defer report.Close() @@ -365,10 +364,6 @@ func TestMaxSQLAndPlanTest(t *testing.T) { require.Empty(t, cPlan) } -func setTopSQLEnable(enabled bool) { - tracecpu.GlobalSQLCPUProfiler.SetTopSQLEnabled(enabled) -} - func mockExecuteSQL(sql, plan string) { ctx := context.Background() sqlDigest := mock.GenSQLDigest(sql) diff --git a/util/topsql/tracecpu/main_test.go b/util/topsql/tracecpu/main_test.go index c93e76eacf011..5983a84f6df9a 100644 --- a/util/topsql/tracecpu/main_test.go +++ b/util/topsql/tracecpu/main_test.go @@ -20,9 +20,9 @@ import ( "github.com/google/pprof/profile" "github.com/pingcap/tidb/config" - "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/util/testbridge" "github.com/pingcap/tidb/util/topsql/tracecpu" + "github.com/pingcap/tidb/util/topsql/tracecpu/mock" "github.com/stretchr/testify/require" "go.uber.org/goleak" ) @@ -33,7 +33,7 @@ func TestMain(m *testing.M) { config.UpdateGlobal(func(conf *config.Config) { conf.TopSQL.ReceiverAddress = "mock" }) - variable.TopSQLVariable.PrecisionSeconds.Store(1) + tracecpu.PrecisionSeconds.Store(1) tracecpu.GlobalSQLCPUProfiler.Run() opts := []goleak.Option{ @@ -51,10 +51,11 @@ func TestPProfCPUProfile(t *testing.T) { err := tracecpu.StartCPUProfile(buf) require.NoError(t, err) // enable top sql. - tracecpu.GlobalSQLCPUProfiler.SetTopSQLEnabled(true) + ctl := mock.NewProfileController(true) + tracecpu.GlobalSQLCPUProfiler.SetCollector(ctl) + defer tracecpu.GlobalSQLCPUProfiler.SetCollector(nil) err = tracecpu.StopCPUProfile() require.NoError(t, err) _, err = profile.Parse(buf) require.NoError(t, err) - tracecpu.GlobalSQLCPUProfiler.SetTopSQLEnabled(false) } diff --git a/util/topsql/tracecpu/mock/mock.go b/util/topsql/tracecpu/mock/mock.go index e9b95bae93740..22458dc84c0fc 100644 --- a/util/topsql/tracecpu/mock/mock.go +++ b/util/topsql/tracecpu/mock/mock.go @@ -76,6 +76,11 @@ func (c *TopSQLCollector) Collect(ts uint64, stats []tracecpu.SQLCPUTimeRecord) } } +// IsPaused implements tracecpu.Collector +func (c *TopSQLCollector) IsPaused() bool { + return false +} + // GetSQLStatsBySQLWithRetry uses for testing. func (c *TopSQLCollector) GetSQLStatsBySQLWithRetry(sql string, planIsNotNull bool) []*tracecpu.SQLCPUTimeRecord { after := time.After(time.Second * 10) @@ -188,3 +193,28 @@ func GenSQLDigest(sql string) *parser.Digest { _, digest := parser.NormalizeDigest(sql) return digest } + +// ProfileController is a mock collector only for controlling profile. +type ProfileController struct { + paused *atomic.Bool +} + +// NewProfileController creates a new ProfileController +func NewProfileController(enabled bool) *ProfileController { + return &ProfileController{paused: atomic.NewBool(!enabled)} +} + +var _ tracecpu.Collector = &ProfileController{} + +// Collect implements tracecpu.Collector +func (*ProfileController) Collect(_ts uint64, _stats []tracecpu.SQLCPUTimeRecord) {} + +// IsPaused implements tracecpu.Collector +func (c *ProfileController) IsPaused() bool { + return c.paused.Load() +} + +// SetEnabled sets enabled +func (c *ProfileController) SetEnabled(v bool) { + c.paused.Store(!v) +} diff --git a/util/topsql/tracecpu/profile.go b/util/topsql/tracecpu/profile.go index 029a3c7753363..e4774a046cc40 100644 --- a/util/topsql/tracecpu/profile.go +++ b/util/topsql/tracecpu/profile.go @@ -24,16 +24,14 @@ import ( "runtime/pprof" "strconv" "sync" - "sync/atomic" "time" "github.com/google/pprof/profile" "github.com/pingcap/tidb/metrics" - "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/hack" "github.com/pingcap/tidb/util/logutil" - atomic2 "go.uber.org/atomic" + "go.uber.org/atomic" "go.uber.org/zap" ) @@ -41,16 +39,31 @@ const ( labelSQL = "sql" labelSQLDigest = "sql_digest" labelPlanDigest = "plan_digest" + + // DefPrecisionSeconds indicates that the default PrecisionSeconds is 10000 + DefPrecisionSeconds = 1 ) -// GlobalSQLCPUProfiler is the global SQL stats profiler. -var GlobalSQLCPUProfiler = newSQLCPUProfiler() +var ( + // GlobalSQLCPUProfiler is the global SQL stats profiler. + GlobalSQLCPUProfiler = newSQLCPUProfiler() + + // EnablePProfSQLCPU is true means want to keep the `sql` label. Otherwise, remove the `sql` label. + EnablePProfSQLCPU = atomic.NewBool(false) + + // PrecisionSeconds indicates profile interval. + PrecisionSeconds = atomic.NewInt64(DefPrecisionSeconds) +) // 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) + + // IsPaused indicates that the Collector is not expecting to receive records for now, + // and may resume in the future. + IsPaused() bool } // SQLCPUTimeRecord represents a single record of how much cpu time a sql plan consumes in one second. @@ -72,8 +85,10 @@ type sqlCPUProfiler struct { ept *exportProfileTask } - topSQLEnabled atomic2.Bool - collector atomic.Value + collector struct { + sync.Mutex + v Collector + } } var ( @@ -99,23 +114,17 @@ func (sp *sqlCPUProfiler) Run() { } func (sp *sqlCPUProfiler) SetCollector(c Collector) { - sp.collector.Store(c) -} + sp.collector.Lock() + defer sp.collector.Unlock() -func (sp *sqlCPUProfiler) GetCollector() Collector { - c, ok := sp.collector.Load().(Collector) - if !ok || c == nil { - return nil - } - return c + sp.collector.v = c } -func (sp *sqlCPUProfiler) SetTopSQLEnabled(v bool) { - sp.topSQLEnabled.Store(v) -} +func (sp *sqlCPUProfiler) GetCollector() Collector { + sp.collector.Lock() + defer sp.collector.Unlock() -func (sp *sqlCPUProfiler) GetTopSQLEnabled() bool { - return sp.topSQLEnabled.Load() + return sp.collector.v } func (sp *sqlCPUProfiler) startCPUProfileWorker() { @@ -131,7 +140,7 @@ func (sp *sqlCPUProfiler) startCPUProfileWorker() { func (sp *sqlCPUProfiler) doCPUProfile() { metrics.TopSQLProfileCounter.Inc() - intervalSecond := variable.TopSQLVariable.PrecisionSeconds.Load() + intervalSecond := PrecisionSeconds.Load() task := sp.newProfileTask() if err := pprof.StartCPUProfile(task.buf); err != nil { // Sleep a while before retry. @@ -291,7 +300,15 @@ func (sp *sqlCPUProfiler) hasExportProfileTask() bool { // ShouldProfile return true if it's required to profile. It exports for tests. func (sp *sqlCPUProfiler) ShouldProfile() bool { - return sp.topSQLEnabled.Load() || sp.hasExportProfileTask() + if sp.hasExportProfileTask() { + return true + } + + c := sp.GetCollector() + if c == nil { + return false + } + return !c.IsPaused() } // StartCPUProfile same like pprof.StartCPUProfile. @@ -350,12 +367,11 @@ func (sp *sqlCPUProfiler) stopExportCPUProfile() error { // removeLabel uses to remove labels for export cpu profile data. // Since the sql_digest and plan_digest label is strange for other users. -// If `variable.EnablePProfSQLCPU` is true means wanto keep the `sql` label, otherwise, remove the `sql` label too. func (sp *sqlCPUProfiler) removeLabel(p *profile.Profile) { if p == nil { return } - keepLabelSQL := variable.EnablePProfSQLCPU.Load() + keepLabelSQL := EnablePProfSQLCPU.Load() for _, s := range p.Sample { for k := range s.Label { switch k { From 3b72c97b58413a67053402d8a2b70407f2c45547 Mon Sep 17 00:00:00 2001 From: Zhenchi Date: Fri, 10 Dec 2021 17:40:10 +0800 Subject: [PATCH 39/47] fix unbuffered Signed-off-by: Zhenchi --- util/topsql/reporter/mock/{publisher.go => pubsub.go} | 0 util/topsql/reporter/pubsub.go | 2 +- util/topsql/reporter/single_target.go | 2 +- util/topsql/topsql.go | 4 +--- util/topsql/tracecpu/profile.go | 10 +++++----- 5 files changed, 8 insertions(+), 10 deletions(-) rename util/topsql/reporter/mock/{publisher.go => pubsub.go} (100%) diff --git a/util/topsql/reporter/mock/publisher.go b/util/topsql/reporter/mock/pubsub.go similarity index 100% rename from util/topsql/reporter/mock/publisher.go rename to util/topsql/reporter/mock/pubsub.go diff --git a/util/topsql/reporter/pubsub.go b/util/topsql/reporter/pubsub.go index cfaa5ad47bcdf..b78e865da0a72 100644 --- a/util/topsql/reporter/pubsub.go +++ b/util/topsql/reporter/pubsub.go @@ -69,7 +69,7 @@ func newPubSubDataSink( ) *pubSubDataSink { return &pubSubDataSink{ stream: stream, - sendTaskCh: make(chan sendTask), + sendTaskCh: make(chan sendTask, 1), isDown: atomic.NewBool(false), } } diff --git a/util/topsql/reporter/single_target.go b/util/topsql/reporter/single_target.go index 27ed687947689..ac551a108fc39 100644 --- a/util/topsql/reporter/single_target.go +++ b/util/topsql/reporter/single_target.go @@ -45,7 +45,7 @@ type sendTask struct { // func NewSingleTargetDataSink() *SingleTargetDataSink { dataSink := &SingleTargetDataSink{ - sendTaskCh: make(chan sendTask), + sendTaskCh: make(chan sendTask, 1), } go util.WithRecovery(dataSink.run, nil) diff --git a/util/topsql/topsql.go b/util/topsql/topsql.go index cbe041c1327c5..9037a8d7a7fd5 100644 --- a/util/topsql/topsql.go +++ b/util/topsql/topsql.go @@ -38,9 +38,7 @@ const ( MaxPlanTextSize = 32 * 1024 ) -var ( - globalTopSQLReport reporter.TopSQLReporter -) +var globalTopSQLReport reporter.TopSQLReporter // SetupTopSQL sets up the top-sql worker. func SetupTopSQL() { diff --git a/util/topsql/tracecpu/profile.go b/util/topsql/tracecpu/profile.go index e4774a046cc40..238314064f3aa 100644 --- a/util/topsql/tracecpu/profile.go +++ b/util/topsql/tracecpu/profile.go @@ -115,16 +115,16 @@ func (sp *sqlCPUProfiler) Run() { func (sp *sqlCPUProfiler) SetCollector(c Collector) { sp.collector.Lock() - defer sp.collector.Unlock() - sp.collector.v = c + sp.collector.Unlock() } -func (sp *sqlCPUProfiler) GetCollector() Collector { +func (sp *sqlCPUProfiler) GetCollector() (res Collector) { sp.collector.Lock() - defer sp.collector.Unlock() + res = sp.collector.v + sp.collector.Unlock() - return sp.collector.v + return } func (sp *sqlCPUProfiler) startCPUProfileWorker() { From df42ffebca465114169d3169b8322d0d7bf44431 Mon Sep 17 00:00:00 2001 From: Zhenchi Date: Fri, 10 Dec 2021 18:01:28 +0800 Subject: [PATCH 40/47] fix InstanceEnabled Signed-off-by: Zhenchi --- server/tidb_test.go | 1 + util/topsql/topsql.go | 5 +++-- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/server/tidb_test.go b/server/tidb_test.go index 9bab2e03dc0a2..c1ebda37c1fca 100644 --- a/server/tidb_test.go +++ b/server/tidb_test.go @@ -1316,6 +1316,7 @@ func TestTopSQLCPUProfile(t *testing.T) { collector := mockTopSQLTraceCPU.NewTopSQLCollector() tracecpu.GlobalSQLCPUProfiler.SetCollector(&collectorWrapper{collector}) + defer tracecpu.GlobalSQLCPUProfiler.SetCollector(nil) dbt := testkit.NewDBTestKit(t, db) dbt.MustExec("drop database if exists topsql") diff --git a/util/topsql/topsql.go b/util/topsql/topsql.go index 9037a8d7a7fd5..3e8e34b8705de 100644 --- a/util/topsql/topsql.go +++ b/util/topsql/topsql.go @@ -54,10 +54,11 @@ func SetupTopSQL() { // InstanceEnabled is used to check if TopSQL is enabled on the current instance. func InstanceEnabled() bool { - if globalTopSQLReport == nil { + collector := tracecpu.GlobalSQLCPUProfiler.GetCollector() + if collector == nil { return false } - return !globalTopSQLReport.IsPaused() + return !collector.IsPaused() } // RegisterPubSubServer registers TopSQLPubSubService to the given gRPC server. From 2a7111fed74dfa5ffce4277167f0e987a0f781fc Mon Sep 17 00:00:00 2001 From: Zhenchi Date: Fri, 10 Dec 2021 18:37:29 +0800 Subject: [PATCH 41/47] fix init NewRemoteTopSQLReporter Signed-off-by: Zhenchi --- util/topsql/reporter/reporter.go | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/util/topsql/reporter/reporter.go b/util/topsql/reporter/reporter.go index 44aefb865bbe9..7e3282e717caf 100644 --- a/util/topsql/reporter/reporter.go +++ b/util/topsql/reporter/reporter.go @@ -150,8 +150,8 @@ type RemoteTopSQLReporter struct { dataSinks []DataSink dataSinkRegCh chan DataSink - // paused means no active datasinks - paused atomic2.Bool + // paused indicates no active datasinks + paused *atomic2.Bool // normalizedSQLMap is a map, whose keys are SQL digest strings and values are SQLMeta. normalizedSQLMap atomic.Value // sync.Map @@ -184,6 +184,7 @@ func NewRemoteTopSQLReporter(planBinaryDecodeFunc planBinaryDecodeFunc) *RemoteT ctx: ctx, cancel: cancel, + paused: atomic2.NewBool(true), dataSinkRegCh: make(chan DataSink), decodePlan: planBinaryDecodeFunc, From 127c72e006c759f7a6584f413b0ad6a35aa61ee7 Mon Sep 17 00:00:00 2001 From: Zhenchi Date: Fri, 10 Dec 2021 20:49:56 +0800 Subject: [PATCH 42/47] use `globalTopSQLReport` to judge topsql enabled Signed-off-by: Zhenchi --- executor/executor_test.go | 9 ++-- executor/main_test.go | 3 ++ server/tidb_test.go | 16 +++---- tidb-server/main.go | 5 ++- util/topsql/main_test.go | 9 ---- util/topsql/reporter/mock/pubsub.go | 28 ++++++------ util/topsql/topsql.go | 10 ++--- util/topsql/topsql_test.go | 64 +++++++++++----------------- util/topsql/tracecpu/main_test.go | 38 +---------------- util/topsql/tracecpu/mock/mock.go | 11 ++++- util/topsql/tracecpu/profile_test.go | 61 ++++++++++++++++++++++++++ 11 files changed, 134 insertions(+), 120 deletions(-) create mode 100644 util/topsql/tracecpu/profile_test.go diff --git a/executor/executor_test.go b/executor/executor_test.go index e8148accdc9b3..9ab7c6ac27275 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -82,8 +82,8 @@ import ( "github.com/pingcap/tidb/util/testkit" "github.com/pingcap/tidb/util/testutil" "github.com/pingcap/tidb/util/timeutil" - "github.com/pingcap/tidb/util/topsql/tracecpu" - tracecpumock "github.com/pingcap/tidb/util/topsql/tracecpu/mock" + "github.com/pingcap/tidb/util/topsql" + topsqlmock "github.com/pingcap/tidb/util/topsql/tracecpu/mock" "github.com/pingcap/tipb/go-tipb" "github.com/stretchr/testify/require" "github.com/tikv/client-go/v2/oracle" @@ -8720,9 +8720,8 @@ func (s *testResourceTagSuite) TestResourceGroupTag(c *C) { tk.MustExec("create table t(a int, b int, unique index idx(a));") tbInfo := testGetTableByName(c, tk.Se, "test", "t") - ctl := tracecpumock.NewProfileController(true) - tracecpu.GlobalSQLCPUProfiler.SetCollector(ctl) - defer tracecpu.GlobalSQLCPUProfiler.SetCollector(nil) + topsql.SetupTopSQL(topsqlmock.NewTopSQLCollector()) + defer topsql.Close() c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/mockstore/unistore/unistoreRPCClientSendHook", `return(true)`), IsNil) defer failpoint.Disable("github.com/pingcap/tidb/store/mockstore/unistore/unistoreRPCClientSendHook") diff --git a/executor/main_test.go b/executor/main_test.go index bff65b72d6a2d..7a9e377ae23c1 100644 --- a/executor/main_test.go +++ b/executor/main_test.go @@ -57,6 +57,9 @@ func TestMain(m *testing.M) { goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), goleak.IgnoreTopFunction("gopkg.in/natefinch/lumberjack%2ev2.(*Logger).millRun"), + goleak.IgnoreTopFunction("time.Sleep"), + goleak.IgnoreTopFunction("runtime/pprof.readProfile"), + goleak.IgnoreTopFunction("github.com/pingcap/tidb/util/topsql/tracecpu.(*sqlCPUProfiler).startAnalyzeProfileWorker"), } callback := func(i int) int { testDataMap.GenerateOutputIfNeeded() diff --git a/server/tidb_test.go b/server/tidb_test.go index c1ebda37c1fca..ce1f28b40b2e1 100644 --- a/server/tidb_test.go +++ b/server/tidb_test.go @@ -49,9 +49,9 @@ import ( "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/plancodec" + "github.com/pingcap/tidb/util/topsql" "github.com/pingcap/tidb/util/topsql/reporter" mockTopSQLReporter "github.com/pingcap/tidb/util/topsql/reporter/mock" - "github.com/pingcap/tidb/util/topsql/tracecpu" mockTopSQLTraceCPU "github.com/pingcap/tidb/util/topsql/tracecpu/mock" "github.com/stretchr/testify/require" ) @@ -131,8 +131,6 @@ func createTidbTestTopSQLSuite(t *testing.T) (*tidbTestTopSQLSuite, func()) { dbt.MustExec("set @@global.tidb_top_sql_report_interval_seconds=2;") dbt.MustExec("set @@global.tidb_top_sql_max_statement_count=5;") - tracecpu.GlobalSQLCPUProfiler.Run() - return ts, cleanup } @@ -1315,8 +1313,8 @@ func TestTopSQLCPUProfile(t *testing.T) { }() collector := mockTopSQLTraceCPU.NewTopSQLCollector() - tracecpu.GlobalSQLCPUProfiler.SetCollector(&collectorWrapper{collector}) - defer tracecpu.GlobalSQLCPUProfiler.SetCollector(nil) + topsql.SetupTopSQL(&collectorWrapper{collector}) + defer topsql.Close() dbt := testkit.NewDBTestKit(t, db) dbt.MustExec("drop database if exists topsql") @@ -1559,15 +1557,15 @@ func TestTopSQLReceiver(t *testing.T) { }) } + report := reporter.NewRemoteTopSQLReporter(plancodec.DecodeNormalizedPlan) + topsql.SetupTopSQL(report) + defer topsql.Close() + setTopSQLReceiverAddress("") dbt.MustExec("set @@global.tidb_top_sql_precision_seconds=1;") dbt.MustExec("set @@global.tidb_top_sql_report_interval_seconds=2;") dbt.MustExec("set @@global.tidb_top_sql_max_statement_count=5;") - r := reporter.NewRemoteTopSQLReporter(plancodec.DecodeNormalizedPlan) - r.DataSinkRegHandle().Register(reporter.NewSingleTargetDataSink()) - tracecpu.GlobalSQLCPUProfiler.SetCollector(&collectorWrapper{r}) - // TODO: change to ensure that the right sql statements are reported, not just counts checkFn := func(n int) { records := receiverServer.GetLatestRecords() diff --git a/tidb-server/main.go b/tidb-server/main.go index f53b1f26f0e88..edc2472b93312 100644 --- a/tidb-server/main.go +++ b/tidb-server/main.go @@ -59,6 +59,7 @@ import ( "github.com/pingcap/tidb/util/kvcache" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/memory" + "github.com/pingcap/tidb/util/plancodec" "github.com/pingcap/tidb/util/printer" "github.com/pingcap/tidb/util/sem" "github.com/pingcap/tidb/util/signal" @@ -66,6 +67,7 @@ import ( storageSys "github.com/pingcap/tidb/util/sys/storage" "github.com/pingcap/tidb/util/systimemon" "github.com/pingcap/tidb/util/topsql" + "github.com/pingcap/tidb/util/topsql/reporter" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/push" "github.com/tikv/client-go/v2/tikv" @@ -209,7 +211,8 @@ func main() { cleanup(svr, storage, dom, graceful) close(exited) }) - topsql.SetupTopSQL() + report := reporter.NewRemoteTopSQLReporter(plancodec.DecodeNormalizedPlan) + topsql.SetupTopSQL(report) terror.MustNil(svr.Run()) <-exited syncLog() diff --git a/util/topsql/main_test.go b/util/topsql/main_test.go index 2000c98f5e794..7d2142aadea6c 100644 --- a/util/topsql/main_test.go +++ b/util/topsql/main_test.go @@ -17,22 +17,13 @@ package topsql import ( "testing" - "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/util/testbridge" - "github.com/pingcap/tidb/util/topsql/tracecpu" "go.uber.org/goleak" ) func TestMain(m *testing.M) { testbridge.WorkaroundGoCheckFlags() - // set up - config.UpdateGlobal(func(conf *config.Config) { - conf.TopSQL.ReceiverAddress = "mock" - }) - tracecpu.PrecisionSeconds.Store(1) - tracecpu.GlobalSQLCPUProfiler.Run() - opts := []goleak.Option{ goleak.IgnoreTopFunction("time.Sleep"), goleak.IgnoreTopFunction("runtime/pprof.readProfile"), diff --git a/util/topsql/reporter/mock/pubsub.go b/util/topsql/reporter/mock/pubsub.go index 1ba26c967077b..493d95c17f827 100644 --- a/util/topsql/reporter/mock/pubsub.go +++ b/util/topsql/reporter/mock/pubsub.go @@ -19,41 +19,43 @@ import ( "net" "github.com/pingcap/tidb/util/logutil" - "github.com/pingcap/tipb/go-tipb" "go.uber.org/zap" "google.golang.org/grpc" ) type mockPubSubServer struct { addr string + listen net.Listener grpcServer *grpc.Server } -// StartMockPubSubServer starts the mock publisher server. -func StartMockPubSubServer( - service tipb.TopSQLPubSubServer, -) (*mockPubSubServer, error) { +// NewMockPubSubServer creates a mock publisher server. +func NewMockPubSubServer() (*mockPubSubServer, error) { addr := "127.0.0.1:0" lis, err := net.Listen("tcp", addr) if err != nil { return nil, err } server := grpc.NewServer() - tipb.RegisterTopSQLPubSubServer(server, service) - - go func() { - err := server.Serve(lis) - if err != nil { - logutil.BgLogger().Warn("[top-sql] mock pubsub server serve failed", zap.Error(err)) - } - }() return &mockPubSubServer{ addr: fmt.Sprintf("127.0.0.1:%d", lis.Addr().(*net.TCPAddr).Port), + listen: lis, grpcServer: server, }, nil } +func (svr *mockPubSubServer) Serve() { + err := svr.grpcServer.Serve(svr.listen) + if err != nil { + logutil.BgLogger().Warn("[top-sql] mock pubsub server serve failed", zap.Error(err)) + } +} + +func (svr *mockPubSubServer) Server() *grpc.Server { + return svr.grpcServer +} + func (svr *mockPubSubServer) Address() string { return svr.addr } diff --git a/util/topsql/topsql.go b/util/topsql/topsql.go index 3e8e34b8705de..fc3d554681869 100644 --- a/util/topsql/topsql.go +++ b/util/topsql/topsql.go @@ -23,7 +23,6 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/util/logutil" - "github.com/pingcap/tidb/util/plancodec" "github.com/pingcap/tidb/util/topsql/reporter" "github.com/pingcap/tidb/util/topsql/tracecpu" "github.com/pingcap/tipb/go-tipb" @@ -41,8 +40,8 @@ const ( var globalTopSQLReport reporter.TopSQLReporter // SetupTopSQL sets up the top-sql worker. -func SetupTopSQL() { - globalTopSQLReport = reporter.NewRemoteTopSQLReporter(plancodec.DecodeNormalizedPlan) +func SetupTopSQL(report reporter.TopSQLReporter) { + globalTopSQLReport = report tracecpu.GlobalSQLCPUProfiler.SetCollector(globalTopSQLReport) tracecpu.GlobalSQLCPUProfiler.Run() @@ -54,11 +53,10 @@ func SetupTopSQL() { // InstanceEnabled is used to check if TopSQL is enabled on the current instance. func InstanceEnabled() bool { - collector := tracecpu.GlobalSQLCPUProfiler.GetCollector() - if collector == nil { + if globalTopSQLReport == nil { return false } - return !collector.IsPaused() + return !globalTopSQLReport.IsPaused() } // RegisterPubSubServer registers TopSQLPubSubService to the given gRPC server. diff --git a/util/topsql/topsql_test.go b/util/topsql/topsql_test.go index aa93b719b7fe8..888d622e9c09b 100644 --- a/util/topsql/topsql_test.go +++ b/util/topsql/topsql_test.go @@ -40,7 +40,10 @@ type collectorWrapper struct { func TestTopSQLCPUProfile(t *testing.T) { collector := mock.NewTopSQLCollector() - tracecpu.GlobalSQLCPUProfiler.SetCollector(&collectorWrapper{collector}) + report := &collectorWrapper{collector} + topsql.SetupTopSQL(report) + defer topsql.Close() + reqs := []struct { sql string plan string @@ -85,28 +88,6 @@ func TestTopSQLCPUProfile(t *testing.T) { } } -func TestIsEnabled(t *testing.T) { - controller := mock.NewProfileController(false) - tracecpu.GlobalSQLCPUProfiler.SetCollector(controller) - defer tracecpu.GlobalSQLCPUProfiler.SetCollector(nil) - - require.False(t, tracecpu.GlobalSQLCPUProfiler.ShouldProfile()) - - controller.SetEnabled(true) - err := tracecpu.StartCPUProfile(bytes.NewBuffer(nil)) - require.NoError(t, err) - require.True(t, tracecpu.GlobalSQLCPUProfiler.ShouldProfile()) - controller.SetEnabled(false) - require.True(t, tracecpu.GlobalSQLCPUProfiler.ShouldProfile()) - err = tracecpu.StopCPUProfile() - require.NoError(t, err) - - controller.SetEnabled(false) - require.False(t, tracecpu.GlobalSQLCPUProfiler.ShouldProfile()) - controller.SetEnabled(true) - require.True(t, tracecpu.GlobalSQLCPUProfiler.ShouldProfile()) -} - func mockPlanBinaryDecoderFunc(plan string) (string, error) { return plan, nil } @@ -120,13 +101,10 @@ func TestTopSQLReporter(t *testing.T) { conf.TopSQL.ReceiverAddress = server.Address() }) - report := reporter.NewRemoteTopSQLReporter(mockPlanBinaryDecoderFunc) - defer report.Close() - - dataSink := reporter.NewSingleTargetDataSink() - report.DataSinkRegHandle().Register(dataSink) + report := &collectorWrapper{reporter.NewRemoteTopSQLReporter(mockPlanBinaryDecoderFunc)} + topsql.SetupTopSQL(report) + defer topsql.Close() - tracecpu.GlobalSQLCPUProfiler.SetCollector(&collectorWrapper{report}) reqs := []struct { sql string plan string @@ -188,14 +166,16 @@ func TestTopSQLPubSub(t *testing.T) { reporter.ReportIntervalSeconds.Store(1) report := reporter.NewRemoteTopSQLReporter(mockPlanBinaryDecoderFunc) - defer report.Close() + topsql.SetupTopSQL(report) + defer topsql.Close() - pubsub := reporter.NewTopSQLPubSubService(report.DataSinkRegHandle()) - server, err := mockServer.StartMockPubSubServer(pubsub) + server, err := mockServer.NewMockPubSubServer() require.NoError(t, err) + + topsql.RegisterPubSubServer(server.Server()) + go server.Serve() defer server.Stop() - tracecpu.GlobalSQLCPUProfiler.SetCollector(&collectorWrapper{report}) conn, err := grpc.Dial( server.Address(), grpc.WithBlock(), @@ -306,17 +286,19 @@ func TestTopSQLPubSub(t *testing.T) { func TestTopSQLPubSubReporterStopBeforePubSub(t *testing.T) { report := reporter.NewRemoteTopSQLReporter(mockPlanBinaryDecoderFunc) + topsql.SetupTopSQL(report) - pubsub := reporter.NewTopSQLPubSubService(report.DataSinkRegHandle()) - server, err := mockServer.StartMockPubSubServer(pubsub) - defer server.Stop() + server, err := mockServer.NewMockPubSubServer() require.NoError(t, err) - // close reporter first - report.Close() + topsql.RegisterPubSubServer(server.Server()) + go server.Serve() + defer server.Stop() + + // stop topsql first + topsql.Close() // try to subscribe - tracecpu.GlobalSQLCPUProfiler.SetCollector(&collectorWrapper{report}) conn, err := grpc.Dial(server.Address(), grpc.WithBlock(), grpc.WithInsecure()) require.NoError(t, err) defer conn.Close() @@ -333,7 +315,9 @@ func TestTopSQLPubSubReporterStopBeforePubSub(t *testing.T) { func TestMaxSQLAndPlanTest(t *testing.T) { collector := mock.NewTopSQLCollector() - tracecpu.GlobalSQLCPUProfiler.SetCollector(&collectorWrapper{collector}) + report := &collectorWrapper{collector} + topsql.SetupTopSQL(report) + defer topsql.Close() ctx := context.Background() diff --git a/util/topsql/tracecpu/main_test.go b/util/topsql/tracecpu/main_test.go index 5983a84f6df9a..84070228d7bce 100644 --- a/util/topsql/tracecpu/main_test.go +++ b/util/topsql/tracecpu/main_test.go @@ -12,50 +12,16 @@ // See the License for the specific language governing permissions and // limitations under the License. -package tracecpu_test +package tracecpu import ( - "bytes" "testing" - "github.com/google/pprof/profile" - "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/util/testbridge" - "github.com/pingcap/tidb/util/topsql/tracecpu" - "github.com/pingcap/tidb/util/topsql/tracecpu/mock" - "github.com/stretchr/testify/require" "go.uber.org/goleak" ) func TestMain(m *testing.M) { testbridge.WorkaroundGoCheckFlags() - - config.UpdateGlobal(func(conf *config.Config) { - conf.TopSQL.ReceiverAddress = "mock" - }) - tracecpu.PrecisionSeconds.Store(1) - tracecpu.GlobalSQLCPUProfiler.Run() - - opts := []goleak.Option{ - goleak.IgnoreTopFunction("time.Sleep"), - goleak.IgnoreTopFunction("runtime/pprof.readProfile"), - goleak.IgnoreTopFunction("internal/poll.runtime_pollWait"), - goleak.IgnoreTopFunction("github.com/pingcap/tidb/util/topsql/tracecpu.(*sqlCPUProfiler).startAnalyzeProfileWorker"), - } - - goleak.VerifyTestMain(m, opts...) -} - -func TestPProfCPUProfile(t *testing.T) { - buf := bytes.NewBuffer(nil) - err := tracecpu.StartCPUProfile(buf) - require.NoError(t, err) - // enable top sql. - ctl := mock.NewProfileController(true) - tracecpu.GlobalSQLCPUProfiler.SetCollector(ctl) - defer tracecpu.GlobalSQLCPUProfiler.SetCollector(nil) - err = tracecpu.StopCPUProfile() - require.NoError(t, err) - _, err = profile.Parse(buf) - require.NoError(t, err) + goleak.VerifyTestMain(m) } diff --git a/util/topsql/tracecpu/mock/mock.go b/util/topsql/tracecpu/mock/mock.go index 22458dc84c0fc..58a933d103aa3 100644 --- a/util/topsql/tracecpu/mock/mock.go +++ b/util/topsql/tracecpu/mock/mock.go @@ -178,7 +178,7 @@ func (c *TopSQLCollector) WaitCollectCnt(count int64) { // DataSinkRegHandle implements the interface. func (c *TopSQLCollector) DataSinkRegHandle() reporter.DataSinkRegHandle { - return nil + return &noopRegHandle{} } // Close implements the interface. @@ -194,6 +194,15 @@ func GenSQLDigest(sql string) *parser.Digest { return digest } +type noopRegHandle struct{} + +// Register implements reporter.DataSinkRegHandle +func (noopRegHandle) Register(dataSink reporter.DataSink) { + dataSink.Close() +} + +var _ reporter.DataSinkRegHandle = &noopRegHandle{} + // ProfileController is a mock collector only for controlling profile. type ProfileController struct { paused *atomic.Bool diff --git a/util/topsql/tracecpu/profile_test.go b/util/topsql/tracecpu/profile_test.go new file mode 100644 index 0000000000000..35419288b850d --- /dev/null +++ b/util/topsql/tracecpu/profile_test.go @@ -0,0 +1,61 @@ +// 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 tracecpu_test + +import ( + "bytes" + "testing" + + "github.com/google/pprof/profile" + "github.com/pingcap/tidb/util/topsql/tracecpu" + "github.com/pingcap/tidb/util/topsql/tracecpu/mock" + "github.com/stretchr/testify/require" +) + +func TestShouldProfile(t *testing.T) { + controller := mock.NewProfileController(false) + tracecpu.GlobalSQLCPUProfiler.SetCollector(controller) + defer tracecpu.GlobalSQLCPUProfiler.SetCollector(nil) + + require.False(t, tracecpu.GlobalSQLCPUProfiler.ShouldProfile()) + + controller.SetEnabled(true) + err := tracecpu.StartCPUProfile(bytes.NewBuffer(nil)) + require.NoError(t, err) + require.True(t, tracecpu.GlobalSQLCPUProfiler.ShouldProfile()) + controller.SetEnabled(false) + require.True(t, tracecpu.GlobalSQLCPUProfiler.ShouldProfile()) + err = tracecpu.StopCPUProfile() + require.NoError(t, err) + + controller.SetEnabled(false) + require.False(t, tracecpu.GlobalSQLCPUProfiler.ShouldProfile()) + controller.SetEnabled(true) + require.True(t, tracecpu.GlobalSQLCPUProfiler.ShouldProfile()) +} + +func TestPProfCPUProfile(t *testing.T) { + buf := bytes.NewBuffer(nil) + err := tracecpu.StartCPUProfile(buf) + require.NoError(t, err) + // enable top sql. + ctl := mock.NewProfileController(true) + tracecpu.GlobalSQLCPUProfiler.SetCollector(ctl) + defer tracecpu.GlobalSQLCPUProfiler.SetCollector(nil) + err = tracecpu.StopCPUProfile() + require.NoError(t, err) + _, err = profile.Parse(buf) + require.NoError(t, err) +} From 04e6a5ad25e1c4c75ee75cdbdd7d757a66397eda Mon Sep 17 00:00:00 2001 From: Zhenchi Date: Mon, 13 Dec 2021 11:22:28 +0800 Subject: [PATCH 43/47] remove redundant `EnablePProfSQLCPU` Signed-off-by: Zhenchi --- executor/executor.go | 3 ++- server/conn.go | 3 ++- sessionctx/variable/sysvar.go | 7 +++---- sessionctx/variable/sysvar_test.go | 3 ++- sessionctx/variable/tidb_vars.go | 2 -- util/topsql/tracecpu/profile.go | 3 +++ 6 files changed, 12 insertions(+), 9 deletions(-) diff --git a/executor/executor.go b/executor/executor.go index f7ab1869685d8..aa42170706d3d 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -65,6 +65,7 @@ import ( "github.com/pingcap/tidb/util/memory" "github.com/pingcap/tidb/util/resourcegrouptag" "github.com/pingcap/tidb/util/topsql" + "github.com/pingcap/tidb/util/topsql/tracecpu" tikverr "github.com/tikv/client-go/v2/error" tikvstore "github.com/tikv/client-go/v2/kv" tikvutil "github.com/tikv/client-go/v2/util" @@ -1726,7 +1727,7 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) { sc.InitSQLDigest(prepareStmt.NormalizedSQL, prepareStmt.SQLDigest) // For `execute stmt` SQL, should reset the SQL digest with the prepare SQL digest. goCtx := context.Background() - if variable.EnablePProfSQLCPU.Load() && len(prepareStmt.NormalizedSQL) > 0 { + if tracecpu.EnablePProfSQLCPU.Load() && len(prepareStmt.NormalizedSQL) > 0 { goCtx = pprof.WithLabels(goCtx, pprof.Labels("sql", util.QueryStrForLog(prepareStmt.NormalizedSQL))) pprof.SetGoroutineLabels(goCtx) } diff --git a/server/conn.go b/server/conn.go index 71a3dba53390b..4a53b21eb4ec0 100644 --- a/server/conn.go +++ b/server/conn.go @@ -86,6 +86,7 @@ import ( "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/memory" "github.com/pingcap/tidb/util/topsql" + "github.com/pingcap/tidb/util/topsql/tracecpu" "github.com/prometheus/client_golang/prometheus" "github.com/tikv/client-go/v2/util" "go.uber.org/zap" @@ -1248,7 +1249,7 @@ func (cc *clientConn) dispatch(ctx context.Context, data []byte) error { if topsql.InstanceEnabled() { defer pprof.SetGoroutineLabels(ctx) } - if variable.EnablePProfSQLCPU.Load() { + if tracecpu.EnablePProfSQLCPU.Load() { label := getLastStmtInConn{cc}.PProfLabel() if len(label) > 0 { defer pprof.SetGoroutineLabels(ctx) diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index b792e60ba67f6..a53c80ba753af 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -801,14 +801,13 @@ var defaultSysVars = []*SysVar{ }, GetSession: func(s *SessionVars) (string, error) { return strconv.FormatInt(int64(GlobalLogMaxDays.Load()), 10), nil }}, - {Scope: ScopeSession, Name: TiDBPProfSQLCPU, Value: strconv.Itoa(DefTiDBPProfSQLCPU), Type: TypeInt, skipInit: true, MinValue: 0, MaxValue: 1, SetSession: func(s *SessionVars, val string) error { - enable := uint32(tidbOptPositiveInt32(val, DefTiDBPProfSQLCPU)) > 0 - EnablePProfSQLCPU.Store(enable) + {Scope: ScopeSession, Name: TiDBPProfSQLCPU, Value: strconv.Itoa(tracecpu.DefEnablePProfSQLCPU), Type: TypeInt, skipInit: true, MinValue: 0, MaxValue: 1, SetSession: func(s *SessionVars, val string) error { + enable := uint32(tidbOptPositiveInt32(val, tracecpu.DefEnablePProfSQLCPU)) > 0 tracecpu.EnablePProfSQLCPU.Store(enable) return nil }, GetSession: func(s *SessionVars) (string, error) { val := "0" - if EnablePProfSQLCPU.Load() { + if tracecpu.EnablePProfSQLCPU.Load() { val = "1" } return val, nil diff --git a/sessionctx/variable/sysvar_test.go b/sessionctx/variable/sysvar_test.go index e15c9f92b92b8..75dc2a581c682 100644 --- a/sessionctx/variable/sysvar_test.go +++ b/sessionctx/variable/sysvar_test.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/parser/terror" + "github.com/pingcap/tidb/util/topsql/tracecpu" "github.com/stretchr/testify/require" ) @@ -568,7 +569,7 @@ func TestInstanceScopedVars(t *testing.T) { val, err = GetSessionOrGlobalSystemVar(vars, TiDBPProfSQLCPU) require.NoError(t, err) expected := "0" - if EnablePProfSQLCPU.Load() { + if tracecpu.EnablePProfSQLCPU.Load() { expected = "1" } require.Equal(t, expected, val) diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index ece2ee45e4c39..b44f3f66ce5e5 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -688,7 +688,6 @@ const ( DefTiDBMemQuotaIndexLookupJoin = 32 << 30 // 32GB. DefTiDBMemQuotaDistSQL = 32 << 30 // 32GB. DefTiDBGeneralLog = false - DefTiDBPProfSQLCPU = 0 DefTiDBRetryLimit = 10 DefTiDBDisableTxnAutoRetry = true DefTiDBConstraintCheckInPlace = false @@ -778,7 +777,6 @@ const ( var ( ProcessGeneralLog = atomic.NewBool(false) GlobalLogMaxDays = atomic.NewInt32(int32(config.GetGlobalConfig().Log.File.MaxDays)) - EnablePProfSQLCPU = atomic.NewBool(false) ddlReorgWorkerCounter int32 = DefTiDBDDLReorgWorkerCount ddlReorgBatchSize int32 = DefTiDBDDLReorgBatchSize ddlErrorCountlimit int64 = DefTiDBDDLErrorCountLimit diff --git a/util/topsql/tracecpu/profile.go b/util/topsql/tracecpu/profile.go index 238314064f3aa..7b2807d9ada9c 100644 --- a/util/topsql/tracecpu/profile.go +++ b/util/topsql/tracecpu/profile.go @@ -42,6 +42,9 @@ const ( // DefPrecisionSeconds indicates that the default PrecisionSeconds is 10000 DefPrecisionSeconds = 1 + + // DefEnablePProfSQLCPU indicates that the default EnablePProfSQLCPU is false + DefEnablePProfSQLCPU = 0 ) var ( From 0ce979513a24132e04e5dea1e51e8483e2aaa835 Mon Sep 17 00:00:00 2001 From: Zhenchi Date: Mon, 13 Dec 2021 13:01:14 +0800 Subject: [PATCH 44/47] Update util/topsql/reporter/reporter.go Co-authored-by: crazycs --- util/topsql/reporter/reporter.go | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/util/topsql/reporter/reporter.go b/util/topsql/reporter/reporter.go index 7e3282e717caf..de634478eb079 100644 --- a/util/topsql/reporter/reporter.go +++ b/util/topsql/reporter/reporter.go @@ -402,13 +402,12 @@ func (tsr *RemoteTopSQLReporter) collectWorker() { // acceptDataSinkRegs accepts datasink registrations func (tsr *RemoteTopSQLReporter) acceptDataSinkRegs() { -out: for { select { case dataSink := <-tsr.dataSinkRegCh: tsr.dataSinks = append(tsr.dataSinks, dataSink) default: - break out + return } } } From cc4cef3c0c32c20341c949740773bbfb6eeff755 Mon Sep 17 00:00:00 2001 From: Zhenchi Date: Mon, 13 Dec 2021 13:16:59 +0800 Subject: [PATCH 45/47] refactor Signed-off-by: Zhenchi --- util/topsql/reporter/reporter.go | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/util/topsql/reporter/reporter.go b/util/topsql/reporter/reporter.go index de634478eb079..8ca94257b8b3b 100644 --- a/util/topsql/reporter/reporter.go +++ b/util/topsql/reporter/reporter.go @@ -572,7 +572,13 @@ func (d *ReportData) hasData() bool { return len(d.CPUTimeRecords) != 0 || len(d.SQLMetas) != 0 || len(d.PlanMetas) != 0 } -func buildReportData(records []*dataPoints, sqlMap *sync.Map, planMap *sync.Map, decodePlan planBinaryDecodeFunc) (res ReportData) { +func buildReportData(records []*dataPoints, sqlMap *sync.Map, planMap *sync.Map, decodePlan planBinaryDecodeFunc) ReportData { + res := ReportData{ + CPUTimeRecords: make([]*tipb.CPUTimeRecord, 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, @@ -605,7 +611,7 @@ func buildReportData(records []*dataPoints, sqlMap *sync.Map, planMap *sync.Map, return true }) - return + return res } // reportWorker sends data to the gRPC endpoint from the `reportCollectedDataChan` one by one. From c8a6ea5945a44ade3515f3de4b67842495e053f6 Mon Sep 17 00:00:00 2001 From: Zhenchi Date: Mon, 13 Dec 2021 13:55:07 +0800 Subject: [PATCH 46/47] move tracecpu out of topsql Signed-off-by: Zhenchi --- domain/sysvar_cache.go | 2 +- executor/executor.go | 2 +- executor/set_test.go | 2 +- server/conn.go | 2 +- server/http_status.go | 2 +- sessionctx/variable/sysvar.go | 2 +- sessionctx/variable/sysvar_test.go | 2 +- util/topsql/reporter/reporter.go | 2 +- util/topsql/reporter/reporter_test.go | 2 +- util/topsql/topsql.go | 2 +- util/topsql/topsql_test.go | 2 +- util/{topsql => }/tracecpu/main_test.go | 0 util/{topsql => }/tracecpu/mock/mock.go | 2 +- util/{topsql => }/tracecpu/profile.go | 0 util/{topsql => }/tracecpu/profile_test.go | 4 ++-- 15 files changed, 14 insertions(+), 14 deletions(-) rename util/{topsql => }/tracecpu/main_test.go (100%) rename util/{topsql => }/tracecpu/mock/mock.go (99%) rename util/{topsql => }/tracecpu/profile.go (100%) rename util/{topsql => }/tracecpu/profile_test.go (94%) diff --git a/domain/sysvar_cache.go b/domain/sysvar_cache.go index 131085de2026a..5791b782d0b0a 100644 --- a/domain/sysvar_cache.go +++ b/domain/sysvar_cache.go @@ -27,7 +27,7 @@ import ( "github.com/pingcap/tidb/util/sqlexec" "github.com/pingcap/tidb/util/stmtsummary" "github.com/pingcap/tidb/util/topsql/reporter" - "github.com/pingcap/tidb/util/topsql/tracecpu" + "github.com/pingcap/tidb/util/tracecpu" storekv "github.com/tikv/client-go/v2/kv" pd "github.com/tikv/pd/client" "go.uber.org/zap" diff --git a/executor/executor.go b/executor/executor.go index aa42170706d3d..21a3b5b4952d7 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -65,7 +65,7 @@ import ( "github.com/pingcap/tidb/util/memory" "github.com/pingcap/tidb/util/resourcegrouptag" "github.com/pingcap/tidb/util/topsql" - "github.com/pingcap/tidb/util/topsql/tracecpu" + "github.com/pingcap/tidb/util/tracecpu" tikverr "github.com/tikv/client-go/v2/error" tikvstore "github.com/tikv/client-go/v2/kv" tikvutil "github.com/tikv/client-go/v2/util" diff --git a/executor/set_test.go b/executor/set_test.go index fd1da23e4ad9f..dea1182e13759 100644 --- a/executor/set_test.go +++ b/executor/set_test.go @@ -39,7 +39,7 @@ import ( "github.com/pingcap/tidb/util/testkit" "github.com/pingcap/tidb/util/testutil" "github.com/pingcap/tidb/util/topsql/reporter" - "github.com/pingcap/tidb/util/topsql/tracecpu" + "github.com/pingcap/tidb/util/tracecpu" ) func (s *testSerialSuite1) TestSetVar(c *C) { diff --git a/server/conn.go b/server/conn.go index 4a53b21eb4ec0..772e0b533b901 100644 --- a/server/conn.go +++ b/server/conn.go @@ -86,7 +86,7 @@ import ( "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/memory" "github.com/pingcap/tidb/util/topsql" - "github.com/pingcap/tidb/util/topsql/tracecpu" + "github.com/pingcap/tidb/util/tracecpu" "github.com/prometheus/client_golang/prometheus" "github.com/tikv/client-go/v2/util" "go.uber.org/zap" diff --git a/server/http_status.go b/server/http_status.go index 8248e7c54e3de..00cbf5ddcc50f 100644 --- a/server/http_status.go +++ b/server/http_status.go @@ -46,7 +46,7 @@ import ( "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/memory" "github.com/pingcap/tidb/util/printer" - "github.com/pingcap/tidb/util/topsql/tracecpu" + "github.com/pingcap/tidb/util/tracecpu" "github.com/pingcap/tidb/util/versioninfo" "github.com/prometheus/client_golang/prometheus/promhttp" "github.com/soheilhy/cmux" diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index a53c80ba753af..f0c08acefb73b 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -37,7 +37,7 @@ import ( "github.com/pingcap/tidb/util/stmtsummary" "github.com/pingcap/tidb/util/topsql" "github.com/pingcap/tidb/util/topsql/reporter" - "github.com/pingcap/tidb/util/topsql/tracecpu" + "github.com/pingcap/tidb/util/tracecpu" "github.com/pingcap/tidb/util/versioninfo" tikvstore "github.com/tikv/client-go/v2/kv" atomic2 "go.uber.org/atomic" diff --git a/sessionctx/variable/sysvar_test.go b/sessionctx/variable/sysvar_test.go index 75dc2a581c682..864a0a33867b1 100644 --- a/sessionctx/variable/sysvar_test.go +++ b/sessionctx/variable/sysvar_test.go @@ -26,7 +26,7 @@ import ( "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/parser/terror" - "github.com/pingcap/tidb/util/topsql/tracecpu" + "github.com/pingcap/tidb/util/tracecpu" "github.com/stretchr/testify/require" ) diff --git a/util/topsql/reporter/reporter.go b/util/topsql/reporter/reporter.go index 8ca94257b8b3b..45fdd684b44ee 100644 --- a/util/topsql/reporter/reporter.go +++ b/util/topsql/reporter/reporter.go @@ -26,7 +26,7 @@ import ( "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/logutil" - "github.com/pingcap/tidb/util/topsql/tracecpu" + "github.com/pingcap/tidb/util/tracecpu" "github.com/pingcap/tipb/go-tipb" "github.com/wangjohn/quickselect" atomic2 "go.uber.org/atomic" diff --git a/util/topsql/reporter/reporter_test.go b/util/topsql/reporter/reporter_test.go index 5a2e2dd5b9997..01f547a41b7b4 100644 --- a/util/topsql/reporter/reporter_test.go +++ b/util/topsql/reporter/reporter_test.go @@ -23,7 +23,7 @@ import ( "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/util/topsql/reporter/mock" - "github.com/pingcap/tidb/util/topsql/tracecpu" + "github.com/pingcap/tidb/util/tracecpu" "github.com/pingcap/tipb/go-tipb" "github.com/stretchr/testify/require" ) diff --git a/util/topsql/topsql.go b/util/topsql/topsql.go index fc3d554681869..0d77547478a19 100644 --- a/util/topsql/topsql.go +++ b/util/topsql/topsql.go @@ -24,7 +24,7 @@ import ( "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/topsql/reporter" - "github.com/pingcap/tidb/util/topsql/tracecpu" + "github.com/pingcap/tidb/util/tracecpu" "github.com/pingcap/tipb/go-tipb" "go.uber.org/zap" "google.golang.org/grpc" diff --git a/util/topsql/topsql_test.go b/util/topsql/topsql_test.go index 888d622e9c09b..305056b56b9e8 100644 --- a/util/topsql/topsql_test.go +++ b/util/topsql/topsql_test.go @@ -26,8 +26,8 @@ import ( "github.com/pingcap/tidb/util/topsql" "github.com/pingcap/tidb/util/topsql/reporter" mockServer "github.com/pingcap/tidb/util/topsql/reporter/mock" - "github.com/pingcap/tidb/util/topsql/tracecpu" "github.com/pingcap/tidb/util/topsql/tracecpu/mock" + "github.com/pingcap/tidb/util/tracecpu" "github.com/pingcap/tipb/go-tipb" "github.com/stretchr/testify/require" "google.golang.org/grpc" diff --git a/util/topsql/tracecpu/main_test.go b/util/tracecpu/main_test.go similarity index 100% rename from util/topsql/tracecpu/main_test.go rename to util/tracecpu/main_test.go diff --git a/util/topsql/tracecpu/mock/mock.go b/util/tracecpu/mock/mock.go similarity index 99% rename from util/topsql/tracecpu/mock/mock.go rename to util/tracecpu/mock/mock.go index 58a933d103aa3..97fc981efddf0 100644 --- a/util/topsql/tracecpu/mock/mock.go +++ b/util/tracecpu/mock/mock.go @@ -23,7 +23,7 @@ import ( "github.com/pingcap/tidb/util/hack" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/topsql/reporter" - "github.com/pingcap/tidb/util/topsql/tracecpu" + "github.com/pingcap/tidb/util/tracecpu" "go.uber.org/atomic" "go.uber.org/zap" ) diff --git a/util/topsql/tracecpu/profile.go b/util/tracecpu/profile.go similarity index 100% rename from util/topsql/tracecpu/profile.go rename to util/tracecpu/profile.go diff --git a/util/topsql/tracecpu/profile_test.go b/util/tracecpu/profile_test.go similarity index 94% rename from util/topsql/tracecpu/profile_test.go rename to util/tracecpu/profile_test.go index 35419288b850d..a92cee52a031e 100644 --- a/util/topsql/tracecpu/profile_test.go +++ b/util/tracecpu/profile_test.go @@ -19,8 +19,8 @@ import ( "testing" "github.com/google/pprof/profile" - "github.com/pingcap/tidb/util/topsql/tracecpu" - "github.com/pingcap/tidb/util/topsql/tracecpu/mock" + "github.com/pingcap/tidb/util/tracecpu" + "github.com/pingcap/tidb/util/tracecpu/mock" "github.com/stretchr/testify/require" ) From 54c34e0dfac1b96172bebdf10bba232541ff0e87 Mon Sep 17 00:00:00 2001 From: Zhenchi Date: Mon, 13 Dec 2021 14:24:32 +0800 Subject: [PATCH 47/47] polish Signed-off-by: Zhenchi --- executor/executor_test.go | 2 +- util/tracecpu/profile.go | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/executor/executor_test.go b/executor/executor_test.go index 5b1dab15ce12d..bd52712673f80 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -83,7 +83,7 @@ import ( "github.com/pingcap/tidb/util/testutil" "github.com/pingcap/tidb/util/timeutil" "github.com/pingcap/tidb/util/topsql" - topsqlmock "github.com/pingcap/tidb/util/topsql/tracecpu/mock" + topsqlmock "github.com/pingcap/tidb/util/tracecpu/mock" "github.com/pingcap/tipb/go-tipb" "github.com/stretchr/testify/require" "github.com/tikv/client-go/v2/oracle" diff --git a/util/tracecpu/profile.go b/util/tracecpu/profile.go index 7b2807d9ada9c..23412a0c7776a 100644 --- a/util/tracecpu/profile.go +++ b/util/tracecpu/profile.go @@ -40,7 +40,7 @@ const ( labelSQLDigest = "sql_digest" labelPlanDigest = "plan_digest" - // DefPrecisionSeconds indicates that the default PrecisionSeconds is 10000 + // DefPrecisionSeconds indicates that the default PrecisionSeconds is 1s DefPrecisionSeconds = 1 // DefEnablePProfSQLCPU indicates that the default EnablePProfSQLCPU is false