From 1167fafdd6e9ff939ed9ff5c226f3068a162114f Mon Sep 17 00:00:00 2001 From: Yexiang Zhang Date: Thu, 9 Feb 2023 18:05:59 +0800 Subject: [PATCH] stmtsummary: add persistence implementation (#40814) ref pingcap/tidb#40812 --- bindinfo/BUILD.bazel | 2 +- bindinfo/handle.go | 4 +- config/config.go | 19 + executor/BUILD.bazel | 4 + executor/adapter.go | 5 +- executor/builder.go | 17 +- executor/infoschema_reader.go | 67 -- executor/slow_query.go | 28 +- executor/stmtsummary.go | 408 +++++++++ executor/stmtsummary_test.go | 182 ++++ planner/core/memtable_predicate_extractor.go | 85 +- sessionctx/variable/BUILD.bazel | 2 +- sessionctx/variable/sysvar.go | 29 +- sessionctx/variable/tidb_vars.go | 10 + tidb-server/BUILD.bazel | 1 + tidb-server/main.go | 17 + util/set/string_set.go | 5 + util/stmtsummary/v2/BUILD.bazel | 60 ++ util/stmtsummary/v2/column.go | 521 +++++++++++ util/stmtsummary/v2/column_test.go | 82 ++ util/stmtsummary/v2/logger.go | 111 +++ util/stmtsummary/v2/main_test.go | 33 + util/stmtsummary/v2/reader.go | 863 +++++++++++++++++++ util/stmtsummary/v2/reader_test.go | 393 +++++++++ util/stmtsummary/v2/record.go | 661 ++++++++++++++ util/stmtsummary/v2/record_test.go | 64 ++ util/stmtsummary/v2/stmtsummary.go | 580 +++++++++++++ util/stmtsummary/v2/stmtsummary_test.go | 86 ++ util/stmtsummary/v2/tests/BUILD.bazel | 22 + util/stmtsummary/v2/tests/main_test.go | 33 + util/stmtsummary/v2/tests/table_test.go | 523 +++++++++++ util/util.go | 50 ++ util/util_test.go | 22 + 33 files changed, 4861 insertions(+), 128 deletions(-) create mode 100644 executor/stmtsummary.go create mode 100644 executor/stmtsummary_test.go create mode 100644 util/stmtsummary/v2/BUILD.bazel create mode 100644 util/stmtsummary/v2/column.go create mode 100644 util/stmtsummary/v2/column_test.go create mode 100644 util/stmtsummary/v2/logger.go create mode 100644 util/stmtsummary/v2/main_test.go create mode 100644 util/stmtsummary/v2/reader.go create mode 100644 util/stmtsummary/v2/reader_test.go create mode 100644 util/stmtsummary/v2/record.go create mode 100644 util/stmtsummary/v2/record_test.go create mode 100644 util/stmtsummary/v2/stmtsummary.go create mode 100644 util/stmtsummary/v2/stmtsummary_test.go create mode 100644 util/stmtsummary/v2/tests/BUILD.bazel create mode 100644 util/stmtsummary/v2/tests/main_test.go create mode 100644 util/stmtsummary/v2/tests/table_test.go diff --git a/bindinfo/BUILD.bazel b/bindinfo/BUILD.bazel index 93b55bfab21c2..64a11f86d5556 100644 --- a/bindinfo/BUILD.bazel +++ b/bindinfo/BUILD.bazel @@ -34,7 +34,7 @@ go_library( "//util/memory", "//util/parser", "//util/sqlexec", - "//util/stmtsummary", + "//util/stmtsummary/v2:stmtsummary", "//util/table-filter", "//util/timeutil", "@org_golang_x_exp//maps", diff --git a/bindinfo/handle.go b/bindinfo/handle.go index 59919e2b5ad85..8501dcfc9939e 100644 --- a/bindinfo/handle.go +++ b/bindinfo/handle.go @@ -41,7 +41,7 @@ import ( "github.com/pingcap/tidb/util/logutil" utilparser "github.com/pingcap/tidb/util/parser" "github.com/pingcap/tidb/util/sqlexec" - "github.com/pingcap/tidb/util/stmtsummary" + stmtsummaryv2 "github.com/pingcap/tidb/util/stmtsummary/v2" tablefilter "github.com/pingcap/tidb/util/table-filter" "github.com/pingcap/tidb/util/timeutil" "go.uber.org/zap" @@ -892,7 +892,7 @@ func (h *BindHandle) CaptureBaselines() { parser4Capture := parser.New() captureFilter := h.extractCaptureFilterFromStorage() emptyCaptureFilter := captureFilter.isEmpty() - bindableStmts := stmtsummary.StmtSummaryByDigestMap.GetMoreThanCntBindableStmt(captureFilter.frequency) + bindableStmts := stmtsummaryv2.GetMoreThanCntBindableStmt(captureFilter.frequency) for _, bindableStmt := range bindableStmts { stmt, err := parser4Capture.ParseOneStmt(bindableStmt.Query, bindableStmt.Charset, bindableStmt.Collation) if err != nil { diff --git a/config/config.go b/config/config.go index a2230f5d73c35..b868e5ac8bdb3 100644 --- a/config/config.go +++ b/config/config.go @@ -502,6 +502,20 @@ type Instance struct { DDLSlowOprThreshold uint32 `toml:"ddl_slow_threshold" json:"ddl_slow_threshold"` // ExpensiveQueryTimeThreshold indicates the time threshold of expensive query. ExpensiveQueryTimeThreshold uint64 `toml:"tidb_expensive_query_time_threshold" json:"tidb_expensive_query_time_threshold"` + // StmtSummaryEnablePersistent indicates whether to enable file persistence for stmtsummary. + StmtSummaryEnablePersistent bool `toml:"tidb_stmt_summary_enable_persistent" json:"tidb_stmt_summary_enable_persistent"` + // StmtSummaryFilename indicates the file name written by stmtsummary + // when StmtSummaryEnablePersistent is true. + StmtSummaryFilename string `toml:"tidb_stmt_summary_filename" json:"tidb_stmt_summary_filename"` + // StmtSummaryFileMaxDays indicates how many days the files written by + // stmtsummary will be kept when StmtSummaryEnablePersistent is true. + StmtSummaryFileMaxDays int `toml:"tidb_stmt_summary_file_max_days" json:"tidb_stmt_summary_file_max_days"` + // StmtSummaryFileMaxSize indicates the maximum size (in mb) of a single file + // written by stmtsummary when StmtSummaryEnablePersistent is true. + StmtSummaryFileMaxSize int `toml:"tidb_stmt_summary_file_max_size" json:"tidb_stmt_summary_file_max_size"` + // StmtSummaryFileMaxBackups indicates the maximum number of files written + // by stmtsummary when StmtSummaryEnablePersistent is true. + StmtSummaryFileMaxBackups int `toml:"tidb_stmt_summary_file_max_backups" json:"tidb_stmt_summary_file_max_backups"` // These variables exist in both 'instance' section and another place. // The configuration in 'instance' section takes precedence. @@ -901,6 +915,11 @@ var defaultConf = Config{ EnablePProfSQLCPU: false, DDLSlowOprThreshold: DefDDLSlowOprThreshold, ExpensiveQueryTimeThreshold: DefExpensiveQueryTimeThreshold, + StmtSummaryEnablePersistent: false, + StmtSummaryFilename: "tidb-statements.log", + StmtSummaryFileMaxDays: 3, + StmtSummaryFileMaxSize: 64, + StmtSummaryFileMaxBackups: 0, EnableSlowLog: *NewAtomicBool(logutil.DefaultTiDBEnableSlowLog), SlowThreshold: logutil.DefaultSlowThreshold, RecordPlanInSlowLog: logutil.DefaultRecordPlanInSlowLog, diff --git a/executor/BUILD.bazel b/executor/BUILD.bazel index 56c6805fdef21..aa288e1180843 100644 --- a/executor/BUILD.bazel +++ b/executor/BUILD.bazel @@ -85,6 +85,7 @@ go_library( "slow_query.go", "sort.go", "split.go", + "stmtsummary.go", "table_reader.go", "trace.go", "union_scan.go", @@ -194,6 +195,7 @@ go_library( "//util/size", "//util/sqlexec", "//util/stmtsummary", + "//util/stmtsummary/v2:stmtsummary", "//util/stringutil", "//util/table-filter", "//util/timeutil", @@ -326,6 +328,7 @@ go_test( "split_test.go", "stale_txn_test.go", "statement_context_test.go", + "stmtsummary_test.go", "table_readers_required_rows_test.go", "temporary_table_test.go", "tikv_regions_peers_table_test.go", @@ -419,6 +422,7 @@ go_test( "//util/rowcodec", "//util/set", "//util/sqlexec", + "//util/stmtsummary/v2:stmtsummary", "//util/stringutil", "//util/tableutil", "//util/timeutil", diff --git a/executor/adapter.go b/executor/adapter.go index f9fba0fea5d25..3758d7d68c430 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -61,6 +61,7 @@ import ( "github.com/pingcap/tidb/util/replayer" "github.com/pingcap/tidb/util/sqlexec" "github.com/pingcap/tidb/util/stmtsummary" + stmtsummaryv2 "github.com/pingcap/tidb/util/stmtsummary/v2" "github.com/pingcap/tidb/util/stringutil" "github.com/pingcap/tidb/util/topsql" topsqlstate "github.com/pingcap/tidb/util/topsql/state" @@ -1804,7 +1805,7 @@ func (a *ExecStmt) SummaryStmt(succ bool) { } // Internal SQLs must also be recorded to keep the consistency of `PrevStmt` and `PrevStmtDigest`. - if !stmtsummary.StmtSummaryByDigestMap.Enabled() || ((sessVars.InRestrictedSQL || len(userString) == 0) && !stmtsummary.StmtSummaryByDigestMap.EnabledInternal()) { + if !stmtsummaryv2.Enabled() || ((sessVars.InRestrictedSQL || len(userString) == 0) && !stmtsummaryv2.EnabledInternal()) { sessVars.SetPrevStmtDigest("") return } @@ -1921,7 +1922,7 @@ func (a *ExecStmt) SummaryStmt(succ bool) { if a.retryCount > 0 { stmtExecInfo.ExecRetryTime = costTime - sessVars.DurationParse - sessVars.DurationCompile - time.Since(a.retryStartTime) } - stmtsummary.StmtSummaryByDigestMap.AddStatement(stmtExecInfo) + stmtsummaryv2.Add(stmtExecInfo) } // GetTextToLog return the query text to log. diff --git a/executor/builder.go b/executor/builder.go index 1005b51a764c6..cb97541660dc2 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -1937,8 +1937,6 @@ func (b *executorBuilder) buildMemTable(v *plannercore.PhysicalMemTable) Executo strings.ToLower(infoschema.TableTiFlashReplica), strings.ToLower(infoschema.TableTiDBServersInfo), strings.ToLower(infoschema.TableTiKVStoreStatus), - strings.ToLower(infoschema.TableStatementsSummaryEvicted), - strings.ToLower(infoschema.ClusterTableStatementsSummaryEvicted), strings.ToLower(infoschema.TableClientErrorsSummaryGlobal), strings.ToLower(infoschema.TableClientErrorsSummaryByUser), strings.ToLower(infoschema.TableClientErrorsSummaryByHost), @@ -1993,16 +1991,18 @@ func (b *executorBuilder) buildMemTable(v *plannercore.PhysicalMemTable) Executo } case strings.ToLower(infoschema.TableStatementsSummary), strings.ToLower(infoschema.TableStatementsSummaryHistory), + strings.ToLower(infoschema.TableStatementsSummaryEvicted), + strings.ToLower(infoschema.ClusterTableStatementsSummary), strings.ToLower(infoschema.ClusterTableStatementsSummaryHistory), - strings.ToLower(infoschema.ClusterTableStatementsSummary): + strings.ToLower(infoschema.ClusterTableStatementsSummaryEvicted): + var extractor *plannercore.StatementsSummaryExtractor + if v.Extractor != nil { + extractor = v.Extractor.(*plannercore.StatementsSummaryExtractor) + } return &MemTableReaderExec{ baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ID()), table: v.Table, - retriever: &stmtSummaryTableRetriever{ - table: v.Table, - columns: v.Columns, - extractor: v.Extractor.(*plannercore.StatementsSummaryExtractor), - }, + retriever: buildStmtSummaryRetriever(b.ctx, v.Table, v.Columns, extractor), } case strings.ToLower(infoschema.TableColumns): return &MemTableReaderExec{ @@ -2016,7 +2016,6 @@ func (b *executorBuilder) buildMemTable(v *plannercore.PhysicalMemTable) Executo viewOutputNamesMap: make(map[int64]types.NameSlice), }, } - case strings.ToLower(infoschema.TableSlowQuery), strings.ToLower(infoschema.ClusterTableSlowLog): memTracker := memory.NewTracker(v.ID(), -1) memTracker.AttachTo(b.ctx.GetSessionVars().StmtCtx.MemTracker) diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index d3e94d5b77353..cf833ab25db7b 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -70,7 +70,6 @@ import ( "github.com/pingcap/tidb/util/servermemorylimit" "github.com/pingcap/tidb/util/set" "github.com/pingcap/tidb/util/sqlexec" - "github.com/pingcap/tidb/util/stmtsummary" "github.com/pingcap/tidb/util/stringutil" "github.com/tikv/client-go/v2/txnkv/txnlock" "go.uber.org/zap" @@ -158,9 +157,6 @@ func (e *memtableRetriever) retrieve(ctx context.Context, sctx sessionctx.Contex e.dataForTableTiFlashReplica(sctx, dbs) case infoschema.TableTiKVStoreStatus: err = e.dataForTiKVStoreStatus(sctx) - case infoschema.TableStatementsSummaryEvicted, - infoschema.ClusterTableStatementsSummaryEvicted: - err = e.setDataForStatementsSummaryEvicted(sctx) case infoschema.TableClientErrorsSummaryGlobal, infoschema.TableClientErrorsSummaryByUser, infoschema.TableClientErrorsSummaryByHost: @@ -2306,22 +2302,6 @@ func (e *memtableRetriever) dataForTableTiFlashReplica(ctx sessionctx.Context, s e.rows = rows } -func (e *memtableRetriever) setDataForStatementsSummaryEvicted(ctx sessionctx.Context) error { - if !hasPriv(ctx, mysql.ProcessPriv) { - return plannercore.ErrSpecificAccessDenied.GenWithStackByArgs("PROCESS") - } - e.rows = stmtsummary.StmtSummaryByDigestMap.ToEvictedCountDatum() - switch e.table.Name.O { - case infoschema.ClusterTableStatementsSummaryEvicted: - rows, err := infoschema.AppendHostInfoToRows(ctx, e.rows) - if err != nil { - return err - } - e.rows = rows - } - return nil -} - func (e *memtableRetriever) setDataForClientErrorsSummary(ctx sessionctx.Context, tableName string) error { // Seeing client errors should require the PROCESS privilege, with the exception of errors for your own user. // This is similar to information_schema.processlist, which is the closest comparison. @@ -2476,50 +2456,6 @@ func (e *memtableRetriever) setDataForClusterMemoryUsageOpsHistory(ctx sessionct return nil } -type stmtSummaryTableRetriever struct { - dummyCloser - table *model.TableInfo - columns []*model.ColumnInfo - retrieved bool - extractor *plannercore.StatementsSummaryExtractor -} - -// retrieve implements the infoschemaRetriever interface -func (e *stmtSummaryTableRetriever) retrieve(ctx context.Context, sctx sessionctx.Context) ([][]types.Datum, error) { - if e.extractor.SkipRequest || e.retrieved { - return nil, nil - } - e.retrieved = true - - var err error - var instanceAddr string - switch e.table.Name.O { - case infoschema.ClusterTableStatementsSummary, - infoschema.ClusterTableStatementsSummaryHistory: - instanceAddr, err = infoschema.GetInstanceAddr(sctx) - if err != nil { - return nil, err - } - } - user := sctx.GetSessionVars().User - reader := stmtsummary.NewStmtSummaryReader(user, hasPriv(sctx, mysql.ProcessPriv), e.columns, instanceAddr, sctx.GetSessionVars().StmtCtx.TimeZone) - if e.extractor.Enable { - checker := stmtsummary.NewStmtSummaryChecker(e.extractor.Digests) - reader.SetChecker(checker) - } - var rows [][]types.Datum - switch e.table.Name.O { - case infoschema.TableStatementsSummary, - infoschema.ClusterTableStatementsSummary: - rows = reader.GetStmtSummaryCurrentRows() - case infoschema.TableStatementsSummaryHistory, - infoschema.ClusterTableStatementsSummaryHistory: - rows = reader.GetStmtSummaryHistoryRows() - } - - return rows, nil -} - // tidbTrxTableRetriever is the memtable retriever for the TIDB_TRX and CLUSTER_TIDB_TRX table. type tidbTrxTableRetriever struct { dummyCloser @@ -3018,9 +2954,6 @@ func (e *hugeMemTableRetriever) retrieve(ctx context.Context, sctx sessionctx.Co } func adjustColumns(input [][]types.Datum, outColumns []*model.ColumnInfo, table *model.TableInfo) [][]types.Datum { - if table.Name.O == infoschema.TableStatementsSummary { - return input - } if len(outColumns) == len(table.Columns) { return input } diff --git a/executor/slow_query.go b/executor/slow_query.go index e8a2731a476e7..f7878839ae24b 100644 --- a/executor/slow_query.go +++ b/executor/slow_query.go @@ -275,33 +275,7 @@ func (sc *slowLogChecker) isTimeValid(t types.Time) bool { } func getOneLine(reader *bufio.Reader) ([]byte, error) { - var resByte []byte - lineByte, isPrefix, err := reader.ReadLine() - if isPrefix { - // Need to read more data. - resByte = make([]byte, len(lineByte), len(lineByte)*2) - } else { - resByte = make([]byte, len(lineByte)) - } - // Use copy here to avoid shallow copy problem. - copy(resByte, lineByte) - if err != nil { - return resByte, err - } - - var tempLine []byte - for isPrefix { - tempLine, isPrefix, err = reader.ReadLine() - resByte = append(resByte, tempLine...) // nozero - // Use the max value of max_allowed_packet to check the single line length. - if len(resByte) > int(variable.MaxOfMaxAllowedPacket) { - return resByte, errors.Errorf("single line length exceeds limit: %v", variable.MaxOfMaxAllowedPacket) - } - if err != nil { - return resByte, err - } - } - return resByte, err + return util.ReadLine(reader, int(variable.MaxOfMaxAllowedPacket)) } type offset struct { diff --git a/executor/stmtsummary.go b/executor/stmtsummary.go new file mode 100644 index 0000000000000..de7cbef6e1aa6 --- /dev/null +++ b/executor/stmtsummary.go @@ -0,0 +1,408 @@ +// Copyright 2023 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 executor + +import ( + "context" + + "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/infoschema" + "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/parser/mysql" + plannercore "github.com/pingcap/tidb/planner/core" + "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/execdetails" + "github.com/pingcap/tidb/util/set" + "github.com/pingcap/tidb/util/stmtsummary" + stmtsummaryv2 "github.com/pingcap/tidb/util/stmtsummary/v2" +) + +const ( + defaultRetrieveCount = 1024 +) + +func buildStmtSummaryRetriever( + ctx sessionctx.Context, + table *model.TableInfo, + columns []*model.ColumnInfo, + extractor *plannercore.StatementsSummaryExtractor, +) memTableRetriever { + if extractor == nil { + extractor = &plannercore.StatementsSummaryExtractor{} + } + if extractor.Digests.Empty() { + extractor.Digests = nil + } + + var retriever memTableRetriever + if extractor.SkipRequest { + retriever = &dummyRetriever{} + } else if config.GetGlobalConfig().Instance.StmtSummaryEnablePersistent { + retriever = &stmtSummaryRetrieverV2{ + stmtSummary: stmtsummaryv2.GlobalStmtSummary, + table: table, + columns: columns, + digests: extractor.Digests, + timeRanges: buildTimeRanges(extractor.CoarseTimeRange), + } + } else { + retriever = &stmtSummaryRetriever{ + table: table, + columns: columns, + digests: extractor.Digests, + } + } + + return retriever +} + +type dummyRetriever struct { + dummyCloser +} + +func (e *dummyRetriever) retrieve(_ context.Context, _ sessionctx.Context) ([][]types.Datum, error) { + return nil, nil +} + +// stmtSummaryRetriever is used to retrieve statements summary. +type stmtSummaryRetriever struct { + table *model.TableInfo + columns []*model.ColumnInfo + digests set.StringSet + + // lazily initialized + rowsReader *rowsReader +} + +func (e *stmtSummaryRetriever) retrieve(ctx context.Context, sctx sessionctx.Context) ([][]types.Datum, error) { + if err := e.ensureRowsReader(sctx); err != nil { + return nil, err + } + return e.rowsReader.read(defaultRetrieveCount) +} + +func (e *stmtSummaryRetriever) close() error { + if e.rowsReader != nil { + return e.rowsReader.close() + } + return nil +} + +func (e *stmtSummaryRetriever) getRuntimeStats() execdetails.RuntimeStats { + return nil +} + +func (e *stmtSummaryRetriever) ensureRowsReader(sctx sessionctx.Context) error { + if e.rowsReader != nil { + return nil + } + + var err error + if isEvictedTable(e.table.Name.O) { + e.rowsReader, err = e.initEvictedRowsReader(sctx) + } else { + e.rowsReader, err = e.initSummaryRowsReader(sctx) + } + + return err +} + +func (e *stmtSummaryRetriever) initEvictedRowsReader(sctx sessionctx.Context) (*rowsReader, error) { + if err := checkPrivilege(sctx); err != nil { + return nil, err + } + + rows := stmtsummary.StmtSummaryByDigestMap.ToEvictedCountDatum() + if !isClusterTable(e.table.Name.O) { + // rows are full-columned, so we need to adjust them to the required columns. + return newSimpleRowsReader(adjustColumns(rows, e.columns, e.table)), nil + } + + // Additional column `INSTANCE` for cluster table + rows, err := infoschema.AppendHostInfoToRows(sctx, rows) + if err != nil { + return nil, err + } + // rows are full-columned, so we need to adjust them to the required columns. + return newSimpleRowsReader(adjustColumns(rows, e.columns, e.table)), nil +} + +func (e *stmtSummaryRetriever) initSummaryRowsReader(sctx sessionctx.Context) (*rowsReader, error) { + vars := sctx.GetSessionVars() + user := vars.User + tz := vars.StmtCtx.TimeZone + columns := e.columns + priv := hasPriv(sctx, mysql.ProcessPriv) + instanceAddr, err := clusterTableInstanceAddr(sctx, e.table.Name.O) + if err != nil { + return nil, err + } + + reader := stmtsummary.NewStmtSummaryReader(user, priv, columns, instanceAddr, tz) + if e.digests != nil { + // set checker to filter out statements not matching the given digests + checker := stmtsummary.NewStmtSummaryChecker(e.digests) + reader.SetChecker(checker) + } + + var rows [][]types.Datum + if isCurrentTable(e.table.Name.O) { + rows = reader.GetStmtSummaryCurrentRows() + } + if isHistoryTable(e.table.Name.O) { + rows = reader.GetStmtSummaryHistoryRows() + } + return newSimpleRowsReader(rows), nil +} + +// stmtSummaryRetriever is used to retrieve statements summary when +// config.GetGlobalConfig().Instance.StmtSummaryEnablePersistent is true +type stmtSummaryRetrieverV2 struct { + stmtSummary *stmtsummaryv2.StmtSummary + table *model.TableInfo + columns []*model.ColumnInfo + digests set.StringSet + timeRanges []*stmtsummaryv2.StmtTimeRange + + // lazily initialized + rowsReader *rowsReader +} + +func (r *stmtSummaryRetrieverV2) retrieve(ctx context.Context, sctx sessionctx.Context) ([][]types.Datum, error) { + if err := r.ensureRowsReader(ctx, sctx); err != nil { + return nil, err + } + return r.rowsReader.read(defaultRetrieveCount) +} + +func (r *stmtSummaryRetrieverV2) close() error { + if r.rowsReader != nil { + return r.rowsReader.close() + } + return nil +} + +func (r *stmtSummaryRetrieverV2) getRuntimeStats() execdetails.RuntimeStats { + return nil +} + +func (r *stmtSummaryRetrieverV2) ensureRowsReader(ctx context.Context, sctx sessionctx.Context) error { + if r.rowsReader != nil { + return nil + } + + var err error + if isEvictedTable(r.table.Name.O) { + r.rowsReader, err = r.initEvictedRowsReader(sctx) + } else { + r.rowsReader, err = r.initSummaryRowsReader(ctx, sctx) + } + + return err +} + +func (r *stmtSummaryRetrieverV2) initEvictedRowsReader(sctx sessionctx.Context) (*rowsReader, error) { + if err := checkPrivilege(sctx); err != nil { + return nil, err + } + + var rows [][]types.Datum + + row := r.stmtSummary.Evicted() + if row != nil { + rows = append(rows, row) + } + if !isClusterTable(r.table.Name.O) { + // rows are full-columned, so we need to adjust them to the required columns. + return newSimpleRowsReader(adjustColumns(rows, r.columns, r.table)), nil + } + + // Additional column `INSTANCE` for cluster table + rows, err := infoschema.AppendHostInfoToRows(sctx, rows) + if err != nil { + return nil, err + } + // rows are full-columned, so we need to adjust them to the required columns. + return newSimpleRowsReader(adjustColumns(rows, r.columns, r.table)), nil +} + +func (r *stmtSummaryRetrieverV2) initSummaryRowsReader(ctx context.Context, sctx sessionctx.Context) (*rowsReader, error) { + vars := sctx.GetSessionVars() + user := vars.User + tz := vars.StmtCtx.TimeZone + stmtSummary := r.stmtSummary + columns := r.columns + timeRanges := r.timeRanges + digests := r.digests + priv := hasPriv(sctx, mysql.ProcessPriv) + instanceAddr, err := clusterTableInstanceAddr(sctx, r.table.Name.O) + if err != nil { + return nil, err + } + + mem := stmtsummaryv2.NewMemReader(stmtSummary, columns, instanceAddr, tz, user, priv, digests, timeRanges) + memRows := mem.Rows() + + var rowsReader *rowsReader + if isCurrentTable(r.table.Name.O) { + rowsReader = newSimpleRowsReader(memRows) + } + if isHistoryTable(r.table.Name.O) { + // history table should return all rows including mem and disk + concurrent := sctx.GetSessionVars().Concurrency.DistSQLScanConcurrency() + history, err := stmtsummaryv2.NewHistoryReader(ctx, columns, instanceAddr, tz, user, priv, digests, timeRanges, concurrent) + if err != nil { + return nil, err + } + rowsReader = newRowsReader(memRows, history) + } + + return rowsReader, nil +} + +type rowsPuller interface { + Closeable + Rows() ([][]types.Datum, error) +} + +type rowsReader struct { + puller rowsPuller + rows [][]types.Datum +} + +func newSimpleRowsReader(rows [][]types.Datum) *rowsReader { + return &rowsReader{rows: rows} +} + +func newRowsReader(rows [][]types.Datum, puller rowsPuller) *rowsReader { + return &rowsReader{puller: puller, rows: rows} +} + +func (r *rowsReader) read(maxCount int) ([][]types.Datum, error) { + if err := r.pull(); err != nil { + return nil, err + } + + if maxCount >= len(r.rows) { + ret := r.rows + r.rows = nil + return ret, nil + } + ret := r.rows[:maxCount] + r.rows = r.rows[maxCount:] + return ret, nil +} + +func (r *rowsReader) pull() error { + if r.puller == nil { + return nil + } + // there are remaining rows + if len(r.rows) > 0 { + return nil + } + + rows, err := r.puller.Rows() + if err != nil { + return err + } + // pulled new rows from the puller + if len(rows) != 0 { + r.rows = rows + return nil + } + + // reach the end of the puller + err = r.puller.Close() + if err != nil { + return err + } + r.puller = nil + return nil +} + +func (r *rowsReader) close() error { + if r.puller != nil { + return r.puller.Close() + } + return nil +} + +func isClusterTable(originalTableName string) bool { + switch originalTableName { + case infoschema.ClusterTableStatementsSummary, + infoschema.ClusterTableStatementsSummaryHistory, + infoschema.ClusterTableStatementsSummaryEvicted: + return true + } + + return false +} + +func isCurrentTable(originalTableName string) bool { + switch originalTableName { + case infoschema.TableStatementsSummary, + infoschema.ClusterTableStatementsSummary: + return true + } + + return false +} + +func isHistoryTable(originalTableName string) bool { + switch originalTableName { + case infoschema.TableStatementsSummaryHistory, + infoschema.ClusterTableStatementsSummaryHistory: + return true + } + + return false +} + +func isEvictedTable(originalTableName string) bool { + switch originalTableName { + case infoschema.TableStatementsSummaryEvicted, + infoschema.ClusterTableStatementsSummaryEvicted: + return true + } + + return false +} + +func checkPrivilege(sctx sessionctx.Context) error { + if !hasPriv(sctx, mysql.ProcessPriv) { + return plannercore.ErrSpecificAccessDenied.GenWithStackByArgs("PROCESS") + } + return nil +} + +func clusterTableInstanceAddr(sctx sessionctx.Context, originalTableName string) (string, error) { + if isClusterTable(originalTableName) { + return infoschema.GetInstanceAddr(sctx) + } + return "", nil +} + +func buildTimeRanges(tr *plannercore.TimeRange) []*stmtsummaryv2.StmtTimeRange { + if tr == nil { + return nil + } + + return []*stmtsummaryv2.StmtTimeRange{{ + Begin: tr.StartTime.Unix(), + End: tr.EndTime.Unix(), + }} +} diff --git a/executor/stmtsummary_test.go b/executor/stmtsummary_test.go new file mode 100644 index 0000000000000..a09966c5f3d59 --- /dev/null +++ b/executor/stmtsummary_test.go @@ -0,0 +1,182 @@ +// Copyright 2022 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 executor + +import ( + "context" + "os" + "testing" + "time" + + "github.com/pingcap/tidb/infoschema" + "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util" + "github.com/pingcap/tidb/util/mock" + stmtsummaryv2 "github.com/pingcap/tidb/util/stmtsummary/v2" + "github.com/stretchr/testify/require" +) + +func TestStmtSummaryRetriverV2_TableStatementsSummary(t *testing.T) { + infoSchemaBuilder, err := infoschema.NewBuilder(nil, nil).InitWithDBInfos(nil, nil, nil, 0) + require.NoError(t, err) + infoSchema := infoSchemaBuilder.Build() + table, err := infoSchema.TableByName(util.InformationSchemaName, model.NewCIStr(infoschema.TableStatementsSummary)) + require.NoError(t, err) + columns := table.Meta().Columns + + stmtSummary := stmtsummaryv2.NewStmtSummary4Test(1000) + defer stmtSummary.Close() + stmtSummary.Add(stmtsummaryv2.GenerateStmtExecInfo4Test("digest1")) + stmtSummary.Add(stmtsummaryv2.GenerateStmtExecInfo4Test("digest1")) + stmtSummary.Add(stmtsummaryv2.GenerateStmtExecInfo4Test("digest2")) + stmtSummary.Add(stmtsummaryv2.GenerateStmtExecInfo4Test("digest2")) + stmtSummary.Add(stmtsummaryv2.GenerateStmtExecInfo4Test("digest3")) + stmtSummary.Add(stmtsummaryv2.GenerateStmtExecInfo4Test("digest3")) + + retriever := stmtSummaryRetrieverV2{ + stmtSummary: stmtSummary, + table: table.Meta(), + columns: columns, + } + defer func() { + require.NoError(t, retriever.close()) + }() + + ctx := context.Background() + sctx := mock.NewContext() + sctx.GetSessionVars().TimeZone, _ = time.LoadLocation("Asia/Shanghai") + + var results [][]types.Datum + for { + rows, err := retriever.retrieve(ctx, sctx) + require.NoError(t, err) + if len(rows) == 0 { + break + } + results = append(results, rows...) + } + require.Len(t, results, 3) +} + +func TestStmtSummaryRetriverV2_TableStatementsSummaryEvicted(t *testing.T) { + infoSchemaBuilder, err := infoschema.NewBuilder(nil, nil).InitWithDBInfos(nil, nil, nil, 0) + require.NoError(t, err) + infoSchema := infoSchemaBuilder.Build() + table, err := infoSchema.TableByName(util.InformationSchemaName, model.NewCIStr(infoschema.TableStatementsSummaryEvicted)) + require.NoError(t, err) + columns := table.Meta().Columns + + stmtSummary := stmtsummaryv2.NewStmtSummary4Test(1) + defer stmtSummary.Close() + stmtSummary.Add(stmtsummaryv2.GenerateStmtExecInfo4Test("digest1")) + stmtSummary.Add(stmtsummaryv2.GenerateStmtExecInfo4Test("digest1")) + stmtSummary.Add(stmtsummaryv2.GenerateStmtExecInfo4Test("digest2")) + stmtSummary.Add(stmtsummaryv2.GenerateStmtExecInfo4Test("digest2")) + stmtSummary.Add(stmtsummaryv2.GenerateStmtExecInfo4Test("digest3")) + stmtSummary.Add(stmtsummaryv2.GenerateStmtExecInfo4Test("digest3")) + + retriever := stmtSummaryRetrieverV2{ + stmtSummary: stmtSummary, + table: table.Meta(), + columns: columns, + } + defer func() { + require.NoError(t, retriever.close()) + }() + + ctx := context.Background() + sctx := mock.NewContext() + sctx.GetSessionVars().TimeZone, _ = time.LoadLocation("Asia/Shanghai") + + var results [][]types.Datum + for { + rows, err := retriever.retrieve(ctx, sctx) + require.NoError(t, err) + if len(rows) == 0 { + break + } + results = append(results, rows...) + } + require.Len(t, results, 1) + require.Equal(t, int64(2), results[0][2].GetInt64()) +} + +func TestStmtSummaryRetriverV2_TableStatementsSummaryHistory(t *testing.T) { + filename1 := "tidb-statements-2022-12-27T16-21-20.245.log" + filename2 := "tidb-statements.log" + + file, err := os.Create(filename1) + require.NoError(t, err) + defer func() { + require.NoError(t, os.Remove(filename1)) + }() + _, err = file.WriteString("{\"begin\":1672128520,\"end\":1672128530,\"digest\":\"digest1\",\"exec_count\":1}\n") + require.NoError(t, err) + _, err = file.WriteString("{\"begin\":1672129270,\"end\":1672129280,\"digest\":\"digest2\",\"exec_count\":2}\n") + require.NoError(t, err) + require.NoError(t, file.Close()) + + file, err = os.Create(filename2) + require.NoError(t, err) + defer func() { + require.NoError(t, os.Remove(filename2)) + }() + _, err = file.WriteString("{\"begin\":1672129270,\"end\":1672129280,\"digest\":\"digest3\",\"exec_count\":3}\n") + require.NoError(t, err) + _, err = file.WriteString("{\"begin\":1672129380,\"end\":1672129390,\"digest\":\"digest4\",\"exec_count\":4}\n") + require.NoError(t, err) + require.NoError(t, file.Close()) + + stmtSummary := stmtsummaryv2.NewStmtSummary4Test(2) + defer stmtSummary.Close() + stmtSummary.Add(stmtsummaryv2.GenerateStmtExecInfo4Test("digest1")) + stmtSummary.Add(stmtsummaryv2.GenerateStmtExecInfo4Test("digest1")) + stmtSummary.Add(stmtsummaryv2.GenerateStmtExecInfo4Test("digest2")) + stmtSummary.Add(stmtsummaryv2.GenerateStmtExecInfo4Test("digest2")) + stmtSummary.Add(stmtsummaryv2.GenerateStmtExecInfo4Test("digest3")) + stmtSummary.Add(stmtsummaryv2.GenerateStmtExecInfo4Test("digest3")) + + infoSchemaBuilder, err := infoschema.NewBuilder(nil, nil).InitWithDBInfos(nil, nil, nil, 0) + require.NoError(t, err) + infoSchema := infoSchemaBuilder.Build() + table, err := infoSchema.TableByName(util.InformationSchemaName, model.NewCIStr(infoschema.TableStatementsSummaryHistory)) + require.NoError(t, err) + columns := table.Meta().Columns + + retriever := stmtSummaryRetrieverV2{ + stmtSummary: stmtSummary, + table: table.Meta(), + columns: columns, + } + defer func() { + require.NoError(t, retriever.close()) + }() + + ctx := context.Background() + sctx := mock.NewContext() + sctx.GetSessionVars().TimeZone, _ = time.LoadLocation("Asia/Shanghai") + + var results [][]types.Datum + for { + rows, err := retriever.retrieve(ctx, sctx) + require.NoError(t, err) + if len(rows) == 0 { + break + } + results = append(results, rows...) + } + require.Len(t, results, 7) +} diff --git a/planner/core/memtable_predicate_extractor.go b/planner/core/memtable_predicate_extractor.go index 07ad325f92163..dc9feed44c2a8 100644 --- a/planner/core/memtable_predicate_extractor.go +++ b/planner/core/memtable_predicate_extractor.go @@ -1402,39 +1402,100 @@ type StatementsSummaryExtractor struct { // Digests represents digest applied to, and we should apply all digest if there is no digest specified. // e.g: SELECT * FROM STATEMENTS_SUMMARY WHERE digest='8019af26debae8aa7642c501dbc43212417b3fb14e6aec779f709976b7e521be' Digests set.StringSet - // Enable is true means the executor should use digest to locate statement summary. - // Enable is false, means the executor should keep the behavior compatible with before. - Enable bool + + // Coarse time range predicate extracted from the where clause as: + // SELECT ... WHERE summary_begin_time <= endTime AND summary_end_time >= startTime + // + // N.B. it's only used by v2, so we should keep predicates not changed when extracting time range, or it will + // affect the correctness with v1. + CoarseTimeRange *TimeRange } // Extract implements the MemTablePredicateExtractor Extract interface func (e *StatementsSummaryExtractor) Extract( - _ sessionctx.Context, + sctx sessionctx.Context, schema *expression.Schema, names []*types.FieldName, predicates []expression.Expression, ) (remained []expression.Expression) { // Extract the `digest` column remained, skip, digests := e.extractCol(schema, names, predicates, "digest", false) - e.SkipRequest = skip - if e.SkipRequest { + if skip { + e.SkipRequest = true return nil } - if digests.Count() > 0 { - e.Enable = true + if !digests.Empty() { e.Digests = digests } + + tr := e.findCoarseTimeRange(sctx, schema, names, remained) + if tr == nil { + return remained + } + + if tr.StartTime.After(tr.EndTime) { + e.SkipRequest = true + return nil + } + e.CoarseTimeRange = tr return remained } -func (e *StatementsSummaryExtractor) explainInfo(_ *PhysicalMemTable) string { +func (e *StatementsSummaryExtractor) explainInfo(p *PhysicalMemTable) string { if e.SkipRequest { return "skip_request: true" } - if !e.Enable { - return "" + buf := bytes.NewBuffer(nil) + if !e.Digests.Empty() { + buf.WriteString(fmt.Sprintf("digests: [%s], ", extractStringFromStringSet(e.Digests))) + } + if e.CoarseTimeRange != nil && p.ctx.GetSessionVars() != nil && p.ctx.GetSessionVars().StmtCtx != nil { + stmtCtx := p.ctx.GetSessionVars().StmtCtx + startTime := e.CoarseTimeRange.StartTime.In(stmtCtx.TimeZone) + endTime := e.CoarseTimeRange.EndTime.In(stmtCtx.TimeZone) + startTimeStr := types.NewTime(types.FromGoTime(startTime), mysql.TypeDatetime, types.MaxFsp).String() + endTimeStr := types.NewTime(types.FromGoTime(endTime), mysql.TypeDatetime, types.MaxFsp).String() + buf.WriteString(fmt.Sprintf("start_time:%v, end_time:%v, ", startTimeStr, endTimeStr)) + } + // remove the last ", " in the message info + s := buf.String() + if len(s) > 2 { + return s[:len(s)-2] + } + return s +} + +func (e *StatementsSummaryExtractor) findCoarseTimeRange( + sctx sessionctx.Context, + schema *expression.Schema, + names []*types.FieldName, + predicates []expression.Expression, +) *TimeRange { + tz := sctx.GetSessionVars().StmtCtx.TimeZone + _, _, endTime := e.extractTimeRange(sctx, schema, names, predicates, "summary_begin_time", tz) + _, startTime, _ := e.extractTimeRange(sctx, schema, names, predicates, "summary_end_time", tz) + return e.buildTimeRange(startTime, endTime) +} + +func (e *StatementsSummaryExtractor) buildTimeRange(start, end int64) *TimeRange { + const defaultStatementsDuration = time.Hour + var startTime, endTime time.Time + if start == 0 && end == 0 { + return nil + } + if start != 0 { + startTime = e.convertToTime(start) + } + if end != 0 { + endTime = e.convertToTime(end) + } + if start == 0 { + startTime = endTime.Add(-defaultStatementsDuration) + } + if end == 0 { + endTime = startTime.Add(defaultStatementsDuration) } - return fmt.Sprintf("digests: [%s]", extractStringFromStringSet(e.Digests)) + return &TimeRange{StartTime: startTime, EndTime: endTime} } // TikvRegionPeersExtractor is used to extract some predicates of cluster table. diff --git a/sessionctx/variable/BUILD.bazel b/sessionctx/variable/BUILD.bazel index 60fec443c6a0a..fa0de88eece14 100644 --- a/sessionctx/variable/BUILD.bazel +++ b/sessionctx/variable/BUILD.bazel @@ -51,7 +51,7 @@ go_library( "//util/replayer", "//util/rowcodec", "//util/size", - "//util/stmtsummary", + "//util/stmtsummary/v2:stmtsummary", "//util/stringutil", "//util/tableutil", "//util/tikvutil", diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 068152b3601aa..09fa1112af053 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -40,7 +40,7 @@ import ( "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/mathutil" "github.com/pingcap/tidb/util/memory" - "github.com/pingcap/tidb/util/stmtsummary" + stmtsummaryv2 "github.com/pingcap/tidb/util/stmtsummary/v2" "github.com/pingcap/tidb/util/tikvutil" "github.com/pingcap/tidb/util/tls" topsqlstate "github.com/pingcap/tidb/util/topsql/state" @@ -478,6 +478,21 @@ var defaultSysVars = []*SysVar{ }, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { return BoolToOnOff(EnableRCReadCheckTS.Load()), nil }}, + {Scope: ScopeInstance, Name: TiDBStmtSummaryEnablePersistent, ReadOnly: true, GetGlobal: func(_ context.Context, _ *SessionVars) (string, error) { + return BoolToOnOff(config.GetGlobalConfig().Instance.StmtSummaryEnablePersistent), nil + }}, + {Scope: ScopeInstance, Name: TiDBStmtSummaryFilename, ReadOnly: true, GetGlobal: func(_ context.Context, _ *SessionVars) (string, error) { + return config.GetGlobalConfig().Instance.StmtSummaryFilename, nil + }}, + {Scope: ScopeInstance, Name: TiDBStmtSummaryFileMaxDays, ReadOnly: true, GetGlobal: func(_ context.Context, _ *SessionVars) (string, error) { + return strconv.Itoa(config.GetGlobalConfig().Instance.StmtSummaryFileMaxDays), nil + }}, + {Scope: ScopeInstance, Name: TiDBStmtSummaryFileMaxSize, ReadOnly: true, GetGlobal: func(_ context.Context, _ *SessionVars) (string, error) { + return strconv.Itoa(config.GetGlobalConfig().Instance.StmtSummaryFileMaxSize), nil + }}, + {Scope: ScopeInstance, Name: TiDBStmtSummaryFileMaxBackups, ReadOnly: true, GetGlobal: func(_ context.Context, _ *SessionVars) (string, error) { + return strconv.Itoa(config.GetGlobalConfig().Instance.StmtSummaryFileMaxBackups), nil + }}, /* The system variables below have GLOBAL scope */ {Scope: ScopeGlobal, Name: MaxPreparedStmtCount, Value: strconv.FormatInt(DefMaxPreparedStmtCount, 10), Type: TypeInt, MinValue: -1, MaxValue: 1048576, @@ -653,28 +668,28 @@ var defaultSysVars = []*SysVar{ {Scope: ScopeGlobal, Name: TiDBScatterRegion, Value: BoolToOnOff(DefTiDBScatterRegion), Type: TypeBool}, {Scope: ScopeGlobal, Name: TiDBEnableStmtSummary, Value: BoolToOnOff(DefTiDBEnableStmtSummary), Type: TypeBool, AllowEmpty: true, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { - return stmtsummary.StmtSummaryByDigestMap.SetEnabled(TiDBOptOn(val)) + return stmtsummaryv2.SetEnabled(TiDBOptOn(val)) }}, {Scope: ScopeGlobal, Name: TiDBStmtSummaryInternalQuery, Value: BoolToOnOff(DefTiDBStmtSummaryInternalQuery), Type: TypeBool, AllowEmpty: true, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { - return stmtsummary.StmtSummaryByDigestMap.SetEnabledInternalQuery(TiDBOptOn(val)) + return stmtsummaryv2.SetEnableInternalQuery(TiDBOptOn(val)) }}, {Scope: ScopeGlobal, Name: TiDBStmtSummaryRefreshInterval, Value: strconv.Itoa(DefTiDBStmtSummaryRefreshInterval), Type: TypeInt, MinValue: 1, MaxValue: math.MaxInt32, AllowEmpty: true, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { // convert val to int64 - return stmtsummary.StmtSummaryByDigestMap.SetRefreshInterval(TidbOptInt64(val, DefTiDBStmtSummaryRefreshInterval)) + return stmtsummaryv2.SetRefreshInterval(TidbOptInt64(val, DefTiDBStmtSummaryRefreshInterval)) }}, {Scope: ScopeGlobal, Name: TiDBStmtSummaryHistorySize, Value: strconv.Itoa(DefTiDBStmtSummaryHistorySize), Type: TypeInt, MinValue: 0, MaxValue: math.MaxUint8, AllowEmpty: true, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { - return stmtsummary.StmtSummaryByDigestMap.SetHistorySize(TidbOptInt(val, DefTiDBStmtSummaryHistorySize)) + return stmtsummaryv2.SetHistorySize(TidbOptInt(val, DefTiDBStmtSummaryHistorySize)) }}, {Scope: ScopeGlobal, Name: TiDBStmtSummaryMaxStmtCount, Value: strconv.Itoa(DefTiDBStmtSummaryMaxStmtCount), Type: TypeInt, MinValue: 1, MaxValue: math.MaxInt16, AllowEmpty: true, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { - return stmtsummary.StmtSummaryByDigestMap.SetMaxStmtCount(uint(TidbOptInt(val, DefTiDBStmtSummaryMaxStmtCount))) + return stmtsummaryv2.SetMaxStmtCount(TidbOptInt(val, DefTiDBStmtSummaryMaxStmtCount)) }}, {Scope: ScopeGlobal, Name: TiDBStmtSummaryMaxSQLLength, Value: strconv.Itoa(DefTiDBStmtSummaryMaxSQLLength), Type: TypeInt, MinValue: 0, MaxValue: math.MaxInt32, AllowEmpty: true, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { - return stmtsummary.StmtSummaryByDigestMap.SetMaxSQLLength(TidbOptInt(val, DefTiDBStmtSummaryMaxSQLLength)) + return stmtsummaryv2.SetMaxSQLLength(TidbOptInt(val, DefTiDBStmtSummaryMaxSQLLength)) }}, {Scope: ScopeGlobal, Name: TiDBCapturePlanBaseline, Value: DefTiDBCapturePlanBaseline, Type: TypeBool, AllowEmptyAll: true}, {Scope: ScopeGlobal, Name: TiDBEvolvePlanTaskMaxTime, Value: strconv.Itoa(DefTiDBEvolvePlanTaskMaxTime), Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64}, diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 3090ab630f3b7..3163487de511a 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -916,6 +916,16 @@ const ( TiDBEnableHistoricalStatsForCapture = "tidb_enable_historical_stats_for_capture" // TiDBEnableResourceControl indicates whether resource control feature is enabled TiDBEnableResourceControl = "tidb_enable_resource_control" + // TiDBStmtSummaryEnablePersistent indicates whether to enable file persistence for stmtsummary. + TiDBStmtSummaryEnablePersistent = "tidb_stmt_summary_enable_persistent" + // TiDBStmtSummaryFilename indicates the file name written by stmtsummary. + TiDBStmtSummaryFilename = "tidb_stmt_summary_filename" + // TiDBStmtSummaryFileMaxDays indicates how many days the files written by stmtsummary will be kept. + TiDBStmtSummaryFileMaxDays = "tidb_stmt_summary_file_max_days" + // TiDBStmtSummaryFileMaxSize indicates the maximum size (in mb) of a single file written by stmtsummary. + TiDBStmtSummaryFileMaxSize = "tidb_stmt_summary_file_max_size" + // TiDBStmtSummaryFileMaxBackups indicates the maximum number of files written by stmtsummary. + TiDBStmtSummaryFileMaxBackups = "tidb_stmt_summary_file_max_backups" ) // TiDB intentional limits diff --git a/tidb-server/BUILD.bazel b/tidb-server/BUILD.bazel index b0be440b7f0d8..139fe8ae0ff64 100644 --- a/tidb-server/BUILD.bazel +++ b/tidb-server/BUILD.bazel @@ -46,6 +46,7 @@ go_library( "//util/printer", "//util/sem", "//util/signal", + "//util/stmtsummary/v2:stmtsummary", "//util/sys/linux", "//util/sys/storage", "//util/systimemon", diff --git a/tidb-server/main.go b/tidb-server/main.go index 9c92c2d2a558c..45ceae6b759d3 100644 --- a/tidb-server/main.go +++ b/tidb-server/main.go @@ -71,6 +71,7 @@ import ( "github.com/pingcap/tidb/util/printer" "github.com/pingcap/tidb/util/sem" "github.com/pingcap/tidb/util/signal" + stmtsummaryv2 "github.com/pingcap/tidb/util/stmtsummary/v2" "github.com/pingcap/tidb/util/sys/linux" storageSys "github.com/pingcap/tidb/util/sys/storage" "github.com/pingcap/tidb/util/systimemon" @@ -200,6 +201,7 @@ func main() { } setupLog() setupExtensions() + setupStmtSummary() err := cpuprofile.StartCPUProfiler() terror.MustNil(err) @@ -881,3 +883,18 @@ func stringToList(repairString string) []string { return r == ',' || r == ' ' || r == '"' }) } + +func setupStmtSummary() { + instanceCfg := config.GetGlobalConfig().Instance + if instanceCfg.StmtSummaryEnablePersistent { + err := stmtsummaryv2.Setup(&stmtsummaryv2.Config{ + Filename: instanceCfg.StmtSummaryFilename, + FileMaxSize: instanceCfg.StmtSummaryFileMaxSize, + FileMaxDays: instanceCfg.StmtSummaryFileMaxDays, + FileMaxBackups: instanceCfg.StmtSummaryFileMaxBackups, + }) + if err != nil { + logutil.BgLogger().Error("failed to setup statements summary", zap.Error(err)) + } + } +} diff --git a/util/set/string_set.go b/util/set/string_set.go index a008612b6ee12..9c2d6946f2683 100644 --- a/util/set/string_set.go +++ b/util/set/string_set.go @@ -52,3 +52,8 @@ func (s StringSet) Intersection(rhs StringSet) StringSet { func (s StringSet) Count() int { return len(s) } + +// Empty returns whether s is empty. +func (s StringSet) Empty() bool { + return len(s) == 0 +} diff --git a/util/stmtsummary/v2/BUILD.bazel b/util/stmtsummary/v2/BUILD.bazel new file mode 100644 index 0000000000000..53f3fb0f0cd98 --- /dev/null +++ b/util/stmtsummary/v2/BUILD.bazel @@ -0,0 +1,60 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") + +go_library( + name = "stmtsummary", + srcs = [ + "column.go", + "logger.go", + "reader.go", + "record.go", + "stmtsummary.go", + ], + importpath = "github.com/pingcap/tidb/util/stmtsummary/v2", + visibility = ["//visibility:public"], + deps = [ + "//config", + "//parser/auth", + "//parser/model", + "//parser/mysql", + "//sessionctx/stmtctx", + "//types", + "//util", + "//util/execdetails", + "//util/hack", + "//util/kvcache", + "//util/logutil", + "//util/plancodec", + "//util/set", + "//util/stmtsummary", + "@com_github_pingcap_log//:log", + "@com_github_tikv_client_go_v2//util", + "@org_golang_x_exp//maps", + "@org_golang_x_exp//slices", + "@org_uber_go_atomic//:atomic", + "@org_uber_go_zap//:zap", + "@org_uber_go_zap//buffer", + "@org_uber_go_zap//zapcore", + ], +) + +go_test( + name = "stmtsummary_test", + srcs = [ + "column_test.go", + "main_test.go", + "reader_test.go", + "record_test.go", + "stmtsummary_test.go", + ], + embed = [":stmtsummary"], + deps = [ + "//parser/auth", + "//parser/model", + "//testkit/testsetup", + "//types", + "//util", + "//util/set", + "@com_github_stretchr_testify//require", + "@org_uber_go_goleak//:goleak", + ], +) diff --git a/util/stmtsummary/v2/column.go b/util/stmtsummary/v2/column.go new file mode 100644 index 0000000000000..471ef29f3659f --- /dev/null +++ b/util/stmtsummary/v2/column.go @@ -0,0 +1,521 @@ +// Copyright 2023 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 stmtsummary + +import ( + "bytes" + "fmt" + "strings" + "time" + + "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/logutil" + "github.com/pingcap/tidb/util/plancodec" + "go.uber.org/zap" + "golang.org/x/exp/slices" +) + +// Statements summary table column name. +const ( + ClusterTableInstanceColumnNameStr = "INSTANCE" + SummaryBeginTimeStr = "SUMMARY_BEGIN_TIME" + SummaryEndTimeStr = "SUMMARY_END_TIME" + StmtTypeStr = "STMT_TYPE" + SchemaNameStr = "SCHEMA_NAME" + DigestStr = "DIGEST" + DigestTextStr = "DIGEST_TEXT" + TableNamesStr = "TABLE_NAMES" + IndexNamesStr = "INDEX_NAMES" + SampleUserStr = "SAMPLE_USER" + ExecCountStr = "EXEC_COUNT" + SumErrorsStr = "SUM_ERRORS" + SumWarningsStr = "SUM_WARNINGS" + SumLatencyStr = "SUM_LATENCY" + MaxLatencyStr = "MAX_LATENCY" + MinLatencyStr = "MIN_LATENCY" + AvgLatencyStr = "AVG_LATENCY" + AvgParseLatencyStr = "AVG_PARSE_LATENCY" + MaxParseLatencyStr = "MAX_PARSE_LATENCY" + AvgCompileLatencyStr = "AVG_COMPILE_LATENCY" + MaxCompileLatencyStr = "MAX_COMPILE_LATENCY" + SumCopTaskNumStr = "SUM_COP_TASK_NUM" + MaxCopProcessTimeStr = "MAX_COP_PROCESS_TIME" + MaxCopProcessAddressStr = "MAX_COP_PROCESS_ADDRESS" + MaxCopWaitTimeStr = "MAX_COP_WAIT_TIME" // #nosec G101 + MaxCopWaitAddressStr = "MAX_COP_WAIT_ADDRESS" // #nosec G101 + AvgProcessTimeStr = "AVG_PROCESS_TIME" + MaxProcessTimeStr = "MAX_PROCESS_TIME" + AvgWaitTimeStr = "AVG_WAIT_TIME" + MaxWaitTimeStr = "MAX_WAIT_TIME" + AvgBackoffTimeStr = "AVG_BACKOFF_TIME" + MaxBackoffTimeStr = "MAX_BACKOFF_TIME" + AvgTotalKeysStr = "AVG_TOTAL_KEYS" + MaxTotalKeysStr = "MAX_TOTAL_KEYS" + AvgProcessedKeysStr = "AVG_PROCESSED_KEYS" + MaxProcessedKeysStr = "MAX_PROCESSED_KEYS" + AvgRocksdbDeleteSkippedCountStr = "AVG_ROCKSDB_DELETE_SKIPPED_COUNT" + MaxRocksdbDeleteSkippedCountStr = "MAX_ROCKSDB_DELETE_SKIPPED_COUNT" + AvgRocksdbKeySkippedCountStr = "AVG_ROCKSDB_KEY_SKIPPED_COUNT" + MaxRocksdbKeySkippedCountStr = "MAX_ROCKSDB_KEY_SKIPPED_COUNT" + AvgRocksdbBlockCacheHitCountStr = "AVG_ROCKSDB_BLOCK_CACHE_HIT_COUNT" + MaxRocksdbBlockCacheHitCountStr = "MAX_ROCKSDB_BLOCK_CACHE_HIT_COUNT" + AvgRocksdbBlockReadCountStr = "AVG_ROCKSDB_BLOCK_READ_COUNT" + MaxRocksdbBlockReadCountStr = "MAX_ROCKSDB_BLOCK_READ_COUNT" + AvgRocksdbBlockReadByteStr = "AVG_ROCKSDB_BLOCK_READ_BYTE" + MaxRocksdbBlockReadByteStr = "MAX_ROCKSDB_BLOCK_READ_BYTE" + AvgPrewriteTimeStr = "AVG_PREWRITE_TIME" + MaxPrewriteTimeStr = "MAX_PREWRITE_TIME" + AvgCommitTimeStr = "AVG_COMMIT_TIME" + MaxCommitTimeStr = "MAX_COMMIT_TIME" + AvgGetCommitTsTimeStr = "AVG_GET_COMMIT_TS_TIME" + MaxGetCommitTsTimeStr = "MAX_GET_COMMIT_TS_TIME" + AvgCommitBackoffTimeStr = "AVG_COMMIT_BACKOFF_TIME" + MaxCommitBackoffTimeStr = "MAX_COMMIT_BACKOFF_TIME" + AvgResolveLockTimeStr = "AVG_RESOLVE_LOCK_TIME" + MaxResolveLockTimeStr = "MAX_RESOLVE_LOCK_TIME" + AvgLocalLatchWaitTimeStr = "AVG_LOCAL_LATCH_WAIT_TIME" + MaxLocalLatchWaitTimeStr = "MAX_LOCAL_LATCH_WAIT_TIME" + AvgWriteKeysStr = "AVG_WRITE_KEYS" + MaxWriteKeysStr = "MAX_WRITE_KEYS" + AvgWriteSizeStr = "AVG_WRITE_SIZE" + MaxWriteSizeStr = "MAX_WRITE_SIZE" + AvgPrewriteRegionsStr = "AVG_PREWRITE_REGIONS" + MaxPrewriteRegionsStr = "MAX_PREWRITE_REGIONS" + AvgTxnRetryStr = "AVG_TXN_RETRY" + MaxTxnRetryStr = "MAX_TXN_RETRY" + SumExecRetryStr = "SUM_EXEC_RETRY" + SumExecRetryTimeStr = "SUM_EXEC_RETRY_TIME" + SumBackoffTimesStr = "SUM_BACKOFF_TIMES" + BackoffTypesStr = "BACKOFF_TYPES" + AvgMemStr = "AVG_MEM" + MaxMemStr = "MAX_MEM" + AvgDiskStr = "AVG_DISK" + MaxDiskStr = "MAX_DISK" + AvgKvTimeStr = "AVG_KV_TIME" + AvgPdTimeStr = "AVG_PD_TIME" + AvgBackoffTotalTimeStr = "AVG_BACKOFF_TOTAL_TIME" + AvgWriteSQLRespTimeStr = "AVG_WRITE_SQL_RESP_TIME" + MaxResultRowsStr = "MAX_RESULT_ROWS" + MinResultRowsStr = "MIN_RESULT_ROWS" + AvgResultRowsStr = "AVG_RESULT_ROWS" + PreparedStr = "PREPARED" + AvgAffectedRowsStr = "AVG_AFFECTED_ROWS" + FirstSeenStr = "FIRST_SEEN" + LastSeenStr = "LAST_SEEN" + PlanInCacheStr = "PLAN_IN_CACHE" + PlanCacheHitsStr = "PLAN_CACHE_HITS" + PlanInBindingStr = "PLAN_IN_BINDING" + QuerySampleTextStr = "QUERY_SAMPLE_TEXT" + PrevSampleTextStr = "PREV_SAMPLE_TEXT" + PlanDigestStr = "PLAN_DIGEST" + PlanStr = "PLAN" + BinaryPlan = "BINARY_PLAN" + Charset = "CHARSET" + Collation = "COLLATION" + PlanHint = "PLAN_HINT" +) + +type columnInfo interface { + getInstanceAddr() string + getTimeLocation() *time.Location +} + +type columnFactory func(info columnInfo, record *StmtRecord) interface{} + +var columnFactoryMap = map[string]columnFactory{ + ClusterTableInstanceColumnNameStr: func(info columnInfo, record *StmtRecord) interface{} { + return info.getInstanceAddr() + }, + SummaryBeginTimeStr: func(info columnInfo, record *StmtRecord) interface{} { + beginTime := time.Unix(record.Begin, 0) + if beginTime.Location() != info.getTimeLocation() { + beginTime = beginTime.In(info.getTimeLocation()) + } + return types.NewTime(types.FromGoTime(beginTime), mysql.TypeTimestamp, 0) + }, + SummaryEndTimeStr: func(info columnInfo, record *StmtRecord) interface{} { + endTime := time.Unix(record.End, 0) + if endTime.Location() != info.getTimeLocation() { + endTime = endTime.In(info.getTimeLocation()) + } + return types.NewTime(types.FromGoTime(endTime), mysql.TypeTimestamp, 0) + }, + StmtTypeStr: func(info columnInfo, record *StmtRecord) interface{} { + return record.StmtType + }, + SchemaNameStr: func(info columnInfo, record *StmtRecord) interface{} { + return convertEmptyToNil(record.SchemaName) + }, + DigestStr: func(info columnInfo, record *StmtRecord) interface{} { + return convertEmptyToNil(record.Digest) + }, + DigestTextStr: func(info columnInfo, record *StmtRecord) interface{} { + return record.NormalizedSQL + }, + TableNamesStr: func(info columnInfo, record *StmtRecord) interface{} { + return convertEmptyToNil(record.TableNames) + }, + IndexNamesStr: func(info columnInfo, record *StmtRecord) interface{} { + return convertEmptyToNil(strings.Join(record.IndexNames, ",")) + }, + SampleUserStr: func(info columnInfo, record *StmtRecord) interface{} { + sampleUser := "" + for key := range record.AuthUsers { + sampleUser = key + break + } + return convertEmptyToNil(sampleUser) + }, + ExecCountStr: func(info columnInfo, record *StmtRecord) interface{} { + return record.ExecCount + }, + SumErrorsStr: func(info columnInfo, record *StmtRecord) interface{} { + return record.SumErrors + }, + SumWarningsStr: func(info columnInfo, record *StmtRecord) interface{} { + return record.SumWarnings + }, + SumLatencyStr: func(info columnInfo, record *StmtRecord) interface{} { + return int64(record.SumLatency) + }, + MaxLatencyStr: func(info columnInfo, record *StmtRecord) interface{} { + return int64(record.MaxLatency) + }, + MinLatencyStr: func(info columnInfo, record *StmtRecord) interface{} { + return int64(record.MinLatency) + }, + AvgLatencyStr: func(info columnInfo, record *StmtRecord) interface{} { + return avgInt(int64(record.SumLatency), record.ExecCount) + }, + AvgParseLatencyStr: func(info columnInfo, record *StmtRecord) interface{} { + return avgInt(int64(record.SumParseLatency), record.ExecCount) + }, + MaxParseLatencyStr: func(info columnInfo, record *StmtRecord) interface{} { + return int64(record.MaxParseLatency) + }, + AvgCompileLatencyStr: func(info columnInfo, record *StmtRecord) interface{} { + return avgInt(int64(record.SumCompileLatency), record.ExecCount) + }, + MaxCompileLatencyStr: func(info columnInfo, record *StmtRecord) interface{} { + return int64(record.MaxCompileLatency) + }, + SumCopTaskNumStr: func(info columnInfo, record *StmtRecord) interface{} { + return record.SumNumCopTasks + }, + MaxCopProcessTimeStr: func(info columnInfo, record *StmtRecord) interface{} { + return int64(record.MaxCopProcessTime) + }, + MaxCopProcessAddressStr: func(info columnInfo, record *StmtRecord) interface{} { + return convertEmptyToNil(record.MaxCopProcessAddress) + }, + MaxCopWaitTimeStr: func(info columnInfo, record *StmtRecord) interface{} { + return int64(record.MaxCopWaitTime) + }, + MaxCopWaitAddressStr: func(info columnInfo, record *StmtRecord) interface{} { + return convertEmptyToNil(record.MaxCopWaitAddress) + }, + AvgProcessTimeStr: func(info columnInfo, record *StmtRecord) interface{} { + return avgInt(int64(record.SumProcessTime), record.ExecCount) + }, + MaxProcessTimeStr: func(info columnInfo, record *StmtRecord) interface{} { + return int64(record.MaxProcessTime) + }, + AvgWaitTimeStr: func(info columnInfo, record *StmtRecord) interface{} { + return avgInt(int64(record.SumWaitTime), record.ExecCount) + }, + MaxWaitTimeStr: func(info columnInfo, record *StmtRecord) interface{} { + return int64(record.MaxWaitTime) + }, + AvgBackoffTimeStr: func(info columnInfo, record *StmtRecord) interface{} { + return avgInt(int64(record.SumBackoffTime), record.ExecCount) + }, + MaxBackoffTimeStr: func(info columnInfo, record *StmtRecord) interface{} { + return int64(record.MaxBackoffTime) + }, + AvgTotalKeysStr: func(info columnInfo, record *StmtRecord) interface{} { + return avgInt(record.SumTotalKeys, record.ExecCount) + }, + MaxTotalKeysStr: func(info columnInfo, record *StmtRecord) interface{} { + return record.MaxTotalKeys + }, + AvgProcessedKeysStr: func(info columnInfo, record *StmtRecord) interface{} { + return avgInt(record.SumProcessedKeys, record.ExecCount) + }, + MaxProcessedKeysStr: func(info columnInfo, record *StmtRecord) interface{} { + return record.MaxProcessedKeys + }, + AvgRocksdbDeleteSkippedCountStr: func(info columnInfo, record *StmtRecord) interface{} { + return avgInt(int64(record.SumRocksdbDeleteSkippedCount), record.ExecCount) + }, + MaxRocksdbDeleteSkippedCountStr: func(info columnInfo, record *StmtRecord) interface{} { + return record.MaxRocksdbDeleteSkippedCount + }, + AvgRocksdbKeySkippedCountStr: func(info columnInfo, record *StmtRecord) interface{} { + return avgInt(int64(record.SumRocksdbKeySkippedCount), record.ExecCount) + }, + MaxRocksdbKeySkippedCountStr: func(info columnInfo, record *StmtRecord) interface{} { + return record.MaxRocksdbKeySkippedCount + }, + AvgRocksdbBlockCacheHitCountStr: func(info columnInfo, record *StmtRecord) interface{} { + return avgInt(int64(record.SumRocksdbBlockCacheHitCount), record.ExecCount) + }, + MaxRocksdbBlockCacheHitCountStr: func(info columnInfo, record *StmtRecord) interface{} { + return record.MaxRocksdbBlockCacheHitCount + }, + AvgRocksdbBlockReadCountStr: func(info columnInfo, record *StmtRecord) interface{} { + return avgInt(int64(record.SumRocksdbBlockReadCount), record.ExecCount) + }, + MaxRocksdbBlockReadCountStr: func(info columnInfo, record *StmtRecord) interface{} { + return record.MaxRocksdbBlockReadCount + }, + AvgRocksdbBlockReadByteStr: func(info columnInfo, record *StmtRecord) interface{} { + return avgInt(int64(record.SumRocksdbBlockReadByte), record.ExecCount) + }, + MaxRocksdbBlockReadByteStr: func(info columnInfo, record *StmtRecord) interface{} { + return record.MaxRocksdbBlockReadByte + }, + AvgPrewriteTimeStr: func(info columnInfo, record *StmtRecord) interface{} { + return avgInt(int64(record.SumPrewriteTime), record.CommitCount) + }, + MaxPrewriteTimeStr: func(info columnInfo, record *StmtRecord) interface{} { + return int64(record.MaxPrewriteTime) + }, + AvgCommitTimeStr: func(info columnInfo, record *StmtRecord) interface{} { + return avgInt(int64(record.SumCommitTime), record.CommitCount) + }, + MaxCommitTimeStr: func(info columnInfo, record *StmtRecord) interface{} { + return int64(record.MaxCommitTime) + }, + AvgGetCommitTsTimeStr: func(info columnInfo, record *StmtRecord) interface{} { + return avgInt(int64(record.SumGetCommitTsTime), record.CommitCount) + }, + MaxGetCommitTsTimeStr: func(info columnInfo, record *StmtRecord) interface{} { + return int64(record.MaxGetCommitTsTime) + }, + AvgCommitBackoffTimeStr: func(info columnInfo, record *StmtRecord) interface{} { + return avgInt(record.SumCommitBackoffTime, record.CommitCount) + }, + MaxCommitBackoffTimeStr: func(info columnInfo, record *StmtRecord) interface{} { + return record.MaxCommitBackoffTime + }, + AvgResolveLockTimeStr: func(info columnInfo, record *StmtRecord) interface{} { + return avgInt(record.SumResolveLockTime, record.CommitCount) + }, + MaxResolveLockTimeStr: func(info columnInfo, record *StmtRecord) interface{} { + return record.MaxResolveLockTime + }, + AvgLocalLatchWaitTimeStr: func(info columnInfo, record *StmtRecord) interface{} { + return avgInt(int64(record.SumLocalLatchTime), record.CommitCount) + }, + MaxLocalLatchWaitTimeStr: func(info columnInfo, record *StmtRecord) interface{} { + return int64(record.MaxLocalLatchTime) + }, + AvgWriteKeysStr: func(info columnInfo, record *StmtRecord) interface{} { + return avgFloat(record.SumWriteKeys, record.CommitCount) + }, + MaxWriteKeysStr: func(info columnInfo, record *StmtRecord) interface{} { + return record.MaxWriteKeys + }, + AvgWriteSizeStr: func(info columnInfo, record *StmtRecord) interface{} { + return avgFloat(record.SumWriteSize, record.CommitCount) + }, + MaxWriteSizeStr: func(info columnInfo, record *StmtRecord) interface{} { + return record.MaxWriteSize + }, + AvgPrewriteRegionsStr: func(info columnInfo, record *StmtRecord) interface{} { + return avgFloat(record.SumPrewriteRegionNum, record.CommitCount) + }, + MaxPrewriteRegionsStr: func(info columnInfo, record *StmtRecord) interface{} { + return int(record.MaxPrewriteRegionNum) + }, + AvgTxnRetryStr: func(info columnInfo, record *StmtRecord) interface{} { + return avgFloat(record.SumTxnRetry, record.CommitCount) + }, + MaxTxnRetryStr: func(info columnInfo, record *StmtRecord) interface{} { + return record.MaxTxnRetry + }, + SumExecRetryStr: func(info columnInfo, record *StmtRecord) interface{} { + return int(record.ExecRetryCount) + }, + SumExecRetryTimeStr: func(info columnInfo, record *StmtRecord) interface{} { + return int64(record.ExecRetryTime) + }, + SumBackoffTimesStr: func(info columnInfo, record *StmtRecord) interface{} { + return record.SumBackoffTimes + }, + BackoffTypesStr: func(info columnInfo, record *StmtRecord) interface{} { + return formatBackoffTypes(record.BackoffTypes) + }, + AvgMemStr: func(info columnInfo, record *StmtRecord) interface{} { + return avgInt(record.SumMem, record.ExecCount) + }, + MaxMemStr: func(info columnInfo, record *StmtRecord) interface{} { + return record.MaxMem + }, + AvgDiskStr: func(info columnInfo, record *StmtRecord) interface{} { + return avgInt(record.SumDisk, record.ExecCount) + }, + MaxDiskStr: func(info columnInfo, record *StmtRecord) interface{} { + return record.MaxDisk + }, + AvgKvTimeStr: func(info columnInfo, record *StmtRecord) interface{} { + return avgInt(int64(record.SumKVTotal), record.CommitCount) + }, + AvgPdTimeStr: func(info columnInfo, record *StmtRecord) interface{} { + return avgInt(int64(record.SumPDTotal), record.CommitCount) + }, + AvgBackoffTotalTimeStr: func(info columnInfo, record *StmtRecord) interface{} { + return avgInt(int64(record.SumBackoffTotal), record.CommitCount) + }, + AvgWriteSQLRespTimeStr: func(info columnInfo, record *StmtRecord) interface{} { + return avgInt(int64(record.SumWriteSQLRespTotal), record.CommitCount) + }, + MaxResultRowsStr: func(info columnInfo, record *StmtRecord) interface{} { + return record.MaxResultRows + }, + MinResultRowsStr: func(info columnInfo, record *StmtRecord) interface{} { + return record.MinResultRows + }, + AvgResultRowsStr: func(info columnInfo, record *StmtRecord) interface{} { + return avgInt(record.SumResultRows, record.ExecCount) + }, + PreparedStr: func(info columnInfo, record *StmtRecord) interface{} { + return record.Prepared + }, + AvgAffectedRowsStr: func(info columnInfo, record *StmtRecord) interface{} { + return avgFloat(int64(record.SumAffectedRows), record.ExecCount) + }, + FirstSeenStr: func(info columnInfo, record *StmtRecord) interface{} { + firstSeen := record.FirstSeen + if firstSeen.Location() != info.getTimeLocation() { + firstSeen = firstSeen.In(info.getTimeLocation()) + } + return types.NewTime(types.FromGoTime(firstSeen), mysql.TypeTimestamp, 0) + }, + LastSeenStr: func(info columnInfo, record *StmtRecord) interface{} { + lastSeen := record.LastSeen + if lastSeen.Location() != info.getTimeLocation() { + lastSeen = lastSeen.In(info.getTimeLocation()) + } + return types.NewTime(types.FromGoTime(lastSeen), mysql.TypeTimestamp, 0) + }, + PlanInCacheStr: func(info columnInfo, record *StmtRecord) interface{} { + return record.PlanInCache + }, + PlanCacheHitsStr: func(info columnInfo, record *StmtRecord) interface{} { + return record.PlanCacheHits + }, + PlanInBindingStr: func(info columnInfo, record *StmtRecord) interface{} { + return record.PlanInBinding + }, + QuerySampleTextStr: func(info columnInfo, record *StmtRecord) interface{} { + return record.SampleSQL + }, + PrevSampleTextStr: func(info columnInfo, record *StmtRecord) interface{} { + return record.PrevSQL + }, + PlanDigestStr: func(info columnInfo, record *StmtRecord) interface{} { + return record.PlanDigest + }, + PlanStr: func(info columnInfo, record *StmtRecord) interface{} { + plan, err := plancodec.DecodePlan(record.SamplePlan) + if err != nil { + logutil.BgLogger().Error("decode plan in statement summary failed", + zap.String("plan", record.SamplePlan), + zap.String("query", record.SampleSQL), zap.Error(err)) + plan = "" + } + return plan + }, + BinaryPlan: func(info columnInfo, record *StmtRecord) interface{} { + return record.SampleBinaryPlan + }, + Charset: func(info columnInfo, record *StmtRecord) interface{} { + return record.Charset + }, + Collation: func(info columnInfo, record *StmtRecord) interface{} { + return record.Collation + }, + PlanHint: func(info columnInfo, record *StmtRecord) interface{} { + return record.PlanHint + }, +} + +func makeColumnFactories(columns []*model.ColumnInfo) []columnFactory { + columnFactories := make([]columnFactory, len(columns)) + for i, col := range columns { + factory, ok := columnFactoryMap[col.Name.O] + if !ok { + panic(fmt.Sprintf("should never happen, should register new column %v into columnValueFactoryMap", col.Name.O)) + } + columnFactories[i] = factory + } + return columnFactories +} + +// Format the backoffType map to a string or nil. +func formatBackoffTypes(backoffMap map[string]int) interface{} { + type backoffStat struct { + backoffType string + count int + } + + size := len(backoffMap) + if size == 0 { + return nil + } + + backoffArray := make([]backoffStat, 0, len(backoffMap)) + for backoffType, count := range backoffMap { + backoffArray = append(backoffArray, backoffStat{backoffType, count}) + } + slices.SortFunc(backoffArray, func(i, j backoffStat) bool { + return i.count > j.count + }) + + var buffer bytes.Buffer + for index, stat := range backoffArray { + if _, err := fmt.Fprintf(&buffer, "%v:%d", stat.backoffType, stat.count); err != nil { + return "FORMAT ERROR" + } + if index < len(backoffArray)-1 { + buffer.WriteString(",") + } + } + return buffer.String() +} + +func avgInt(sum int64, count int64) int64 { + if count > 0 { + return sum / count + } + return 0 +} + +func avgFloat(sum int64, count int64) float64 { + if count > 0 { + return float64(sum) / float64(count) + } + return 0 +} + +func convertEmptyToNil(str string) interface{} { + if str == "" { + return nil + } + return str +} diff --git a/util/stmtsummary/v2/column_test.go b/util/stmtsummary/v2/column_test.go new file mode 100644 index 0000000000000..5e87cbe89a01c --- /dev/null +++ b/util/stmtsummary/v2/column_test.go @@ -0,0 +1,82 @@ +// Copyright 2023 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 stmtsummary + +import ( + "strings" + "testing" + "time" + + "github.com/pingcap/tidb/parser/model" + "github.com/stretchr/testify/require" +) + +func TestColumn(t *testing.T) { + columns := []*model.ColumnInfo{ + {Name: model.NewCIStr(ClusterTableInstanceColumnNameStr)}, + {Name: model.NewCIStr(StmtTypeStr)}, + {Name: model.NewCIStr(SchemaNameStr)}, + {Name: model.NewCIStr(DigestStr)}, + {Name: model.NewCIStr(DigestTextStr)}, + {Name: model.NewCIStr(TableNamesStr)}, + {Name: model.NewCIStr(IndexNamesStr)}, + {Name: model.NewCIStr(SampleUserStr)}, + {Name: model.NewCIStr(ExecCountStr)}, + {Name: model.NewCIStr(SumLatencyStr)}, + {Name: model.NewCIStr(MaxLatencyStr)}, + } + factories := makeColumnFactories(columns) + info := GenerateStmtExecInfo4Test("digest") + record := NewStmtRecord(info) + record.Add(info) + for n, f := range factories { + column := f(mockColumnInfo{}, record) + switch columns[n].Name.O { + case ClusterTableInstanceColumnNameStr: + require.Equal(t, "instance_addr", column) + case StmtTypeStr: + require.Equal(t, record.StmtType, column) + case SchemaNameStr: + require.Equal(t, record.SchemaName, column) + case DigestStr: + require.Equal(t, record.Digest, column) + case DigestTextStr: + require.Equal(t, record.NormalizedSQL, column) + case TableNamesStr: + require.Equal(t, record.TableNames, column) + case IndexNamesStr: + require.Equal(t, strings.Join(record.IndexNames, ","), column) + case SampleUserStr: + require.Equal(t, info.User, column) + case ExecCountStr: + require.Equal(t, int64(1), column) + case SumLatencyStr: + require.Equal(t, int64(record.SumLatency), column) + case MaxLatencyStr: + require.Equal(t, int64(record.MaxLatency), column) + } + } +} + +type mockColumnInfo struct{} + +func (mockColumnInfo) getInstanceAddr() string { + return "instance_addr" +} + +func (mockColumnInfo) getTimeLocation() *time.Location { + loc, _ := time.LoadLocation("Asia/Shanghai") + return loc +} diff --git a/util/stmtsummary/v2/logger.go b/util/stmtsummary/v2/logger.go new file mode 100644 index 0000000000000..5b244a838237b --- /dev/null +++ b/util/stmtsummary/v2/logger.go @@ -0,0 +1,111 @@ +// Copyright 2023 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 stmtsummary + +import ( + "encoding/json" + "fmt" + "time" + + "github.com/pingcap/log" + "github.com/pingcap/tidb/util/logutil" + "go.uber.org/zap" + "go.uber.org/zap/buffer" + "go.uber.org/zap/zapcore" +) + +var stmtLogEncoderPool = buffer.NewPool() + +type stmtLogStorage struct { + logger *zap.Logger +} + +func newStmtLogStorage(cfg *log.Config) *stmtLogStorage { + // Create the stmt logger + logger, prop, err := log.InitLogger(cfg) + if err != nil { + logutil.BgLogger().Error("failed to init logger", zap.Error(err)) + return &stmtLogStorage{logger: zap.NewNop()} + } + // Replace 2018-12-19-unified-log-format text encoder with statements encoder + newCore := log.NewTextCore(&stmtLogEncoder{}, prop.Syncer, prop.Level) + logger = logger.WithOptions(zap.WrapCore(func(core zapcore.Core) zapcore.Core { + return newCore + })) + return &stmtLogStorage{logger} +} + +func (s *stmtLogStorage) persist(w *stmtWindow, end time.Time) { + begin := w.begin.Unix() + for _, v := range w.lru.Values() { + r := v.(*lockedStmtRecord) + r.Lock() + r.Begin = begin + r.End = end.Unix() + s.log(r.StmtRecord) + r.Unlock() + } + w.evicted.Lock() + if w.evicted.other.ExecCount > 0 { + w.evicted.other.Begin = begin + w.evicted.other.End = end.Unix() + s.log(w.evicted.other) + } + w.evicted.Unlock() +} + +func (s *stmtLogStorage) log(r *StmtRecord) { + b, err := json.Marshal(r) + if err != nil { + logutil.BgLogger().Warn("failed to marshal statement summary", zap.Error(err)) + return + } + s.logger.Info(string(b)) +} + +type stmtLogEncoder struct{} + +func (*stmtLogEncoder) EncodeEntry(entry zapcore.Entry, _ []zapcore.Field) (*buffer.Buffer, error) { + b := stmtLogEncoderPool.Get() + fmt.Fprintf(b, "%s\n", entry.Message) + return b, nil +} + +func (e *stmtLogEncoder) Clone() zapcore.Encoder { return e } +func (*stmtLogEncoder) AddArray(string, zapcore.ArrayMarshaler) error { return nil } +func (*stmtLogEncoder) AddObject(string, zapcore.ObjectMarshaler) error { return nil } +func (*stmtLogEncoder) AddBinary(string, []byte) {} +func (*stmtLogEncoder) AddByteString(string, []byte) {} +func (*stmtLogEncoder) AddBool(string, bool) {} +func (*stmtLogEncoder) AddComplex128(string, complex128) {} +func (*stmtLogEncoder) AddComplex64(string, complex64) {} +func (*stmtLogEncoder) AddDuration(string, time.Duration) {} +func (*stmtLogEncoder) AddFloat64(string, float64) {} +func (*stmtLogEncoder) AddFloat32(string, float32) {} +func (*stmtLogEncoder) AddInt(string, int) {} +func (*stmtLogEncoder) AddInt64(string, int64) {} +func (*stmtLogEncoder) AddInt32(string, int32) {} +func (*stmtLogEncoder) AddInt16(string, int16) {} +func (*stmtLogEncoder) AddInt8(string, int8) {} +func (*stmtLogEncoder) AddString(string, string) {} +func (*stmtLogEncoder) AddTime(string, time.Time) {} +func (*stmtLogEncoder) AddUint(string, uint) {} +func (*stmtLogEncoder) AddUint64(string, uint64) {} +func (*stmtLogEncoder) AddUint32(string, uint32) {} +func (*stmtLogEncoder) AddUint16(string, uint16) {} +func (*stmtLogEncoder) AddUint8(string, uint8) {} +func (*stmtLogEncoder) AddUintptr(string, uintptr) {} +func (*stmtLogEncoder) AddReflected(string, interface{}) error { return nil } +func (*stmtLogEncoder) OpenNamespace(string) {} diff --git a/util/stmtsummary/v2/main_test.go b/util/stmtsummary/v2/main_test.go new file mode 100644 index 0000000000000..5756815f7492d --- /dev/null +++ b/util/stmtsummary/v2/main_test.go @@ -0,0 +1,33 @@ +// Copyright 2023 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 stmtsummary + +import ( + "testing" + + "github.com/pingcap/tidb/testkit/testsetup" + "go.uber.org/goleak" +) + +func TestMain(m *testing.M) { + testsetup.SetupForCommonTest() + opts := []goleak.Option{ + goleak.IgnoreTopFunction("github.com/golang/glog.(*loggingT).flushDaemon"), + goleak.IgnoreTopFunction("github.com/lestrrat-go/httprc.runFetchWorker"), + goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), + } + goleak.VerifyTestMain(m, opts...) +} diff --git a/util/stmtsummary/v2/reader.go b/util/stmtsummary/v2/reader.go new file mode 100644 index 0000000000000..d7c63e162ed0f --- /dev/null +++ b/util/stmtsummary/v2/reader.go @@ -0,0 +1,863 @@ +// Copyright 2023 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 stmtsummary + +import ( + "bufio" + "context" + "encoding/json" + "io" + "math" + "os" + "path/filepath" + "strings" + "sync" + "time" + + "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/parser/auth" + "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util" + "github.com/pingcap/tidb/util/logutil" + "github.com/pingcap/tidb/util/set" + "go.uber.org/zap" + "golang.org/x/exp/slices" +) + +const ( + logFileTimeFormat = "2006-01-02T15-04-05.000" // depends on lumberjack.go#backupTimeFormat + maxLineSize = 1073741824 + + batchScanSize = 64 +) + +// StmtTimeRange is the time range type used in the stmtsummary package. +// [Begin, End) +type StmtTimeRange struct { + Begin int64 + End int64 +} + +// MemReader is used to read the current window's data maintained in memory by StmtSummary. +type MemReader struct { + s *StmtSummary + columns []*model.ColumnInfo + instanceAddr string + timeLocation *time.Location + + columnFactories []columnFactory + checker *stmtChecker +} + +// NewMemReader creates a MemReader from StmtSummary and other necessary parameters. +func NewMemReader(s *StmtSummary, + columns []*model.ColumnInfo, + instanceAddr string, + timeLocation *time.Location, + user *auth.UserIdentity, + hasProcessPriv bool, + digests set.StringSet, + timeRanges []*StmtTimeRange) *MemReader { + return &MemReader{ + s: s, + columns: columns, + instanceAddr: instanceAddr, + timeLocation: timeLocation, + columnFactories: makeColumnFactories(columns), + checker: &stmtChecker{ + user: user, + hasProcessPriv: hasProcessPriv, + digests: digests, + timeRanges: timeRanges, + }, + } +} + +// Rows returns rows converted from the current window's data maintained +// in memory by StmtSummary. All evicted data will be aggregated into a +// single row appended at the end. +func (r *MemReader) Rows() [][]types.Datum { + if r.s == nil { + return nil + } + end := timeNow().Unix() + r.s.windowLock.Lock() + w := r.s.window + if !r.checker.isTimeValid(w.begin.Unix(), end) { + r.s.windowLock.Unlock() + return nil + } + values := w.lru.Values() + evicted := w.evicted + r.s.windowLock.Unlock() + rows := make([][]types.Datum, 0, len(values)+1) + for _, v := range values { + record := v.(*lockedStmtRecord) + if !r.checker.isDigestValid(record.Digest) { + continue + } + func() { + record.Lock() + defer record.Unlock() + if !r.checker.hasPrivilege(record.AuthUsers) { + return + } + record.Begin = w.begin.Unix() + record.End = end + row := make([]types.Datum, len(r.columnFactories)) + for i, factory := range r.columnFactories { + row[i] = types.NewDatum(factory(r, record.StmtRecord)) + } + rows = append(rows, row) + }() + } + if r.checker.digests == nil { + func() { + evicted.Lock() + defer evicted.Unlock() + if evicted.other.ExecCount == 0 { + return + } + if !r.checker.hasPrivilege(evicted.other.AuthUsers) { + return + } + evicted.other.Begin = w.begin.Unix() + evicted.other.End = end + row := make([]types.Datum, len(r.columnFactories)) + for i, factory := range r.columnFactories { + row[i] = types.NewDatum(factory(r, evicted.other)) + } + rows = append(rows, row) + }() + } + return rows +} + +// getInstanceAddr implements columnInfo. +func (r *MemReader) getInstanceAddr() string { + return r.instanceAddr +} + +// getInstanceAddr implements columnInfo. +func (r *MemReader) getTimeLocation() *time.Location { + return r.timeLocation +} + +// HistoryReader is used to read data that has been persisted to files. +type HistoryReader struct { + ctx context.Context + cancel context.CancelFunc + wg sync.WaitGroup + + instanceAddr string + timeLocation *time.Location + + columnFactories []columnFactory + checker *stmtChecker + files *stmtFiles + + concurrent int + rowsCh <-chan [][]types.Datum + errCh <-chan error +} + +// NewHistoryReader creates a HisroryReader from StmtSummary and other +// necessary parameters. If timeRanges is present, only files within +// the time range will be read. +func NewHistoryReader( + ctx context.Context, + columns []*model.ColumnInfo, + instanceAddr string, + timeLocation *time.Location, + user *auth.UserIdentity, + hasProcessPriv bool, + digests set.StringSet, + timeRanges []*StmtTimeRange, + concurrent int, +) (*HistoryReader, error) { + files, err := newStmtFiles(ctx, timeRanges) + if err != nil { + return nil, err + } + + if concurrent < 2 { + concurrent = 2 + } + rowsCh := make(chan [][]types.Datum, concurrent) + errCh := make(chan error, concurrent) + + ctx, cancel := context.WithCancel(ctx) + r := &HistoryReader{ + ctx: ctx, + cancel: cancel, + + instanceAddr: instanceAddr, + timeLocation: timeLocation, + columnFactories: makeColumnFactories(columns), + checker: &stmtChecker{ + user: user, + hasProcessPriv: hasProcessPriv, + digests: digests, + timeRanges: timeRanges, + }, + files: files, + concurrent: concurrent, + rowsCh: rowsCh, + errCh: errCh, + } + + r.wg.Add(1) + go func() { + defer r.wg.Done() + r.scheduleTasks(rowsCh, errCh) + }() + return r, nil +} + +// Rows returns rows converted from records in files. Reading and parsing +// works asynchronously. If (nil, nil) is returned, it means that the +// reading has been completed. +func (r *HistoryReader) Rows() ([][]types.Datum, error) { + ctx := r.ctx + for { + select { + case err := <-r.errCh: + return nil, err + case rows, ok := <-r.rowsCh: + if !ok { + select { + case err := <-r.errCh: + return nil, err + default: + return nil, nil + } + } + if len(rows) == 0 { + continue + } + return rows, nil + case <-ctx.Done(): + return nil, ctx.Err() + } + } +} + +// Close ends reading and closes all files. +func (r *HistoryReader) Close() error { + r.files.close() + if r.cancel != nil { + r.cancel() + } + r.wg.Wait() + return nil +} + +// 4 roles to handle the read task in pipeline: +// +// ## Pipeline +// . +--------------+ +---------------+ +// == files => | scan workers | == lines => | parse workers | == rows => +// . filesCh +--------------+ linesCh +---------------+ rowsCh +// +// ## Roles +// +--------------+--------------+------------------------------------+ +// | ROLE | COUNT | DESCRIPTION | +// +--------------+--------------+------------------------------------+ +// | Scan Worker | concurrent/2 | Scan files (I/O) first, then help | +// | | | parse workers to parse lines (CPU) | +// +--------------+--------------+------------------------------------+ +// | Parse Worker | concurrent- | Parse lines (CPU) to rows | +// | | concurrent/2 | | +// +--------------+--------------+------------------------------------+ +// | Manager | 1 | Drive the whole process and notify | +// | | | scan workers to switch role | +// +--------------+--------------+------------------------------------+ +// | Monitor | 1 | Cover failures and notify workers | +// | | | to exit | +// +--------------+--------------+------------------------------------+ +func (r *HistoryReader) scheduleTasks( + rowsCh chan<- [][]types.Datum, + errCh chan<- error, +) { + if r.files == nil || len(r.files.files) == 0 { + close(rowsCh) + return + } + + ctx, cancel := context.WithCancel(r.ctx) + defer cancel() + + scanWorker := &stmtScanWorker{ + ctx: ctx, + batchSize: batchScanSize, + checker: r.checker, + } + parseWorker := &stmtParseWorker{ + ctx: ctx, + instanceAddr: r.instanceAddr, + timeLocation: r.timeLocation, + checker: r.checker, + columnFactories: r.columnFactories, + } + + concurrent := r.concurrent + filesCh := make(chan *os.File, concurrent) + linesCh := make(chan [][]byte, concurrent) + innerErrCh := make(chan error, concurrent) + + var scanWg sync.WaitGroup + scanWg.Add(concurrent / 2) + scanDone := scanWg.Done + waitScanAllDone := scanWg.Wait + + var parseWg sync.WaitGroup + parseWg.Add(concurrent) // finally all workers will become parse workers + parseDone := parseWg.Done + waitParseAllDone := parseWg.Wait + + // Half of workers are scheduled to scan files and then parse lines. + for i := 0; i < concurrent/2; i++ { + go func() { + scanWorker.run(filesCh, linesCh, innerErrCh) + scanDone() + + parseWorker.run(linesCh, rowsCh, innerErrCh) + parseDone() + }() + } + + // Remaining workers are scheduled to parse lines. + for i := concurrent / 2; i < concurrent; i++ { + go func() { + parseWorker.run(linesCh, rowsCh, innerErrCh) + parseDone() + }() + } + + // Manager drives the whole process + var mgrWg sync.WaitGroup + mgrWg.Add(1) + go func() { + defer mgrWg.Done() + + func() { + for _, file := range r.files.files { + select { + case filesCh <- file.file: + case <-ctx.Done(): + return + } + } + }() + // No scan tasks to be generating. Notify idle scan + // workers to become parse workers + close(filesCh) + + // No parse tasks to be generating once all scan + // tasks are done. Notify idle parse workers to exit + waitScanAllDone() + close(linesCh) + + // No rows to be generating once all parse tasks + // are done. Notify monitor to close rowsCh + waitParseAllDone() + cancel() + }() + + // Monitor to cover failures and notify workers to exit + select { + case err := <-innerErrCh: + select { + case errCh <- err: + default: + } + cancel() // notify workers to exit + case <-ctx.Done(): + // notified by manager or parent ctx is canceled + } + close(rowsCh) // task done + mgrWg.Wait() +} + +type stmtChecker struct { + user *auth.UserIdentity + hasProcessPriv bool // If the user has the 'PROCESS' privilege, he can read all statements. + digests set.StringSet + timeRanges []*StmtTimeRange +} + +func (c *stmtChecker) hasPrivilege(authUsers map[string]struct{}) bool { + authed := true + if c.user != nil && !c.hasProcessPriv { + if len(authUsers) == 0 { + return false + } + _, authed = authUsers[c.user.Username] + } + return authed +} + +func (c *stmtChecker) isDigestValid(digest string) bool { + if c.digests == nil { + return true + } + return c.digests.Exist(digest) +} + +func (c *stmtChecker) isTimeValid(begin, end int64) bool { + if len(c.timeRanges) == 0 { + return true + } + for _, tr := range c.timeRanges { + if timeRangeOverlap(begin, end, tr.Begin, tr.End) { + return true + } + } + return false +} + +func (c *stmtChecker) needStop(curBegin int64) bool { + if len(c.timeRanges) == 0 { + return false + } + stop := true + for _, tr := range c.timeRanges { + if tr.End == 0 || tr.End >= curBegin { + stop = false + } + } + return stop +} + +type stmtTinyRecord struct { + Begin int64 `json:"begin"` + End int64 `json:"end"` +} + +type stmtFile struct { + file *os.File + begin int64 + end int64 +} + +func openStmtFile(path string) (*stmtFile, error) { + file, err := os.OpenFile(path, os.O_RDONLY, os.ModePerm) + if err != nil { + return nil, err + } + begin, err := parseBeginTsAndReseek(file) + if err != nil { + if err != io.EOF { + _ = file.Close() + return nil, err + } + } + end, err := parseEndTs(file) + if err != nil { + _ = file.Close() + return nil, err + } + + return &stmtFile{ + file: file, + begin: begin, + end: end, + }, nil +} + +func parseBeginTsAndReseek(file *os.File) (int64, error) { + if _, err := file.Seek(0, io.SeekStart); err != nil { + return 0, err + } + firstLine, err := readLine(bufio.NewReader(file)) + if err != nil { + return 0, err + } + if _, err := file.Seek(0, io.SeekStart); err != nil { + return 0, err + } + if len(firstLine) == 0 { + return 0, nil + } + var record stmtTinyRecord + if err := json.Unmarshal(firstLine, &record); err != nil { + return 0, err + } + return record.Begin, nil +} + +func parseEndTs(file *os.File) (int64, error) { + // tidb-statements.log + filename := config.GetGlobalConfig().Instance.StmtSummaryFilename + // .log + ext := filepath.Ext(filename) + // tidb-statements + prefix := filename[:len(filename)-len(ext)] + + // tidb-statements-2022-12-27T16-21-20.245.log + filename = filepath.Base(file.Name()) + // .log + ext = filepath.Ext(file.Name()) + // tidb-statements-2022-12-27T16-21-20.245 + filename = filename[:len(filename)-len(ext)] + + if strings.HasPrefix(filename, prefix+"-") { + // 2022-12-27T16-21-20.245 + timeStr := strings.TrimPrefix(filename, prefix+"-") + end, err := time.ParseInLocation(logFileTimeFormat, timeStr, time.Local) + if err != nil { + return 0, err + } + return end.Unix(), nil + } + return 0, nil +} + +func (f *stmtFile) close() error { + if f.file != nil { + return f.file.Close() + } + return nil +} + +type stmtFiles struct { + files []*stmtFile +} + +func newStmtFiles(ctx context.Context, timeRanges []*StmtTimeRange) (*stmtFiles, error) { + filename := config.GetGlobalConfig().Instance.StmtSummaryFilename + ext := filepath.Ext(filename) + prefix := filename[:len(filename)-len(ext)] + var files []*stmtFile + walkFn := func(path string, info os.DirEntry) error { + if info.IsDir() { + return nil + } + if !strings.HasPrefix(path, prefix) { + return nil + } + if isCtxDone(ctx) { + return ctx.Err() + } + file, err := openStmtFile(path) + if err != nil { + logutil.BgLogger().Warn("failed to open or parse statements file", zap.Error(err), zap.String("path", path)) + return nil + } + if len(timeRanges) == 0 { + files = append(files, file) + return nil + } + for _, tr := range timeRanges { + if timeRangeOverlap(file.begin, file.end, tr.Begin, tr.End) { + files = append(files, file) + return nil + } + } + return nil + } + + dir := filepath.Dir(filename) + entries, err := os.ReadDir(dir) + if err != nil { + return nil, err + } + for _, entry := range entries { + if err := walkFn(filepath.Join(dir, entry.Name()), entry); err != nil { + for _, f := range files { + _ = f.close() + } + return nil, err + } + } + slices.SortFunc(files, func(i, j *stmtFile) bool { + return i.begin < j.begin + }) + return &stmtFiles{files: files}, nil +} + +func (f *stmtFiles) close() { + for _, f := range f.files { + _ = f.close() + } +} + +type stmtScanWorker struct { + ctx context.Context + batchSize int + checker *stmtChecker +} + +func (w *stmtScanWorker) run( + fileCh <-chan *os.File, + linesCh chan<- [][]byte, + errCh chan<- error, +) { + for { + select { + case file, ok := <-fileCh: + if !ok { + return + } + w.handleFile(file, linesCh, errCh) + case <-w.ctx.Done(): + return + } + } +} + +func (w *stmtScanWorker) handleFile( + file *os.File, + linesCh chan<- [][]byte, + errCh chan<- error, +) { + if file == nil { + return + } + + reader := bufio.NewReader(file) + for { + if isCtxDone(w.ctx) { + return + } + + lines, err := w.readlines(reader) + if err == io.EOF { + return + } + if err != nil { + w.putErr(err, errCh) + return + } + + w.putLines(lines, linesCh) + } +} + +func (w *stmtScanWorker) putErr( + err error, + errCh chan<- error, +) { + select { + case errCh <- err: + case <-w.ctx.Done(): + } +} + +func (w *stmtScanWorker) putLines( + lines [][]byte, + linesCh chan<- [][]byte, +) { + select { + case linesCh <- lines: + case <-w.ctx.Done(): + } +} + +func (w *stmtScanWorker) readlines(reader *bufio.Reader) ([][]byte, error) { + firstLine, err := readLine(reader) + if err != nil { + return nil, err + } + + record, err := w.parse(firstLine) + if err != nil { + return nil, err + } + + if w.needStop(record) { + // done because remaining lines in file + // are not in the time range + return nil, io.EOF + } + + lines := make([][]byte, 0, w.batchSize) + lines = append(lines, firstLine) + + newLines, err := readLines(reader, w.batchSize-1) + if err == io.EOF { + return lines, nil + } + if err != nil { + return nil, err + } + + lines = append(lines, newLines...) + return lines, nil +} + +func (*stmtScanWorker) parse(raw []byte) (*stmtTinyRecord, error) { + var record stmtTinyRecord + if err := json.Unmarshal(raw, &record); err != nil { + return nil, err + } + return &record, nil +} + +func (w *stmtScanWorker) needStop(record *stmtTinyRecord) bool { + return w.checker.needStop(record.Begin) +} + +type stmtParseWorker struct { + ctx context.Context + instanceAddr string + timeLocation *time.Location + checker *stmtChecker + columnFactories []columnFactory +} + +func (w *stmtParseWorker) run( + linesCh <-chan [][]byte, + rowsCh chan<- [][]types.Datum, + errCh chan<- error, +) { + for { + select { + case lines, ok := <-linesCh: + if !ok { + return + } + w.handleLines(lines, rowsCh, errCh) + case <-w.ctx.Done(): + return + } + } +} + +func (w *stmtParseWorker) handleLines( + lines [][]byte, + rowsCh chan<- [][]types.Datum, + errCh chan<- error, +) { + if len(lines) == 0 { + return + } + + rows := make([][]types.Datum, 0, len(lines)) + for _, line := range lines { + record, err := w.parse(line) + if err != nil { + w.putErr(err, errCh) + return + } + + if w.needStop(record) { + break + } + + if !w.matchConds(record) { + continue + } + + row := w.buildRow(record) + rows = append(rows, row) + } + + if len(rows) > 0 { + w.putRows(rows, rowsCh) + } +} + +func (w *stmtParseWorker) putErr( + err error, + errCh chan<- error, +) { + select { + case errCh <- err: + case <-w.ctx.Done(): + } +} + +func (w *stmtParseWorker) putRows( + rows [][]types.Datum, + rowsCh chan<- [][]types.Datum, +) { + select { + case rowsCh <- rows: + case <-w.ctx.Done(): + } +} + +func (*stmtParseWorker) parse(raw []byte) (*StmtRecord, error) { + var record StmtRecord + if err := json.Unmarshal(raw, &record); err != nil { + return nil, err + } + return &record, nil +} + +func (w *stmtParseWorker) needStop(record *StmtRecord) bool { + return w.checker.needStop(record.Begin) +} + +func (w *stmtParseWorker) matchConds(record *StmtRecord) bool { + if !w.checker.isTimeValid(record.Begin, record.End) { + return false + } + if !w.checker.isDigestValid(record.Digest) { + return false + } + if !w.checker.hasPrivilege(record.AuthUsers) { + return false + } + return true +} + +func (w *stmtParseWorker) buildRow(record *StmtRecord) []types.Datum { + row := make([]types.Datum, len(w.columnFactories)) + for n, factory := range w.columnFactories { + row[n] = types.NewDatum(factory(w, record)) + } + return row +} + +// getInstanceAddr implements columnInfo. +func (w *stmtParseWorker) getInstanceAddr() string { + return w.instanceAddr +} + +// getInstanceAddr implements columnInfo. +func (w *stmtParseWorker) getTimeLocation() *time.Location { + return w.timeLocation +} + +func isCtxDone(ctx context.Context) bool { + select { + case <-ctx.Done(): + return true + default: + return false + } +} + +func readLine(reader *bufio.Reader) ([]byte, error) { + return util.ReadLine(reader, maxLineSize) +} + +func readLines(reader *bufio.Reader, count int) ([][]byte, error) { + return util.ReadLines(reader, count, maxLineSize) +} + +func timeRangeOverlap(aBegin, aEnd, bBegin, bEnd int64) bool { + if aEnd == 0 || aEnd < aBegin { + aEnd = math.MaxInt64 + } + if bEnd == 0 || bEnd < bBegin { + bEnd = math.MaxInt64 + } + // https://stackoverflow.com/questions/3269434/whats-the-most-efficient-way-to-test-if-two-ranges-overlap + return aBegin <= bEnd && aEnd >= bBegin +} diff --git a/util/stmtsummary/v2/reader_test.go b/util/stmtsummary/v2/reader_test.go new file mode 100644 index 0000000000000..626560a318910 --- /dev/null +++ b/util/stmtsummary/v2/reader_test.go @@ -0,0 +1,393 @@ +// Copyright 2023 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 stmtsummary + +import ( + "bufio" + "context" + "os" + "testing" + "time" + + "github.com/pingcap/tidb/parser/auth" + "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util" + "github.com/pingcap/tidb/util/set" + "github.com/stretchr/testify/require" +) + +func TestTimeRangeOverlap(t *testing.T) { + require.False(t, timeRangeOverlap(1, 2, 3, 4)) + require.False(t, timeRangeOverlap(3, 4, 1, 2)) + require.True(t, timeRangeOverlap(1, 2, 2, 3)) + require.True(t, timeRangeOverlap(1, 3, 2, 4)) + require.True(t, timeRangeOverlap(2, 4, 1, 3)) + require.True(t, timeRangeOverlap(1, 0, 3, 4)) + require.True(t, timeRangeOverlap(1, 0, 2, 0)) +} + +func TestStmtFile(t *testing.T) { + filename := "tidb-statements-2022-12-27T16-21-20.245.log" + + file, err := os.Create(filename) + require.NoError(t, err) + defer func() { + require.NoError(t, os.Remove(filename)) + }() + _, err = file.WriteString("{\"begin\":1,\"end\":2}\n") + require.NoError(t, err) + _, err = file.WriteString("{\"begin\":3,\"end\":4}\n") + require.NoError(t, err) + require.NoError(t, file.Close()) + + f, err := openStmtFile(filename) + require.NoError(t, err) + defer func() { + require.NoError(t, f.file.Close()) + }() + require.Equal(t, int64(1), f.begin) + require.Equal(t, int64(1672129280), f.end) // 2022-12-27T16-21-20.245 == 1672129280 + + // Check if seek 0. + firstLine, err := util.ReadLine(bufio.NewReader(f.file), maxLineSize) + require.NoError(t, err) + require.Equal(t, `{"begin":1,"end":2}`, string(firstLine)) +} + +func TestStmtFiles(t *testing.T) { + filename1 := "tidb-statements-2022-12-27T16-21-20.245.log" + filename2 := "tidb-statements.log" + + file, err := os.Create(filename1) + require.NoError(t, err) + defer func() { + require.NoError(t, os.Remove(filename1)) + }() + _, err = file.WriteString("{\"begin\":1672128520,\"end\":1672128530}\n") + require.NoError(t, err) + _, err = file.WriteString("{\"begin\":1672129270,\"end\":1672129280}\n") + require.NoError(t, err) + require.NoError(t, file.Close()) + + file, err = os.Create(filename2) + require.NoError(t, err) + defer func() { + require.NoError(t, os.Remove(filename2)) + }() + _, err = file.WriteString("{\"begin\":1672129270,\"end\":1672129280}\n") + require.NoError(t, err) + _, err = file.WriteString("{\"begin\":1672129380,\"end\":1672129390}\n") + require.NoError(t, err) + require.NoError(t, file.Close()) + + func() { + files, err := newStmtFiles(context.Background(), nil) + require.NoError(t, err) + defer files.close() + require.Len(t, files.files, 2) + require.Equal(t, filename1, files.files[0].file.Name()) + require.Equal(t, filename2, files.files[1].file.Name()) + }() + + func() { + files, err := newStmtFiles(context.Background(), []*StmtTimeRange{ + {Begin: 1672129270, End: 1672129271}, + }) + require.NoError(t, err) + defer files.close() + require.Len(t, files.files, 2) + require.Equal(t, filename1, files.files[0].file.Name()) + require.Equal(t, filename2, files.files[1].file.Name()) + }() + + func() { + files, err := newStmtFiles(context.Background(), []*StmtTimeRange{ + {Begin: 0, End: 1672129270}, + }) + require.NoError(t, err) + defer files.close() + require.Len(t, files.files, 2) + require.Equal(t, filename1, files.files[0].file.Name()) + require.Equal(t, filename2, files.files[1].file.Name()) + }() + + func() { + files, err := newStmtFiles(context.Background(), []*StmtTimeRange{ + {Begin: 0, End: 1672129269}, + }) + require.NoError(t, err) + defer files.close() + require.Len(t, files.files, 1) + require.Equal(t, filename1, files.files[0].file.Name()) + }() + + func() { + files, err := newStmtFiles(context.Background(), []*StmtTimeRange{ + {Begin: 0, End: 1}, + }) + require.NoError(t, err) + defer files.close() + require.Empty(t, files.files) + }() + + func() { + files, err := newStmtFiles(context.Background(), []*StmtTimeRange{ + {Begin: 1672129281, End: 0}, + }) + require.NoError(t, err) + defer files.close() + require.Len(t, files.files, 1) + require.Equal(t, filename2, files.files[0].file.Name()) + }() +} + +func TestStmtChecker(t *testing.T) { + checker := &stmtChecker{} + require.True(t, checker.hasPrivilege(nil)) + + checker = &stmtChecker{ + user: &auth.UserIdentity{Username: "user1"}, + } + require.False(t, checker.hasPrivilege(nil)) + require.False(t, checker.hasPrivilege(map[string]struct{}{"user2": {}})) + require.True(t, checker.hasPrivilege(map[string]struct{}{"user1": {}, "user2": {}})) + + checker = &stmtChecker{} + require.True(t, checker.isDigestValid("digest1")) + + checker = &stmtChecker{ + digests: set.NewStringSet("digest2"), + } + require.False(t, checker.isDigestValid("digest1")) + require.True(t, checker.isDigestValid("digest2")) + + checker = &stmtChecker{ + digests: set.NewStringSet("digest1", "digest2"), + } + require.True(t, checker.isDigestValid("digest1")) + require.True(t, checker.isDigestValid("digest2")) + + checker = &stmtChecker{} + require.True(t, checker.isTimeValid(1, 2)) + require.False(t, checker.needStop(2)) + require.False(t, checker.needStop(3)) + + checker = &stmtChecker{ + timeRanges: []*StmtTimeRange{ + {Begin: 1, End: 2}, + }, + } + require.True(t, checker.isTimeValid(1, 2)) + require.False(t, checker.isTimeValid(3, 4)) + require.False(t, checker.needStop(2)) + require.True(t, checker.needStop(3)) +} + +func TestMemReader(t *testing.T) { + timeLocation, err := time.LoadLocation("Asia/Shanghai") + require.NoError(t, err) + columns := []*model.ColumnInfo{ + {Name: model.NewCIStr(DigestStr)}, + {Name: model.NewCIStr(ExecCountStr)}, + } + + ss := NewStmtSummary4Test(3) + defer ss.Close() + + ss.Add(GenerateStmtExecInfo4Test("digest1")) + ss.Add(GenerateStmtExecInfo4Test("digest1")) + ss.Add(GenerateStmtExecInfo4Test("digest2")) + ss.Add(GenerateStmtExecInfo4Test("digest2")) + ss.Add(GenerateStmtExecInfo4Test("digest3")) + ss.Add(GenerateStmtExecInfo4Test("digest3")) + ss.Add(GenerateStmtExecInfo4Test("digest4")) + ss.Add(GenerateStmtExecInfo4Test("digest4")) + ss.Add(GenerateStmtExecInfo4Test("digest5")) + ss.Add(GenerateStmtExecInfo4Test("digest5")) + reader := NewMemReader(ss, columns, "", timeLocation, nil, false, nil, nil) + rows := reader.Rows() + require.Len(t, rows, 4) // 3 rows + 1 other + require.Equal(t, len(reader.columnFactories), len(rows[0])) + evicted := ss.Evicted() + require.Len(t, evicted, 3) // begin, end, count +} + +func TestHistoryReader(t *testing.T) { + filename1 := "tidb-statements-2022-12-27T16-21-20.245.log" + filename2 := "tidb-statements.log" + + file, err := os.Create(filename1) + require.NoError(t, err) + defer func() { + require.NoError(t, os.Remove(filename1)) + }() + _, err = file.WriteString("{\"begin\":1672128520,\"end\":1672128530,\"digest\":\"digest1\",\"exec_count\":10}\n") + require.NoError(t, err) + _, err = file.WriteString("{\"begin\":1672129270,\"end\":1672129280,\"digest\":\"digest2\",\"exec_count\":20}\n") + require.NoError(t, err) + require.NoError(t, file.Close()) + + file, err = os.Create(filename2) + require.NoError(t, err) + defer func() { + require.NoError(t, os.Remove(filename2)) + }() + _, err = file.WriteString("{\"begin\":1672129270,\"end\":1672129280,\"digest\":\"digest2\",\"exec_count\":30}\n") + require.NoError(t, err) + _, err = file.WriteString("{\"begin\":1672129380,\"end\":1672129390,\"digest\":\"digest3\",\"exec_count\":40}\n") + require.NoError(t, err) + require.NoError(t, file.Close()) + + timeLocation, err := time.LoadLocation("Asia/Shanghai") + require.NoError(t, err) + columns := []*model.ColumnInfo{ + {Name: model.NewCIStr(DigestStr)}, + {Name: model.NewCIStr(ExecCountStr)}, + } + + func() { + reader, err := NewHistoryReader(context.Background(), columns, "", timeLocation, nil, false, nil, nil, 2) + require.NoError(t, err) + defer reader.Close() + rows := readAllRows(t, reader) + require.Len(t, rows, 4) + for _, row := range rows { + require.Equal(t, len(columns), len(row)) + } + }() + + func() { + reader, err := NewHistoryReader(context.Background(), columns, "", timeLocation, nil, false, set.NewStringSet("digest2"), nil, 2) + require.NoError(t, err) + defer reader.Close() + rows := readAllRows(t, reader) + require.Len(t, rows, 2) + for _, row := range rows { + require.Equal(t, len(columns), len(row)) + } + }() + + func() { + reader, err := NewHistoryReader(context.Background(), columns, "", timeLocation, nil, false, nil, []*StmtTimeRange{ + {Begin: 0, End: 1672128520 - 1}, + }, 2) + require.NoError(t, err) + defer reader.Close() + rows := readAllRows(t, reader) + require.Len(t, rows, 0) + }() + + func() { + reader, err := NewHistoryReader(context.Background(), columns, "", timeLocation, nil, false, nil, []*StmtTimeRange{ + {Begin: 0, End: 1672129270 - 1}, + }, 2) + require.NoError(t, err) + defer reader.Close() + rows := readAllRows(t, reader) + require.Len(t, rows, 1) + for _, row := range rows { + require.Equal(t, len(columns), len(row)) + } + }() + + func() { + reader, err := NewHistoryReader(context.Background(), columns, "", timeLocation, nil, false, nil, []*StmtTimeRange{ + {Begin: 0, End: 1672129270}, + }, 2) + require.NoError(t, err) + defer reader.Close() + rows := readAllRows(t, reader) + require.Len(t, rows, 3) + for _, row := range rows { + require.Equal(t, len(columns), len(row)) + } + }() + + func() { + reader, err := NewHistoryReader(context.Background(), columns, "", timeLocation, nil, false, nil, []*StmtTimeRange{ + {Begin: 0, End: 1672129380}, + }, 2) + require.NoError(t, err) + defer reader.Close() + rows := readAllRows(t, reader) + require.Len(t, rows, 4) + for _, row := range rows { + require.Equal(t, len(columns), len(row)) + } + }() + + func() { + reader, err := NewHistoryReader(context.Background(), columns, "", timeLocation, nil, false, nil, []*StmtTimeRange{ + {Begin: 1672129270, End: 1672129380}, + }, 2) + require.NoError(t, err) + defer reader.Close() + rows := readAllRows(t, reader) + require.Len(t, rows, 3) + for _, row := range rows { + require.Equal(t, len(columns), len(row)) + } + }() + + func() { + reader, err := NewHistoryReader(context.Background(), columns, "", timeLocation, nil, false, nil, []*StmtTimeRange{ + {Begin: 1672129390, End: 0}, + }, 2) + require.NoError(t, err) + defer reader.Close() + rows := readAllRows(t, reader) + require.Len(t, rows, 1) + for _, row := range rows { + require.Equal(t, len(columns), len(row)) + } + }() + + func() { + reader, err := NewHistoryReader(context.Background(), columns, "", timeLocation, nil, false, nil, []*StmtTimeRange{ + {Begin: 1672129391, End: 0}, + }, 2) + require.NoError(t, err) + defer reader.Close() + rows := readAllRows(t, reader) + require.Len(t, rows, 0) + }() + + func() { + reader, err := NewHistoryReader(context.Background(), columns, "", timeLocation, nil, false, nil, []*StmtTimeRange{ + {Begin: 0, End: 0}, + }, 2) + require.NoError(t, err) + defer reader.Close() + rows := readAllRows(t, reader) + require.Len(t, rows, 4) + for _, row := range rows { + require.Equal(t, len(columns), len(row)) + } + }() +} + +func readAllRows(t *testing.T, reader *HistoryReader) [][]types.Datum { + var results [][]types.Datum + for { + rows, err := reader.Rows() + require.NoError(t, err) + if rows == nil { + break + } + results = append(results, rows...) + } + return results +} diff --git a/util/stmtsummary/v2/record.go b/util/stmtsummary/v2/record.go new file mode 100644 index 0000000000000..2bc92a8fd3217 --- /dev/null +++ b/util/stmtsummary/v2/record.go @@ -0,0 +1,661 @@ +// Copyright 2023 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 stmtsummary + +import ( + "bytes" + "fmt" + "math" + "strings" + "sync" + "sync/atomic" + "time" + + "github.com/pingcap/tidb/sessionctx/stmtctx" + "github.com/pingcap/tidb/util/execdetails" + "github.com/pingcap/tidb/util/plancodec" + "github.com/pingcap/tidb/util/stmtsummary" + "github.com/tikv/client-go/v2/util" +) + +// MaxEncodedPlanSizeInBytes is the upper limit of the size of the plan and the binary plan in the stmt summary. +var MaxEncodedPlanSizeInBytes = 1024 * 1024 + +// StmtRecord represents a statement statistics record. +// StmtRecord is addable and mergable. +type StmtRecord struct { + // Each record is summarized between [Begin, End). + Begin int64 `json:"begin"` + End int64 `json:"end"` + // Immutable + SchemaName string `json:"schema_name"` + Digest string `json:"digest"` + PlanDigest string `json:"plan_digest"` + StmtType string `json:"stmt_type"` + NormalizedSQL string `json:"normalized_sql"` + TableNames string `json:"table_names"` + IsInternal bool `json:"is_internal"` + // Basic + SampleSQL string `json:"sample_sql"` + Charset string `json:"charset"` + Collation string `json:"collation"` + PrevSQL string `json:"prev_sql"` + SamplePlan string `json:"sample_plan"` + SampleBinaryPlan string `json:"sample_binary_plan"` + PlanHint string `json:"plan_hint"` + IndexNames []string `json:"index_names"` + ExecCount int64 `json:"exec_count"` + SumErrors int `json:"sum_errors"` + SumWarnings int `json:"sum_warnings"` + // Latency + SumLatency time.Duration `json:"sum_latency"` + MaxLatency time.Duration `json:"max_latency"` + MinLatency time.Duration `json:"min_latency"` + SumParseLatency time.Duration `json:"sum_parse_latency"` + MaxParseLatency time.Duration `json:"max_parse_latency"` + SumCompileLatency time.Duration `json:"sum_compile_latency"` + MaxCompileLatency time.Duration `json:"max_compile_latency"` + // Coprocessor + SumNumCopTasks int64 `json:"sum_num_cop_tasks"` + MaxCopProcessTime time.Duration `json:"max_cop_process_time"` + MaxCopProcessAddress string `json:"max_cop_process_address"` + MaxCopWaitTime time.Duration `json:"max_cop_wait_time"` + MaxCopWaitAddress string `json:"max_cop_wait_address"` + // TiKV + SumProcessTime time.Duration `json:"sum_process_time"` + MaxProcessTime time.Duration `json:"max_process_time"` + SumWaitTime time.Duration `json:"sum_wait_time"` + MaxWaitTime time.Duration `json:"max_wait_time"` + SumBackoffTime time.Duration `json:"sum_backoff_time"` + MaxBackoffTime time.Duration `json:"max_backoff_time"` + SumTotalKeys int64 `json:"sum_total_keys"` + MaxTotalKeys int64 `json:"max_total_keys"` + SumProcessedKeys int64 `json:"sum_processed_keys"` + MaxProcessedKeys int64 `json:"max_processed_keys"` + SumRocksdbDeleteSkippedCount uint64 `json:"sum_rocksdb_delete_skipped_count"` + MaxRocksdbDeleteSkippedCount uint64 `json:"max_rocksdb_delete_skipped_count"` + SumRocksdbKeySkippedCount uint64 `json:"sum_rocksdb_key_skipped_count"` + MaxRocksdbKeySkippedCount uint64 `json:"max_rocksdb_key_skipped_count"` + SumRocksdbBlockCacheHitCount uint64 `json:"sum_rocksdb_block_cache_hit_count"` + MaxRocksdbBlockCacheHitCount uint64 `json:"max_rocksdb_block_cache_hit_count"` + SumRocksdbBlockReadCount uint64 `json:"sum_rocksdb_block_read_count"` + MaxRocksdbBlockReadCount uint64 `json:"max_rocksdb_block_read_count"` + SumRocksdbBlockReadByte uint64 `json:"sum_rocksdb_block_read_byte"` + MaxRocksdbBlockReadByte uint64 `json:"max_rocksdb_block_read_byte"` + // Txn + CommitCount int64 `json:"commit_count"` + SumGetCommitTsTime time.Duration `json:"sum_get_commit_ts_time"` + MaxGetCommitTsTime time.Duration `json:"max_get_commit_ts_time"` + SumPrewriteTime time.Duration `json:"sum_prewrite_time"` + MaxPrewriteTime time.Duration `json:"max_prewrite_time"` + SumCommitTime time.Duration `json:"sum_commit_time"` + MaxCommitTime time.Duration `json:"max_commit_time"` + SumLocalLatchTime time.Duration `json:"sum_local_latch_time"` + MaxLocalLatchTime time.Duration `json:"max_local_latch_time"` + SumCommitBackoffTime int64 `json:"sum_commit_backoff_time"` + MaxCommitBackoffTime int64 `json:"max_commit_backoff_time"` + SumResolveLockTime int64 `json:"sum_resolve_lock_time"` + MaxResolveLockTime int64 `json:"max_resolve_lock_time"` + SumWriteKeys int64 `json:"sum_write_keys"` + MaxWriteKeys int `json:"max_write_keys"` + SumWriteSize int64 `json:"sum_write_size"` + MaxWriteSize int `json:"max_write_size"` + SumPrewriteRegionNum int64 `json:"sum_prewrite_region_num"` + MaxPrewriteRegionNum int32 `json:"max_prewrite_region_num"` + SumTxnRetry int64 `json:"sum_txn_retry"` + MaxTxnRetry int `json:"max_txn_retry"` + SumBackoffTimes int64 `json:"sum_backoff_times"` + BackoffTypes map[string]int `json:"backoff_types"` + AuthUsers map[string]struct{} `json:"auth_users"` + // Other + SumMem int64 `json:"sum_mem"` + MaxMem int64 `json:"max_mem"` + SumDisk int64 `json:"sum_disk"` + MaxDisk int64 `json:"max_disk"` + SumAffectedRows uint64 `json:"sum_affected_rows"` + SumKVTotal time.Duration `json:"sum_kv_total"` + SumPDTotal time.Duration `json:"sum_pd_total"` + SumBackoffTotal time.Duration `json:"sum_backoff_total"` + SumWriteSQLRespTotal time.Duration `json:"sum_write_sql_resp_total"` + SumResultRows int64 `json:"sum_result_rows"` + MaxResultRows int64 `json:"max_result_rows"` + MinResultRows int64 `json:"min_result_rows"` + Prepared bool `json:"prepared"` + // The first time this type of SQL executes. + FirstSeen time.Time `json:"first_seen"` + // The last time this type of SQL executes. + LastSeen time.Time `json:"last_seen"` + // Plan cache + PlanInCache bool `json:"plan_in_cache"` + PlanCacheHits int64 `json:"plan_cache_hits"` + PlanInBinding bool `json:"plan_in_binding"` + // Pessimistic execution retry information. + ExecRetryCount uint `json:"exec_retry_count"` + ExecRetryTime time.Duration `json:"exec_retry_time"` +} + +// NewStmtRecord creates a new StmtRecord from StmtExecInfo. +// StmtExecInfo is only used to initialize the basic information +// of StmtRecord. Next we need to call StmtRecord.Add to add the +// statistics of the StmtExecInfo into the StmtRecord. +func NewStmtRecord(info *stmtsummary.StmtExecInfo) *StmtRecord { + // Use "," to separate table names to support FIND_IN_SET. + var buffer bytes.Buffer + for i, value := range info.StmtCtx.Tables { + // In `create database` statement, DB name is not empty but table name is empty. + if len(value.Table) == 0 { + continue + } + buffer.WriteString(strings.ToLower(value.DB)) + buffer.WriteString(".") + buffer.WriteString(strings.ToLower(value.Table)) + if i < len(info.StmtCtx.Tables)-1 { + buffer.WriteString(",") + } + } + tableNames := buffer.String() + planDigest := info.PlanDigest + if info.PlanDigestGen != nil && len(planDigest) == 0 { + // It comes here only when the plan is 'Point_Get'. + planDigest = info.PlanDigestGen() + } + // sampleSQL / authUsers(sampleUser) / samplePlan / prevSQL / indexNames store the values shown at the first time, + // because it compacts performance to update every time. + samplePlan, planHint := info.PlanGenerator() + if len(samplePlan) > MaxEncodedPlanSizeInBytes { + samplePlan = plancodec.PlanDiscardedEncoded + } + binPlan := "" + if info.BinaryPlanGenerator != nil { + binPlan = info.BinaryPlanGenerator() + if len(binPlan) > MaxEncodedPlanSizeInBytes { + binPlan = plancodec.BinaryPlanDiscardedEncoded + } + } + return &StmtRecord{ + SchemaName: info.SchemaName, + Digest: info.Digest, + PlanDigest: planDigest, + StmtType: info.StmtCtx.StmtType, + NormalizedSQL: info.NormalizedSQL, + TableNames: tableNames, + IsInternal: info.IsInternal, + SampleSQL: formatSQL(info.OriginalSQL), + Charset: info.Charset, + Collation: info.Collation, + // PrevSQL is already truncated to cfg.Log.QueryLogMaxLen. + PrevSQL: info.PrevSQL, + // SamplePlan needs to be decoded so it can't be truncated. + SamplePlan: samplePlan, + SampleBinaryPlan: binPlan, + PlanHint: planHint, + IndexNames: info.StmtCtx.IndexNames, + MinLatency: info.TotalLatency, + BackoffTypes: make(map[string]int), + AuthUsers: make(map[string]struct{}), + MinResultRows: math.MaxInt64, + Prepared: info.Prepared, + FirstSeen: info.StartTime, + LastSeen: info.StartTime, + } +} + +// Add adds the statistics of StmtExecInfo to StmtRecord. +func (r *StmtRecord) Add(info *stmtsummary.StmtExecInfo) { + r.IsInternal = r.IsInternal && info.IsInternal + // Add user to auth users set + if len(info.User) > 0 { + r.AuthUsers[info.User] = struct{}{} + } + r.ExecCount++ + if !info.Succeed { + r.SumErrors++ + } + r.SumWarnings += int(info.StmtCtx.WarningCount()) + // Latency + r.SumLatency += info.TotalLatency + if info.TotalLatency > r.MaxLatency { + r.MaxLatency = info.TotalLatency + } + if info.TotalLatency < r.MinLatency { + r.MinLatency = info.TotalLatency + } + r.SumParseLatency += info.ParseLatency + if info.ParseLatency > r.MaxParseLatency { + r.MaxParseLatency = info.ParseLatency + } + r.SumCompileLatency += info.CompileLatency + if info.CompileLatency > r.MaxCompileLatency { + r.MaxCompileLatency = info.CompileLatency + } + // Coprocessor + numCopTasks := int64(info.CopTasks.NumCopTasks) + r.SumNumCopTasks += numCopTasks + if info.CopTasks.MaxProcessTime > r.MaxCopProcessTime { + r.MaxCopProcessTime = info.CopTasks.MaxProcessTime + r.MaxCopProcessAddress = info.CopTasks.MaxProcessAddress + } + if info.CopTasks.MaxWaitTime > r.MaxCopWaitTime { + r.MaxCopWaitTime = info.CopTasks.MaxWaitTime + r.MaxCopWaitAddress = info.CopTasks.MaxWaitAddress + } + // TiKV + r.SumProcessTime += info.ExecDetail.TimeDetail.ProcessTime + if info.ExecDetail.TimeDetail.ProcessTime > r.MaxProcessTime { + r.MaxProcessTime = info.ExecDetail.TimeDetail.ProcessTime + } + r.SumWaitTime += info.ExecDetail.TimeDetail.WaitTime + if info.ExecDetail.TimeDetail.WaitTime > r.MaxWaitTime { + r.MaxWaitTime = info.ExecDetail.TimeDetail.WaitTime + } + r.SumBackoffTime += info.ExecDetail.BackoffTime + if info.ExecDetail.BackoffTime > r.MaxBackoffTime { + r.MaxBackoffTime = info.ExecDetail.BackoffTime + } + if info.ExecDetail.ScanDetail != nil { + r.SumTotalKeys += info.ExecDetail.ScanDetail.TotalKeys + if info.ExecDetail.ScanDetail.TotalKeys > r.MaxTotalKeys { + r.MaxTotalKeys = info.ExecDetail.ScanDetail.TotalKeys + } + r.SumProcessedKeys += info.ExecDetail.ScanDetail.ProcessedKeys + if info.ExecDetail.ScanDetail.ProcessedKeys > r.MaxProcessedKeys { + r.MaxProcessedKeys = info.ExecDetail.ScanDetail.ProcessedKeys + } + r.SumRocksdbDeleteSkippedCount += info.ExecDetail.ScanDetail.RocksdbDeleteSkippedCount + if info.ExecDetail.ScanDetail.RocksdbDeleteSkippedCount > r.MaxRocksdbDeleteSkippedCount { + r.MaxRocksdbDeleteSkippedCount = info.ExecDetail.ScanDetail.RocksdbDeleteSkippedCount + } + r.SumRocksdbKeySkippedCount += info.ExecDetail.ScanDetail.RocksdbKeySkippedCount + if info.ExecDetail.ScanDetail.RocksdbKeySkippedCount > r.MaxRocksdbKeySkippedCount { + r.MaxRocksdbKeySkippedCount = info.ExecDetail.ScanDetail.RocksdbKeySkippedCount + } + r.SumRocksdbBlockCacheHitCount += info.ExecDetail.ScanDetail.RocksdbBlockCacheHitCount + if info.ExecDetail.ScanDetail.RocksdbBlockCacheHitCount > r.MaxRocksdbBlockCacheHitCount { + r.MaxRocksdbBlockCacheHitCount = info.ExecDetail.ScanDetail.RocksdbBlockCacheHitCount + } + r.SumRocksdbBlockReadCount += info.ExecDetail.ScanDetail.RocksdbBlockReadCount + if info.ExecDetail.ScanDetail.RocksdbBlockReadCount > r.MaxRocksdbBlockReadCount { + r.MaxRocksdbBlockReadCount = info.ExecDetail.ScanDetail.RocksdbBlockReadCount + } + r.SumRocksdbBlockReadByte += info.ExecDetail.ScanDetail.RocksdbBlockReadByte + if info.ExecDetail.ScanDetail.RocksdbBlockReadByte > r.MaxRocksdbBlockReadByte { + r.MaxRocksdbBlockReadByte = info.ExecDetail.ScanDetail.RocksdbBlockReadByte + } + } + // Txn + commitDetails := info.ExecDetail.CommitDetail + if commitDetails != nil { + r.CommitCount++ + r.SumPrewriteTime += commitDetails.PrewriteTime + if commitDetails.PrewriteTime > r.MaxPrewriteTime { + r.MaxPrewriteTime = commitDetails.PrewriteTime + } + r.SumCommitTime += commitDetails.CommitTime + if commitDetails.CommitTime > r.MaxCommitTime { + r.MaxCommitTime = commitDetails.CommitTime + } + r.SumGetCommitTsTime += commitDetails.GetCommitTsTime + if commitDetails.GetCommitTsTime > r.MaxGetCommitTsTime { + r.MaxGetCommitTsTime = commitDetails.GetCommitTsTime + } + resolveLockTime := atomic.LoadInt64(&commitDetails.ResolveLock.ResolveLockTime) + r.SumResolveLockTime += resolveLockTime + if resolveLockTime > r.MaxResolveLockTime { + r.MaxResolveLockTime = resolveLockTime + } + r.SumLocalLatchTime += commitDetails.LocalLatchTime + if commitDetails.LocalLatchTime > r.MaxLocalLatchTime { + r.MaxLocalLatchTime = commitDetails.LocalLatchTime + } + r.SumWriteKeys += int64(commitDetails.WriteKeys) + if commitDetails.WriteKeys > r.MaxWriteKeys { + r.MaxWriteKeys = commitDetails.WriteKeys + } + r.SumWriteSize += int64(commitDetails.WriteSize) + if commitDetails.WriteSize > r.MaxWriteSize { + r.MaxWriteSize = commitDetails.WriteSize + } + prewriteRegionNum := atomic.LoadInt32(&commitDetails.PrewriteRegionNum) + r.SumPrewriteRegionNum += int64(prewriteRegionNum) + if prewriteRegionNum > r.MaxPrewriteRegionNum { + r.MaxPrewriteRegionNum = prewriteRegionNum + } + r.SumTxnRetry += int64(commitDetails.TxnRetry) + if commitDetails.TxnRetry > r.MaxTxnRetry { + r.MaxTxnRetry = commitDetails.TxnRetry + } + commitDetails.Mu.Lock() + commitBackoffTime := commitDetails.Mu.CommitBackoffTime + r.SumCommitBackoffTime += commitBackoffTime + if commitBackoffTime > r.MaxCommitBackoffTime { + r.MaxCommitBackoffTime = commitBackoffTime + } + r.SumBackoffTimes += int64(len(commitDetails.Mu.PrewriteBackoffTypes)) + for _, backoffType := range commitDetails.Mu.PrewriteBackoffTypes { + r.BackoffTypes[backoffType]++ + } + r.SumBackoffTimes += int64(len(commitDetails.Mu.CommitBackoffTypes)) + for _, backoffType := range commitDetails.Mu.CommitBackoffTypes { + r.BackoffTypes[backoffType]++ + } + commitDetails.Mu.Unlock() + } + // Plan cache + if info.PlanInCache { + r.PlanInCache = true + r.PlanCacheHits++ + } else { + r.PlanInCache = false + } + // SPM + if info.PlanInBinding { + r.PlanInBinding = true + } else { + r.PlanInBinding = false + } + // Other + r.SumAffectedRows += info.StmtCtx.AffectedRows() + r.SumMem += info.MemMax + if info.MemMax > r.MaxMem { + r.MaxMem = info.MemMax + } + r.SumDisk += info.DiskMax + if info.DiskMax > r.MaxDisk { + r.MaxDisk = info.DiskMax + } + if info.StartTime.Before(r.FirstSeen) { + r.FirstSeen = info.StartTime + } + if r.LastSeen.Before(info.StartTime) { + r.LastSeen = info.StartTime + } + if info.ExecRetryCount > 0 { + r.ExecRetryCount += info.ExecRetryCount + r.ExecRetryTime += info.ExecRetryTime + } + if info.ResultRows > 0 { + r.SumResultRows += info.ResultRows + if r.MaxResultRows < info.ResultRows { + r.MaxResultRows = info.ResultRows + } + if r.MinResultRows > info.ResultRows { + r.MinResultRows = info.ResultRows + } + } else { + r.MinResultRows = 0 + } + r.SumKVTotal += time.Duration(atomic.LoadInt64(&info.TiKVExecDetails.WaitKVRespDuration)) + r.SumPDTotal += time.Duration(atomic.LoadInt64(&info.TiKVExecDetails.WaitPDRespDuration)) + r.SumBackoffTotal += time.Duration(atomic.LoadInt64(&info.TiKVExecDetails.BackoffDuration)) + r.SumWriteSQLRespTotal += info.StmtExecDetails.WriteSQLRespDuration +} + +// Merge merges the statistics of another StmtRecord to this StmtRecord. +func (r *StmtRecord) Merge(other *StmtRecord) { + // User + for user := range other.AuthUsers { + r.AuthUsers[user] = struct{}{} + } + // ExecCount and SumWarnings + r.ExecCount += other.ExecCount + r.SumWarnings += other.SumWarnings + // Latency + r.SumLatency += other.SumLatency + if r.MaxLatency < other.MaxLatency { + r.MaxLatency = other.MaxLatency + } + if r.MinLatency > other.MinLatency { + r.MinLatency = other.MinLatency + } + r.SumParseLatency += other.SumParseLatency + if r.MaxParseLatency < other.MaxParseLatency { + r.MaxParseLatency = other.MaxParseLatency + } + r.SumCompileLatency += other.SumCompileLatency + if r.MaxCompileLatency < other.MaxCompileLatency { + r.MaxCompileLatency = other.MaxCompileLatency + } + // Coprocessor + r.SumNumCopTasks += other.SumNumCopTasks + if r.MaxCopProcessTime < other.MaxCopProcessTime { + r.MaxCopProcessTime = other.MaxCopProcessTime + r.MaxCopProcessAddress = other.MaxCopProcessAddress + } + if r.MaxCopWaitTime < other.MaxCopWaitTime { + r.MaxCopWaitTime = other.MaxCopWaitTime + r.MaxCopWaitAddress = other.MaxCopWaitAddress + } + // TiKV + r.SumProcessTime += other.SumProcessTime + if r.MaxProcessTime < other.MaxProcessTime { + r.MaxProcessTime = other.MaxProcessTime + } + r.SumWaitTime += other.SumWaitTime + if r.MaxWaitTime < other.MaxWaitTime { + r.MaxWaitTime = other.MaxWaitTime + } + r.SumBackoffTime += other.SumBackoffTime + if r.MaxBackoffTime < other.MaxBackoffTime { + r.MaxBackoffTime = other.MaxBackoffTime + } + r.SumTotalKeys += other.SumTotalKeys + if r.MaxTotalKeys < other.MaxTotalKeys { + r.MaxTotalKeys = other.MaxTotalKeys + } + r.SumProcessedKeys += other.SumProcessedKeys + if r.MaxProcessedKeys < other.MaxProcessedKeys { + r.MaxProcessedKeys = other.MaxProcessedKeys + } + r.SumRocksdbDeleteSkippedCount += other.SumRocksdbDeleteSkippedCount + if r.MaxRocksdbDeleteSkippedCount < other.MaxRocksdbDeleteSkippedCount { + r.MaxRocksdbDeleteSkippedCount = other.MaxRocksdbDeleteSkippedCount + } + r.SumRocksdbKeySkippedCount += other.SumRocksdbKeySkippedCount + if r.MaxRocksdbKeySkippedCount < other.MaxRocksdbKeySkippedCount { + r.MaxRocksdbKeySkippedCount = other.MaxRocksdbKeySkippedCount + } + r.SumRocksdbBlockCacheHitCount += other.SumRocksdbBlockCacheHitCount + if r.MaxRocksdbBlockCacheHitCount < other.MaxRocksdbBlockCacheHitCount { + r.MaxRocksdbBlockCacheHitCount = other.MaxRocksdbBlockCacheHitCount + } + r.SumRocksdbBlockReadCount += other.SumRocksdbBlockReadCount + if r.MaxRocksdbBlockReadCount < other.MaxRocksdbBlockReadCount { + r.MaxRocksdbBlockReadCount = other.MaxRocksdbBlockReadCount + } + r.SumRocksdbBlockReadByte += other.SumRocksdbBlockReadByte + if r.MaxRocksdbBlockReadByte < other.MaxRocksdbBlockReadByte { + r.MaxRocksdbBlockReadByte = other.MaxRocksdbBlockReadByte + } + // Txn + r.CommitCount += other.CommitCount + r.SumPrewriteTime += other.SumPrewriteTime + if r.MaxPrewriteTime < other.MaxPrewriteTime { + r.MaxPrewriteTime = other.MaxPrewriteTime + } + r.SumCommitTime += other.SumCommitTime + if r.MaxCommitTime < other.MaxCommitTime { + r.MaxCommitTime = other.MaxCommitTime + } + r.SumGetCommitTsTime += other.SumGetCommitTsTime + if r.MaxGetCommitTsTime < other.MaxGetCommitTsTime { + r.MaxGetCommitTsTime = other.MaxGetCommitTsTime + } + r.SumCommitBackoffTime += other.SumCommitBackoffTime + if r.MaxCommitBackoffTime < other.MaxCommitBackoffTime { + r.MaxCommitBackoffTime = other.MaxCommitBackoffTime + } + r.SumResolveLockTime += other.SumResolveLockTime + if r.MaxResolveLockTime < other.MaxResolveLockTime { + r.MaxResolveLockTime = other.MaxResolveLockTime + } + r.SumLocalLatchTime += other.SumLocalLatchTime + if r.MaxLocalLatchTime < other.MaxLocalLatchTime { + r.MaxLocalLatchTime = other.MaxLocalLatchTime + } + r.SumWriteKeys += other.SumWriteKeys + if r.MaxWriteKeys < other.MaxWriteKeys { + r.MaxWriteKeys = other.MaxWriteKeys + } + r.SumWriteSize += other.SumWriteSize + if r.MaxWriteSize < other.MaxWriteSize { + r.MaxWriteSize = other.MaxWriteSize + } + r.SumPrewriteRegionNum += other.SumPrewriteRegionNum + if r.MaxPrewriteRegionNum < other.MaxPrewriteRegionNum { + r.MaxPrewriteRegionNum = other.MaxPrewriteRegionNum + } + r.SumTxnRetry += other.SumTxnRetry + if r.MaxTxnRetry < other.MaxTxnRetry { + r.MaxTxnRetry = other.MaxTxnRetry + } + r.SumBackoffTimes += other.SumBackoffTimes + for backoffType, backoffValue := range other.BackoffTypes { + _, ok := r.BackoffTypes[backoffType] + if ok { + r.BackoffTypes[backoffType] += backoffValue + } else { + r.BackoffTypes[backoffType] = backoffValue + } + } + // Plan cache + r.PlanCacheHits += other.PlanCacheHits + // Other + r.SumAffectedRows += other.SumAffectedRows + r.SumMem += other.SumMem + if r.MaxMem < other.MaxMem { + r.MaxMem = other.MaxMem + } + r.SumDisk += other.SumDisk + if r.MaxDisk < other.MaxDisk { + r.MaxDisk = other.MaxDisk + } + if r.FirstSeen.After(other.FirstSeen) { + r.FirstSeen = other.FirstSeen + } + if r.LastSeen.Before(other.LastSeen) { + r.LastSeen = other.LastSeen + } + r.ExecRetryCount += other.ExecRetryCount + r.ExecRetryTime += other.ExecRetryTime + r.SumKVTotal += other.SumKVTotal + r.SumPDTotal += other.SumPDTotal + r.SumBackoffTotal += other.SumBackoffTotal + r.SumWriteSQLRespTotal += other.SumWriteSQLRespTotal + r.SumErrors += other.SumErrors +} + +// Truncate SQL to maxSQLLength. +func formatSQL(sql string) string { + maxSQLLength := int(maxSQLLength()) + length := len(sql) + if length > maxSQLLength { + sql = fmt.Sprintf("%.*s(len:%d)", maxSQLLength, sql, length) + } + return sql +} + +func maxSQLLength() uint32 { + if GlobalStmtSummary != nil { + return GlobalStmtSummary.MaxSQLLength() + } + return 4096 +} + +// GenerateStmtExecInfo4Test generates a new StmtExecInfo for testing purposes. +func GenerateStmtExecInfo4Test(digest string) *stmtsummary.StmtExecInfo { + tables := []stmtctx.TableEntry{{DB: "db1", Table: "tb1"}, {DB: "db2", Table: "tb2"}} + indexes := []string{"a"} + stmtExecInfo := &stmtsummary.StmtExecInfo{ + SchemaName: "schema_name", + OriginalSQL: "original_sql1", + NormalizedSQL: "normalized_sql", + Digest: digest, + PlanDigest: "plan_digest", + PlanGenerator: func() (string, string) { return "", "" }, + User: "user", + TotalLatency: 10000, + ParseLatency: 100, + CompileLatency: 1000, + CopTasks: &stmtctx.CopTasksDetails{ + NumCopTasks: 10, + AvgProcessTime: 1000, + P90ProcessTime: 10000, + MaxProcessAddress: "127", + MaxProcessTime: 15000, + AvgWaitTime: 100, + P90WaitTime: 1000, + MaxWaitAddress: "128", + MaxWaitTime: 1500, + }, + ExecDetail: &execdetails.ExecDetails{ + BackoffTime: 80, + RequestCount: 10, + CommitDetail: &util.CommitDetails{ + GetCommitTsTime: 100, + PrewriteTime: 10000, + CommitTime: 1000, + LocalLatchTime: 10, + Mu: struct { + sync.Mutex + CommitBackoffTime int64 + PrewriteBackoffTypes []string + CommitBackoffTypes []string + SlowestPrewrite util.ReqDetailInfo + CommitPrimary util.ReqDetailInfo + }{ + CommitBackoffTime: 200, + PrewriteBackoffTypes: []string{"txnlock"}, + CommitBackoffTypes: []string{}, + SlowestPrewrite: util.ReqDetailInfo{}, + CommitPrimary: util.ReqDetailInfo{}, + }, + WriteKeys: 20000, + WriteSize: 200000, + PrewriteRegionNum: 20, + TxnRetry: 2, + ResolveLock: util.ResolveLockDetail{ + ResolveLockTime: 2000, + }, + }, + ScanDetail: &util.ScanDetail{ + TotalKeys: 1000, + ProcessedKeys: 500, + RocksdbDeleteSkippedCount: 100, + RocksdbKeySkippedCount: 10, + RocksdbBlockCacheHitCount: 10, + RocksdbBlockReadCount: 10, + RocksdbBlockReadByte: 1000, + }, + DetailsNeedP90: execdetails.DetailsNeedP90{ + TimeDetail: util.TimeDetail{ + ProcessTime: 500, + WaitTime: 50, + }, + CalleeAddress: "129", + }, + }, + StmtCtx: &stmtctx.StatementContext{ + StmtType: "Select", + Tables: tables, + IndexNames: indexes, + }, + MemMax: 10000, + DiskMax: 10000, + StartTime: time.Date(2019, 1, 1, 10, 10, 10, 10, time.UTC), + Succeed: true, + } + stmtExecInfo.StmtCtx.AddAffectedRows(10000) + return stmtExecInfo +} diff --git a/util/stmtsummary/v2/record_test.go b/util/stmtsummary/v2/record_test.go new file mode 100644 index 0000000000000..dd3064daba732 --- /dev/null +++ b/util/stmtsummary/v2/record_test.go @@ -0,0 +1,64 @@ +// Copyright 2023 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 stmtsummary + +import ( + "testing" + + "github.com/stretchr/testify/require" +) + +func TestStmtRecord(t *testing.T) { + info := GenerateStmtExecInfo4Test("digest1") + record1 := NewStmtRecord(info) + require.Equal(t, info.SchemaName, record1.SchemaName) + require.Equal(t, info.Digest, record1.Digest) + require.Equal(t, info.PlanDigest, record1.PlanDigest) + require.Equal(t, info.StmtCtx.StmtType, record1.StmtType) + require.Equal(t, info.NormalizedSQL, record1.NormalizedSQL) + require.Equal(t, "db1.tb1,db2.tb2", record1.TableNames) + require.Equal(t, info.IsInternal, record1.IsInternal) + require.Equal(t, formatSQL(info.OriginalSQL), record1.SampleSQL) + require.Equal(t, info.Charset, record1.Charset) + require.Equal(t, info.Collation, record1.Collation) + require.Equal(t, info.PrevSQL, record1.PrevSQL) + require.Equal(t, info.StmtCtx.IndexNames, record1.IndexNames) + require.Equal(t, info.TotalLatency, record1.MinLatency) + require.Equal(t, info.Prepared, record1.Prepared) + require.Equal(t, info.StartTime, record1.FirstSeen) + require.Equal(t, info.StartTime, record1.LastSeen) + require.Empty(t, record1.AuthUsers) + require.Zero(t, record1.ExecCount) + require.Zero(t, record1.SumLatency) + require.Zero(t, record1.MaxLatency) + + record1.Add(info) + require.Len(t, record1.AuthUsers, 1) + require.Contains(t, record1.AuthUsers, "user") + require.Equal(t, int64(1), record1.ExecCount) + require.Equal(t, info.TotalLatency, record1.SumLatency) + require.Equal(t, info.TotalLatency, record1.MaxLatency) + require.Equal(t, info.TotalLatency, record1.MinLatency) + + record2 := NewStmtRecord(info) + record2.Add(info) + record2.Merge(record1) + require.Len(t, record2.AuthUsers, 1) + require.Contains(t, record2.AuthUsers, "user") + require.Equal(t, int64(2), record2.ExecCount) + require.Equal(t, info.TotalLatency*2, record2.SumLatency) + require.Equal(t, info.TotalLatency, record2.MaxLatency) + require.Equal(t, info.TotalLatency, record2.MinLatency) +} diff --git a/util/stmtsummary/v2/stmtsummary.go b/util/stmtsummary/v2/stmtsummary.go new file mode 100644 index 0000000000000..d3a8c21c44751 --- /dev/null +++ b/util/stmtsummary/v2/stmtsummary.go @@ -0,0 +1,580 @@ +// Copyright 2023 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 stmtsummary + +import ( + "context" + "errors" + "math" + "sync" + "sync/atomic" + "time" + + "github.com/pingcap/log" + "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/hack" + "github.com/pingcap/tidb/util/kvcache" + "github.com/pingcap/tidb/util/stmtsummary" + atomic2 "go.uber.org/atomic" + "golang.org/x/exp/maps" +) + +const ( + defaultEnabled = true + defaultEnableInternalQuery = false + defaultMaxStmtCount = 3000 + defaultMaxSQLLength = 4096 + defaultRefreshInterval = 30 * 60 // 30 min + defaultRotateCheckInterval = 1 // s +) + +var ( + // GlobalStmtSummary is the global StmtSummary instance, we need + // to explicitly call Setup() to initialize it. It will then be + // referenced by SessionVars.StmtSummary for each session. + GlobalStmtSummary *StmtSummary + + timeNow = time.Now +) + +// Setup initializes the GlobalStmtSummary. +func Setup(cfg *Config) (err error) { + GlobalStmtSummary, err = NewStmtSummary(cfg) + return +} + +// Config is the static configuration of StmtSummary. It cannot be +// modified at runtime. +type Config struct { + Filename string + FileMaxSize int + FileMaxDays int + FileMaxBackups int +} + +// StmtSummary represents the complete statements summary statistics. +// It controls data rotation and persistence internally, and provides +// reading interface through MemReader and HistoryReader. +type StmtSummary struct { + ctx context.Context + cancel context.CancelFunc + + optEnabled *atomic2.Bool + optEnableInternalQuery *atomic2.Bool + optMaxStmtCount *atomic2.Uint32 + optMaxSQLLength *atomic2.Uint32 + optRefreshInterval *atomic2.Uint32 + + window *stmtWindow + windowLock sync.Mutex + storage stmtStorage + closeWg sync.WaitGroup + closed atomic.Bool +} + +// NewStmtSummary creates a new StmtSummary from Config. +func NewStmtSummary(cfg *Config) (*StmtSummary, error) { + if cfg.Filename == "" { + return nil, errors.New("stmtsummary: empty filename") + } + + ctx, cancel := context.WithCancel(context.Background()) + s := &StmtSummary{ + ctx: ctx, + cancel: cancel, + + // These options can be changed dynamically at runtime. + // The default values here are just placeholders, and the real values in + // sessionctx/variables/tidb_vars.go will overwrite them after TiDB starts. + optEnabled: atomic2.NewBool(defaultEnabled), + optEnableInternalQuery: atomic2.NewBool(defaultEnableInternalQuery), + optMaxStmtCount: atomic2.NewUint32(defaultMaxStmtCount), + optMaxSQLLength: atomic2.NewUint32(defaultMaxSQLLength), + optRefreshInterval: atomic2.NewUint32(defaultRefreshInterval), + window: newStmtWindow(timeNow(), uint(defaultMaxStmtCount)), + storage: newStmtLogStorage(&log.Config{ + File: log.FileLogConfig{ + Filename: cfg.Filename, + MaxSize: cfg.FileMaxSize, + MaxDays: cfg.FileMaxDays, + MaxBackups: cfg.FileMaxBackups, + }, + }), + } + + s.closeWg.Add(1) + go func() { + defer s.closeWg.Done() + s.rotateLoop() + }() + + return s, nil +} + +// NewStmtSummary4Test creates a new StmtSummary for testing purposes. +func NewStmtSummary4Test(maxStmtCount uint) *StmtSummary { + ctx, cancel := context.WithCancel(context.Background()) + + ss := &StmtSummary{ + ctx: ctx, + cancel: cancel, + + optEnabled: atomic2.NewBool(defaultEnabled), + optEnableInternalQuery: atomic2.NewBool(defaultEnableInternalQuery), + optMaxStmtCount: atomic2.NewUint32(defaultMaxStmtCount), + optMaxSQLLength: atomic2.NewUint32(defaultMaxSQLLength), + optRefreshInterval: atomic2.NewUint32(60 * 60 * 24 * 365), // 1 year + window: newStmtWindow(timeNow(), maxStmtCount), + storage: &mockStmtStorage{}, + } + + ss.closeWg.Add(1) + go func() { + defer ss.closeWg.Done() + ss.rotateLoop() + }() + + return ss +} + +// Enabled returns whether the StmtSummary is enabled. +func (s *StmtSummary) Enabled() bool { + return s.optEnabled.Load() +} + +// SetEnabled is used to enable or disable StmtSummary. If disabled, in-memory +// data will be cleared, (persisted data will still be remained). +func (s *StmtSummary) SetEnabled(v bool) error { + s.optEnabled.Store(v) + if !v { + s.Clear() + } + + return nil +} + +// EnableInternalQuery returns whether the StmtSummary counts internal queries. +func (s *StmtSummary) EnableInternalQuery() bool { + return s.optEnableInternalQuery.Load() +} + +// SetEnableInternalQuery is used to enable or disable StmtSummary's internal +// query statistics. If disabled, in-memory internal queries will be cleared, +// (persisted internal queries will still be remained). +func (s *StmtSummary) SetEnableInternalQuery(v bool) error { + s.optEnableInternalQuery.Store(v) + if !v { + s.ClearInternal() + } + + return nil +} + +// MaxStmtCount returns the maximum number of statements. +func (s *StmtSummary) MaxStmtCount() uint32 { + return s.optMaxStmtCount.Load() +} + +// SetMaxStmtCount is used to set the maximum number of statements. +// If the current number exceeds the maximum number, the excess will be evicted. +func (s *StmtSummary) SetMaxStmtCount(v uint32) error { + if v < 1 { + v = 1 + } + s.optMaxStmtCount.Store(v) + s.windowLock.Lock() + _ = s.window.lru.SetCapacity(uint(v)) + s.windowLock.Unlock() + + return nil +} + +// MaxSQLLength returns the maximum size of a single SQL statement. +func (s *StmtSummary) MaxSQLLength() uint32 { + return s.optMaxSQLLength.Load() +} + +// SetMaxSQLLength sets the maximum size of a single SQL statement. +func (s *StmtSummary) SetMaxSQLLength(v uint32) error { + s.optMaxSQLLength.Store(v) + + return nil +} + +// RefreshInterval returns the period (in seconds) at which the statistics +// window is refreshed (persisted). +func (s *StmtSummary) RefreshInterval() uint32 { + return s.optRefreshInterval.Load() +} + +// SetRefreshInterval sets the period (in seconds) for the statistics window +// to be refreshed (persisted). This may trigger a refresh (persistence) of +// the current statistics window early. +func (s *StmtSummary) SetRefreshInterval(v uint32) error { + if v < 1 { + v = 1 + } + s.optRefreshInterval.Store(v) + + return nil +} + +// Add adds a single stmtsummary.StmtExecInfo to the current statistics window +// of StmtSummary. Before adding, it will check whether the current window has +// expired, and if it has expired, the window will be persisted asynchronously +// and a new window will be created to replace the current one. +func (s *StmtSummary) Add(info *stmtsummary.StmtExecInfo) { + if s.closed.Load() { + return + } + + k := &stmtKey{ + schemaName: info.SchemaName, + digest: info.Digest, + prevDigest: info.PrevSQLDigest, + planDigest: info.PlanDigest, + } + k.Hash() // Calculate hash value in advance, to reduce the time holding the window lock. + + // Add info to the current statistics window. + s.windowLock.Lock() + var record *lockedStmtRecord + if v, ok := s.window.lru.Get(k); ok { + record = v.(*lockedStmtRecord) + } else { + record = &lockedStmtRecord{StmtRecord: NewStmtRecord(info)} + s.window.lru.Put(k, record) + } + s.windowLock.Unlock() + + record.Lock() + record.Add(info) + record.Unlock() +} + +// Evicted returns the number of statements evicted for the current +// time window. The returned type is one row consisting of three +// columns: [BEGIN_TIME, END_TIME, EVICTED_COUNT]. +func (s *StmtSummary) Evicted() []types.Datum { + s.windowLock.Lock() + count := int64(s.window.evicted.count()) + s.windowLock.Unlock() + if count == 0 { + return nil + } + begin := types.NewTime(types.FromGoTime(s.window.begin), mysql.TypeTimestamp, 0) + end := types.NewTime(types.FromGoTime(timeNow()), mysql.TypeTimestamp, 0) + return types.MakeDatums(begin, end, count) +} + +// Clear clears all data in the current window, and the data that +// has been persisted will not be cleared. +func (s *StmtSummary) Clear() { + s.windowLock.Lock() + defer s.windowLock.Unlock() + s.window.clear() +} + +// ClearInternal clears all internal queries of the current window, +// and the data that has been persisted will not be cleared. +func (s *StmtSummary) ClearInternal() { + s.windowLock.Lock() + defer s.windowLock.Unlock() + for _, k := range s.window.lru.Keys() { + v, _ := s.window.lru.Get(k) + if v.(*lockedStmtRecord).IsInternal { + s.window.lru.Delete(k) + } + } +} + +// Close closes the work of StmtSummary. +func (s *StmtSummary) Close() { + if s.cancel != nil { + s.cancel() + s.closeWg.Wait() + } + s.closed.Store(true) +} + +// GetMoreThanCntBindableStmt is used to get bindable statements. +// Statements whose execution times exceed the threshold will be +// returned. Since the historical data has been persisted, we only +// refer to the statistics data of the current window in memory. +func (s *StmtSummary) GetMoreThanCntBindableStmt(cnt int64) []*stmtsummary.BindableStmt { + s.windowLock.Lock() + values := s.window.lru.Values() + s.windowLock.Unlock() + stmts := make([]*stmtsummary.BindableStmt, 0, len(values)) + for _, value := range values { + record := value.(*lockedStmtRecord) + func() { + record.Lock() + defer record.Unlock() + if record.StmtType == "Select" || + record.StmtType == "Delete" || + record.StmtType == "Update" || + record.StmtType == "Insert" || + record.StmtType == "Replace" { + if len(record.AuthUsers) > 0 && record.ExecCount > cnt { + stmt := &stmtsummary.BindableStmt{ + Schema: record.SchemaName, + Query: record.SampleSQL, + PlanHint: record.PlanHint, + Charset: record.Charset, + Collation: record.Collation, + Users: make(map[string]struct{}), + } + maps.Copy(stmt.Users, record.AuthUsers) + + // If it is SQL command prepare / execute, the ssElement.sampleSQL + // is `execute ...`, we should get the original select query. + // If it is binary protocol prepare / execute, ssbd.normalizedSQL + // should be same as ssElement.sampleSQL. + if record.Prepared { + stmt.Query = record.NormalizedSQL + } + stmts = append(stmts, stmt) + } + } + }() + } + return stmts +} + +func (s *StmtSummary) rotateLoop() { + tick := time.NewTicker(defaultRotateCheckInterval * time.Second) + defer tick.Stop() + + for { + select { + case <-s.ctx.Done(): + return + case <-tick.C: + now := timeNow() + s.windowLock.Lock() + w := s.window + // The current window has expired and needs to be refreshed and persisted. + if now.After(w.begin.Add(time.Duration(s.RefreshInterval()) * time.Second)) { + s.window = newStmtWindow(now, uint(s.MaxStmtCount())) + size := w.lru.Size() + if size > 0 { + // Persist window asynchronously. + s.closeWg.Add(1) + go func() { + defer s.closeWg.Done() + s.storage.persist(w, now) + }() + } + } + s.windowLock.Unlock() + } + } +} + +// stmtWindow represents a single statistical window, which has a begin +// time and an end time. Data within a single window is eliminated +// according to the LRU strategy. All evicted data will be aggregated +// into stmtEvicted. +type stmtWindow struct { + begin time.Time + lru *kvcache.SimpleLRUCache // *stmtKey => *lockedStmtRecord + evicted *stmtEvicted +} + +func newStmtWindow(begin time.Time, capacity uint) *stmtWindow { + w := &stmtWindow{ + begin: begin, + lru: kvcache.NewSimpleLRUCache(capacity, 0, 0), + evicted: newStmtEvicted(), + } + w.lru.SetOnEvict(func(k kvcache.Key, v kvcache.Value) { + r := v.(*lockedStmtRecord) + r.Lock() + defer r.Unlock() + w.evicted.add(k.(*stmtKey), r.StmtRecord) + }) + return w +} + +func (w *stmtWindow) clear() { + w.lru.DeleteAll() + w.evicted = newStmtEvicted() +} + +type stmtStorage interface { + persist(w *stmtWindow, end time.Time) +} + +// stmtKey defines key for stmtElement. +type stmtKey struct { + // Same statements may appear in different schema, but they refer to different tables. + schemaName string + digest string + // The digest of the previous statement. + prevDigest string + // The digest of the plan of this SQL. + planDigest string + // `hash` is the hash value of this object. + hash []byte +} + +// Hash implements SimpleLRUCache.Key. +// Only when current SQL is `commit` do we record `prevSQL`. Otherwise, `prevSQL` is empty. +// `prevSQL` is included in the key To distinguish different transactions. +func (k *stmtKey) Hash() []byte { + if len(k.hash) == 0 { + k.hash = make([]byte, 0, len(k.schemaName)+len(k.digest)+len(k.prevDigest)+len(k.planDigest)) + k.hash = append(k.hash, hack.Slice(k.digest)...) + k.hash = append(k.hash, hack.Slice(k.schemaName)...) + k.hash = append(k.hash, hack.Slice(k.prevDigest)...) + k.hash = append(k.hash, hack.Slice(k.planDigest)...) + } + return k.hash +} + +type stmtEvicted struct { + sync.Mutex + keys map[string]struct{} + other *StmtRecord +} + +func newStmtEvicted() *stmtEvicted { + return &stmtEvicted{ + keys: make(map[string]struct{}), + other: &StmtRecord{ + AuthUsers: make(map[string]struct{}), + MinLatency: time.Duration(math.MaxInt64), + BackoffTypes: make(map[string]int), + FirstSeen: time.Unix(math.MaxInt64, 0), + }, + } +} + +func (e *stmtEvicted) add(key *stmtKey, record *StmtRecord) { + if key == nil || record == nil { + return + } + e.Lock() + defer e.Unlock() + e.keys[string(key.Hash())] = struct{}{} + e.other.Merge(record) +} + +func (e *stmtEvicted) count() int { + e.Lock() + defer e.Unlock() + return len(e.keys) +} + +type lockedStmtRecord struct { + sync.Mutex + *StmtRecord +} + +type mockStmtStorage struct { + windows []*stmtWindow +} + +func (s *mockStmtStorage) persist(w *stmtWindow, _ time.Time) { + s.windows = append(s.windows, w) +} + +/* Public proxy functions between v1 and v2 */ + +// Add wraps GlobalStmtSummary.Add and stmtsummary.StmtSummaryByDigestMap.AddStatement. +func Add(stmtExecInfo *stmtsummary.StmtExecInfo) { + if config.GetGlobalConfig().Instance.StmtSummaryEnablePersistent { + GlobalStmtSummary.Add(stmtExecInfo) + } else { + stmtsummary.StmtSummaryByDigestMap.AddStatement(stmtExecInfo) + } +} + +// Enabled wraps GlobalStmtSummary.Enabled and stmtsummary.StmtSummaryByDigestMap.Enabled. +func Enabled() bool { + if config.GetGlobalConfig().Instance.StmtSummaryEnablePersistent { + return GlobalStmtSummary.Enabled() + } + return stmtsummary.StmtSummaryByDigestMap.Enabled() +} + +// EnabledInternal wraps GlobalStmtSummary.EnableInternalQuery and stmtsummary.StmtSummaryByDigestMap.EnabledInternal. +func EnabledInternal() bool { + if config.GetGlobalConfig().Instance.StmtSummaryEnablePersistent { + return GlobalStmtSummary.EnableInternalQuery() + } + return stmtsummary.StmtSummaryByDigestMap.EnabledInternal() +} + +// SetEnabled wraps GlobalStmtSummary.SetEnabled and stmtsummary.StmtSummaryByDigestMap.SetEnabled. +func SetEnabled(v bool) error { + if config.GetGlobalConfig().Instance.StmtSummaryEnablePersistent { + return GlobalStmtSummary.SetEnabled(v) + } + return stmtsummary.StmtSummaryByDigestMap.SetEnabled(v) +} + +// SetEnableInternalQuery wraps GlobalStmtSummary.SetEnableInternalQuery and +// stmtsummary.StmtSummaryByDigestMap.SetEnabledInternalQuery. +func SetEnableInternalQuery(v bool) error { + if config.GetGlobalConfig().Instance.StmtSummaryEnablePersistent { + return GlobalStmtSummary.SetEnableInternalQuery(v) + } + return stmtsummary.StmtSummaryByDigestMap.SetEnabledInternalQuery(v) +} + +// SetRefreshInterval wraps GlobalStmtSummary.SetRefreshInterval and stmtsummary.StmtSummaryByDigestMap.SetRefreshInterval. +func SetRefreshInterval(v int64) error { + if config.GetGlobalConfig().Instance.StmtSummaryEnablePersistent { + return GlobalStmtSummary.SetRefreshInterval(uint32(v)) + } + return stmtsummary.StmtSummaryByDigestMap.SetRefreshInterval(v) +} + +// SetHistorySize wraps stmtsummary.StmtSummaryByDigestMap.SetHistorySize. +func SetHistorySize(v int) error { + if config.GetGlobalConfig().Instance.StmtSummaryEnablePersistent { + return nil // not support + } + return stmtsummary.StmtSummaryByDigestMap.SetHistorySize(v) +} + +// SetMaxStmtCount wraps GlobalStmtSummary.SetMaxStmtCount and stmtsummary.StmtSummaryByDigestMap.SetMaxStmtCount. +func SetMaxStmtCount(v int) error { + if config.GetGlobalConfig().Instance.StmtSummaryEnablePersistent { + return GlobalStmtSummary.SetMaxStmtCount(uint32(v)) + } + return stmtsummary.StmtSummaryByDigestMap.SetMaxStmtCount(uint(v)) +} + +// SetMaxSQLLength wraps GlobalStmtSummary.SetMaxSQLLength and stmtsummary.StmtSummaryByDigestMap.SetMaxSQLLength. +func SetMaxSQLLength(v int) error { + if config.GetGlobalConfig().Instance.StmtSummaryEnablePersistent { + return GlobalStmtSummary.SetMaxSQLLength(uint32(v)) + } + return stmtsummary.StmtSummaryByDigestMap.SetMaxSQLLength(v) +} + +// GetMoreThanCntBindableStmt wraps GlobalStmtSummary.GetMoreThanCntBindableStmt and +// stmtsummary.StmtSummaryByDigestMap.GetMoreThanCntBindableStmt. +func GetMoreThanCntBindableStmt(frequency int64) []*stmtsummary.BindableStmt { + if config.GetGlobalConfig().Instance.StmtSummaryEnablePersistent { + return GlobalStmtSummary.GetMoreThanCntBindableStmt(frequency) + } + return stmtsummary.StmtSummaryByDigestMap.GetMoreThanCntBindableStmt(frequency) +} diff --git a/util/stmtsummary/v2/stmtsummary_test.go b/util/stmtsummary/v2/stmtsummary_test.go new file mode 100644 index 0000000000000..a01d0143d4573 --- /dev/null +++ b/util/stmtsummary/v2/stmtsummary_test.go @@ -0,0 +1,86 @@ +// Copyright 2023 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 stmtsummary + +import ( + "sync" + "testing" + "time" + + "github.com/stretchr/testify/require" +) + +func TestStmtWindow(t *testing.T) { + ss := NewStmtSummary4Test(5) + defer ss.Close() + ss.Add(GenerateStmtExecInfo4Test("digest1")) + ss.Add(GenerateStmtExecInfo4Test("digest1")) + ss.Add(GenerateStmtExecInfo4Test("digest2")) + ss.Add(GenerateStmtExecInfo4Test("digest2")) + ss.Add(GenerateStmtExecInfo4Test("digest3")) + ss.Add(GenerateStmtExecInfo4Test("digest4")) + ss.Add(GenerateStmtExecInfo4Test("digest5")) + ss.Add(GenerateStmtExecInfo4Test("digest6")) + ss.Add(GenerateStmtExecInfo4Test("digest7")) + require.Equal(t, 5, ss.window.lru.Size()) + require.Equal(t, 2, ss.window.evicted.count()) + require.Equal(t, int64(4), ss.window.evicted.other.ExecCount) // digest1 digest1 digest2 digest2 + ss.Clear() + require.Equal(t, 0, ss.window.lru.Size()) + require.Equal(t, 0, ss.window.evicted.count()) + require.Equal(t, int64(0), ss.window.evicted.other.ExecCount) +} + +func TestStmtSummary(t *testing.T) { + ss := NewStmtSummary4Test(3) + defer ss.Close() + + ss.storage = &waitableMockStmtStorage{mockStmtStorage: ss.storage.(*mockStmtStorage)} + w := ss.window + ss.Add(GenerateStmtExecInfo4Test("digest1")) + ss.Add(GenerateStmtExecInfo4Test("digest2")) + ss.Add(GenerateStmtExecInfo4Test("digest3")) + ss.Add(GenerateStmtExecInfo4Test("digest4")) + ss.Add(GenerateStmtExecInfo4Test("digest5")) + require.Equal(t, 3, w.lru.Size()) + require.Equal(t, 2, w.evicted.count()) + + ss.storage.(*waitableMockStmtStorage).Add(1) + newEnd := w.begin.Add(time.Duration(ss.RefreshInterval()+1) * time.Second) + timeNow = func() time.Time { + return newEnd + } + ss.storage.(*waitableMockStmtStorage).Wait() + + timeNow = time.Now + ss.Add(GenerateStmtExecInfo4Test("digest6")) + ss.Add(GenerateStmtExecInfo4Test("digest7")) + w = ss.window + require.Equal(t, 2, w.lru.Size()) + require.Equal(t, 0, w.evicted.count()) + + ss.Clear() + require.Equal(t, 0, w.lru.Size()) +} + +type waitableMockStmtStorage struct { + sync.WaitGroup + *mockStmtStorage +} + +func (s *waitableMockStmtStorage) persist(w *stmtWindow, end time.Time) { + defer s.Done() + s.mockStmtStorage.persist(w, end) +} diff --git a/util/stmtsummary/v2/tests/BUILD.bazel b/util/stmtsummary/v2/tests/BUILD.bazel new file mode 100644 index 0000000000000..22a042878e6fe --- /dev/null +++ b/util/stmtsummary/v2/tests/BUILD.bazel @@ -0,0 +1,22 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_test") + +go_test( + name = "tests_test", + srcs = [ + "main_test.go", + "table_test.go", + ], + deps = [ + "//config", + "//kv", + "//parser/auth", + "//planner/core", + "//session", + "//testkit", + "//testkit/testsetup", + "//util/stmtsummary/v2:stmtsummary", + "@com_github_pingcap_failpoint//:failpoint", + "@com_github_stretchr_testify//require", + "@org_uber_go_goleak//:goleak", + ], +) diff --git a/util/stmtsummary/v2/tests/main_test.go b/util/stmtsummary/v2/tests/main_test.go new file mode 100644 index 0000000000000..305780d660949 --- /dev/null +++ b/util/stmtsummary/v2/tests/main_test.go @@ -0,0 +1,33 @@ +// Copyright 2023 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 tests + +import ( + "testing" + + "github.com/pingcap/tidb/testkit/testsetup" + "go.uber.org/goleak" +) + +func TestMain(m *testing.M) { + testsetup.SetupForCommonTest() + opts := []goleak.Option{ + goleak.IgnoreTopFunction("github.com/golang/glog.(*loggingT).flushDaemon"), + goleak.IgnoreTopFunction("github.com/lestrrat-go/httprc.runFetchWorker"), + goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), + } + goleak.VerifyTestMain(m, opts...) +} diff --git a/util/stmtsummary/v2/tests/table_test.go b/util/stmtsummary/v2/tests/table_test.go new file mode 100644 index 0000000000000..9403a0f8a3b4c --- /dev/null +++ b/util/stmtsummary/v2/tests/table_test.go @@ -0,0 +1,523 @@ +// Copyright 2023 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 tests + +import ( + "fmt" + "math" + "testing" + + "github.com/pingcap/failpoint" + "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/parser/auth" + plannercore "github.com/pingcap/tidb/planner/core" + "github.com/pingcap/tidb/session" + "github.com/pingcap/tidb/testkit" + stmtsummaryv2 "github.com/pingcap/tidb/util/stmtsummary/v2" + "github.com/stretchr/testify/require" +) + +func TestStmtSummaryTable(t *testing.T) { + setupStmtSummary() + defer closeStmtSummary() + + store := testkit.CreateMockStore(t) + tk := newTestKitWithRoot(t, store) + + tk.MustExec("set @@tidb_enable_collect_execution_info=0;") + tk.MustQuery("select column_comment from information_schema.columns " + + "where table_name='STATEMENTS_SUMMARY' and column_name='STMT_TYPE'", + ).Check(testkit.Rows("Statement type")) + + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int, b varchar(10), key k(a))") + + // Clear all statements. + tk.MustExec("set global tidb_enable_stmt_summary = 0") + tk.MustExec("set global tidb_enable_stmt_summary = 1") + tk.MustQuery("select @@global.tidb_enable_stmt_summary").Check(testkit.Rows("1")) + + // Create a new session to test. + tk = newTestKitWithRoot(t, store) + + // Test INSERT + tk.MustExec("insert into t values(1, 'a')") + tk.MustExec("insert into t values(2, 'b')") + tk.MustExec("insert into t VALUES(3, 'c')") + tk.MustExec("/**/insert into t values(4, 'd')") + + sql := "select stmt_type, schema_name, table_names, index_names, exec_count, sum_cop_task_num, avg_total_keys, " + + "max_total_keys, avg_processed_keys, max_processed_keys, avg_write_keys, max_write_keys, avg_prewrite_regions, " + + "max_prewrite_regions, avg_affected_rows, query_sample_text " + + "from information_schema.statements_summary " + + "where digest_text like 'insert into `t`%'" + tk.MustQuery(sql).Check(testkit.Rows("Insert test test.t 4 0 0 0 0 0 2 2 1 1 1 insert into t values(1, 'a')")) + + // Test point get. + tk.MustExec("drop table if exists p") + tk.MustExec("create table p(a int primary key, b int)") + for i := 1; i < 3; i++ { + tk.MustQuery("select b from p where a=1") + expectedResult := fmt.Sprintf("%d \tid \ttask\testRows\toperator info\n\tPoint_Get_1\troot\t1 \ttable:p, handle:1 %s", i, "test.p") + // Also make sure that the plan digest is not empty + sql = "select exec_count, plan, table_names from information_schema.statements_summary " + + "where digest_text like 'select `b` from `p`%' and plan_digest != ''" + tk.MustQuery(sql).Check(testkit.Rows(expectedResult)) + } + + // Point get another database. + tk.MustQuery("select variable_value from mysql.tidb where variable_name = 'system_tz'") + // Test for Encode plan cache. + p1 := tk.Session().GetSessionVars().StmtCtx.GetEncodedPlan() + require.Greater(t, len(p1), 0) + rows := tk.MustQuery("select tidb_decode_plan('" + p1 + "');").Rows() + require.Equal(t, 1, len(rows)) + require.Equal(t, 1, len(rows[0])) + require.Regexp(t, "\n.*Point_Get.*table.tidb, index.PRIMARY.VARIABLE_NAME", rows[0][0]) + + sql = "select table_names from information_schema.statements_summary " + + "where digest_text like 'select `variable_value`%' and `schema_name`='test'" + tk.MustQuery(sql).Check(testkit.Rows("mysql.tidb")) + + // Test `create database`. + tk.MustExec("create database if not exists test") + // Test for Encode plan cache. + p2 := tk.Session().GetSessionVars().StmtCtx.GetEncodedPlan() + require.Equal(t, "", p2) + tk.MustQuery(`select table_names + from information_schema.statements_summary + where digest_text like 'create database%' and schema_name='test'`, + ).Check(testkit.Rows("")) + + // Test SELECT. + const failpointName = "github.com/pingcap/tidb/planner/core/mockPlanRowCount" + require.NoError(t, failpoint.Enable(failpointName, "return(100)")) + defer func() { require.NoError(t, failpoint.Disable(failpointName)) }() + tk.MustQuery("select * from t where a=2") + + // sum_cop_task_num is always 0 if tidb_enable_collect_execution_info disabled + sql = "select stmt_type, schema_name, table_names, index_names, exec_count, sum_cop_task_num, avg_total_keys, " + + "max_total_keys, avg_processed_keys, max_processed_keys, avg_write_keys, max_write_keys, avg_prewrite_regions, " + + "max_prewrite_regions, avg_affected_rows, query_sample_text, plan " + + "from information_schema.statements_summary " + + "where digest_text like 'select * from `t`%'" + tk.MustQuery(sql).Check(testkit.Rows("Select test test.t t:k 1 0 0 0 0 0 0 0 0 0 0 select * from t where a=2 \tid \ttask \testRows\toperator info\n" + + "\tIndexLookUp_10 \troot \t100 \t\n" + + "\t├─IndexRangeScan_8(Build)\tcop[tikv]\t100 \ttable:t, index:k(a), range:[2,2], keep order:false, stats:pseudo\n" + + "\t└─TableRowIDScan_9(Probe)\tcop[tikv]\t100 \ttable:t, keep order:false, stats:pseudo")) + + // select ... order by + tk.MustQuery(`select stmt_type, schema_name, table_names, index_names, exec_count, sum_cop_task_num, avg_total_keys, + max_total_keys, avg_processed_keys, max_processed_keys, avg_write_keys, max_write_keys, avg_prewrite_regions, + max_prewrite_regions, avg_affected_rows, query_sample_text + from information_schema.statements_summary + order by exec_count desc limit 1`, + ).Check(testkit.Rows("Insert test test.t 4 0 0 0 0 0 2 2 1 1 1 insert into t values(1, 'a')")) + + // Test different plans with same digest. + require.NoError(t, failpoint.Enable(failpointName, "return(1000)")) + tk.MustQuery("select * from t where a=3") + sql = "select stmt_type, schema_name, table_names, index_names, exec_count, sum_cop_task_num, avg_total_keys, " + + "max_total_keys, avg_processed_keys, max_processed_keys, avg_write_keys, max_write_keys, avg_prewrite_regions, " + + "max_prewrite_regions, avg_affected_rows, query_sample_text, plan " + + "from information_schema.statements_summary " + + "where digest_text like 'select * from `t`%'" + tk.MustQuery(sql).Check(testkit.Rows( + "Select test test.t t:k 2 0 0 0 0 0 0 0 0 0 0 select * from t where a=2 \tid \ttask \testRows\toperator info\n" + + "\tIndexLookUp_10 \troot \t100 \t\n" + + "\t├─IndexRangeScan_8(Build)\tcop[tikv]\t100 \ttable:t, index:k(a), range:[2,2], keep order:false, stats:pseudo\n" + + "\t└─TableRowIDScan_9(Probe)\tcop[tikv]\t100 \ttable:t, keep order:false, stats:pseudo")) + + // Disable it again. + tk.MustExec("set global tidb_enable_stmt_summary = false") + defer tk.MustExec("set global tidb_enable_stmt_summary = 1") + tk.MustQuery("select @@global.tidb_enable_stmt_summary").Check(testkit.Rows("0")) + + // Create a new session to test + tk = newTestKitWithRoot(t, store) + + // This statement shouldn't be summarized. + tk.MustQuery("select * from t where a=2") + + // The table should be cleared. + tk.MustQuery(`select stmt_type, schema_name, table_names, index_names, exec_count, sum_cop_task_num, avg_total_keys, + max_total_keys, avg_processed_keys, max_processed_keys, avg_write_keys, max_write_keys, avg_prewrite_regions, + max_prewrite_regions, avg_affected_rows, query_sample_text, plan + from information_schema.statements_summary`, + ).Check(testkit.Rows()) + + tk.MustExec("SET GLOBAL tidb_enable_stmt_summary = on") + // It should work immediately. + tk.MustExec("begin") + tk.MustExec("insert into t values(1, 'a')") + tk.MustExec("commit") + sql = "select stmt_type, schema_name, table_names, index_names, exec_count, sum_cop_task_num, avg_total_keys, " + + "max_total_keys, avg_processed_keys, max_processed_keys, avg_write_keys, max_write_keys, avg_prewrite_regions, " + + "max_prewrite_regions, avg_affected_rows, query_sample_text, prev_sample_text " + + "from information_schema.statements_summary " + + "where digest_text like 'insert into `t`%'" + tk.MustQuery(sql).Check(testkit.Rows("Insert test test.t 1 0 0 0 0 0 0 0 0 0 1 insert into t values(1, 'a') ")) + tk.MustQuery(`select stmt_type, schema_name, table_names, index_names, exec_count, sum_cop_task_num, avg_total_keys, + max_total_keys, avg_processed_keys, max_processed_keys, avg_write_keys, max_write_keys, avg_prewrite_regions, + max_prewrite_regions, avg_affected_rows, query_sample_text, prev_sample_text + from information_schema.statements_summary + where digest_text='commit'`, + ).Check(testkit.Rows("Commit test 1 0 0 0 0 0 2 2 1 1 0 commit insert into t values(1, 'a')")) + + tk.MustQuery("select * from t where a=2") + sql = "select stmt_type, schema_name, table_names, index_names, exec_count, sum_cop_task_num, avg_total_keys, " + + "max_total_keys, avg_processed_keys, max_processed_keys, avg_write_keys, max_write_keys, avg_prewrite_regions, " + + "max_prewrite_regions, avg_affected_rows, query_sample_text, plan " + + "from information_schema.statements_summary " + + "where digest_text like 'select * from `t`%'" + tk.MustQuery(sql).Check(testkit.Rows("Select test test.t t:k 1 0 0 0 0 0 0 0 0 0 0 select * from t where a=2 \tid \ttask \testRows\toperator info\n" + + "\tIndexLookUp_10 \troot \t1000 \t\n" + + "\t├─IndexRangeScan_8(Build)\tcop[tikv]\t1000 \ttable:t, index:k(a), range:[2,2], keep order:false, stats:pseudo\n" + + "\t└─TableRowIDScan_9(Probe)\tcop[tikv]\t1000 \ttable:t, keep order:false, stats:pseudo")) + + // Disable it in global scope. + tk.MustExec("set global tidb_enable_stmt_summary = false") + + // Create a new session to test. + tk = newTestKitWithRoot(t, store) + + // Statement summary is disabled. + tk.MustQuery(`select stmt_type, schema_name, table_names, index_names, exec_count, sum_cop_task_num, avg_total_keys, + max_total_keys, avg_processed_keys, max_processed_keys, avg_write_keys, max_write_keys, avg_prewrite_regions, + max_prewrite_regions, avg_affected_rows, query_sample_text, plan + from information_schema.statements_summary`, + ).Check(testkit.Rows()) + + tk.MustExec("set global tidb_enable_stmt_summary = on") + tk.MustExec("set global tidb_stmt_summary_history_size = 24") +} + +func TestStmtSummaryTablePrivilege(t *testing.T) { + setupStmtSummary() + defer closeStmtSummary() + + store := testkit.CreateMockStore(t) + tk := newTestKitWithRoot(t, store) + + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int, b varchar(10), key k(a))") + defer tk.MustExec("drop table if exists t") + + // Clear all statements. + tk.MustExec("set global tidb_enable_stmt_summary = 0") + tk.MustExec("set global tidb_enable_stmt_summary = 1") + + // Create a new user to test statements summary table privilege + tk.MustExec("drop user if exists 'test_user'@'localhost'") + tk.MustExec("create user 'test_user'@'localhost'") + defer tk.MustExec("drop user if exists 'test_user'@'localhost'") + tk.MustExec("grant select on test.t to 'test_user'@'localhost'") + tk.MustExec("select * from t where a=1") + result := tk.MustQuery("select * from information_schema.statements_summary where digest_text like 'select * from `t`%'") + require.Equal(t, 1, len(result.Rows())) + result = tk.MustQuery("select * from information_schema.statements_summary_history where digest_text like 'select * from `t`%'") + require.Equal(t, 1, len(result.Rows())) + + tk1 := newTestKit(t, store) + tk1.Session().Auth(&auth.UserIdentity{ + Username: "test_user", + Hostname: "localhost", + AuthUsername: "test_user", + AuthHostname: "localhost", + }, nil, nil) + + result = tk1.MustQuery("select * from information_schema.statements_summary where digest_text like 'select * from `t`%'") + // Ordinary users can not see others' records + require.Equal(t, 0, len(result.Rows())) + result = tk1.MustQuery("select * from information_schema.statements_summary_history where digest_text like 'select * from `t`%'") + require.Equal(t, 0, len(result.Rows())) + tk1.MustExec("select * from t where b=1") + result = tk1.MustQuery("select * from information_schema.statements_summary where digest_text like 'select * from `t`%'") + // Ordinary users can see his own records + require.Equal(t, 1, len(result.Rows())) + result = tk1.MustQuery("select * from information_schema.statements_summary_history where digest_text like 'select * from `t`%'") + require.Equal(t, 1, len(result.Rows())) + + tk.MustExec("grant process on *.* to 'test_user'@'localhost'") + result = tk1.MustQuery("select * from information_schema.statements_summary where digest_text like 'select * from `t`%'") + // Users with 'PROCESS' privileges can query all records. + require.Equal(t, 2, len(result.Rows())) + result = tk1.MustQuery("select * from information_schema.statements_summary_history where digest_text like 'select * from `t`%'") + require.Equal(t, 2, len(result.Rows())) +} + +func TestCapturePrivilege(t *testing.T) { + setupStmtSummary() + defer closeStmtSummary() + + store := testkit.CreateMockStore(t) + tk := newTestKitWithRoot(t, store) + + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int, b varchar(10), key k(a))") + defer tk.MustExec("drop table if exists t") + + tk.MustExec("drop table if exists t1") + tk.MustExec("create table t1(a int, b varchar(10), key k(a))") + defer tk.MustExec("drop table if exists t1") + + // Clear all statements. + tk.MustExec("set global tidb_enable_stmt_summary = 0") + tk.MustExec("set global tidb_enable_stmt_summary = 1") + + // Create a new user to test statements summary table privilege + tk.MustExec("drop user if exists 'test_user'@'localhost'") + tk.MustExec("create user 'test_user'@'localhost'") + defer tk.MustExec("drop user if exists 'test_user'@'localhost'") + tk.MustExec("grant select on test.t1 to 'test_user'@'localhost'") + tk.MustExec("select * from t where a=1") + tk.MustExec("select * from t where a=1") + tk.MustExec("admin capture bindings") + rows := tk.MustQuery("show global bindings").Rows() + require.Len(t, rows, 1) + + tk1 := newTestKit(t, store) + tk1.Session().Auth(&auth.UserIdentity{ + Username: "test_user", + Hostname: "localhost", + AuthUsername: "test_user", + AuthHostname: "localhost", + }, nil, nil) + + rows = tk1.MustQuery("show global bindings").Rows() + // Ordinary users can not see others' records + require.Len(t, rows, 0) + tk1.MustExec("select * from t1 where b=1") + tk1.MustExec("select * from t1 where b=1") + tk1.MustExec("admin capture bindings") + rows = tk1.MustQuery("show global bindings").Rows() + require.Len(t, rows, 1) + + tk.MustExec("grant all on *.* to 'test_user'@'localhost'") + tk1.MustExec("admin capture bindings") + rows = tk1.MustQuery("show global bindings").Rows() + require.Len(t, rows, 2) +} + +func TestStmtSummaryErrorCount(t *testing.T) { + setupStmtSummary() + defer closeStmtSummary() + + store := testkit.CreateMockStore(t) + tk := newTestKitWithRoot(t, store) + + // Clear summaries. + tk.MustExec("set global tidb_enable_stmt_summary = 0") + tk.MustExec("set global tidb_enable_stmt_summary = 1") + + tk.MustExec("use test") + tk.MustExec("drop table if exists stmt_summary_test") + tk.MustExec("create table stmt_summary_test(id int primary key)") + tk.MustExec("insert into stmt_summary_test values(1)") + _, err := tk.Exec("insert into stmt_summary_test values(1)") + require.Error(t, err) + + sql := "select exec_count, sum_errors, sum_warnings from information_schema.statements_summary where digest_text like \"insert into `stmt_summary_test`%\"" + tk.MustQuery(sql).Check(testkit.Rows("2 1 0")) + + tk.MustExec("insert ignore into stmt_summary_test values(1)") + sql = "select exec_count, sum_errors, sum_warnings from information_schema.statements_summary where digest_text like \"insert ignore into `stmt_summary_test`%\"" + tk.MustQuery(sql).Check(testkit.Rows("1 0 1")) +} + +func TestStmtSummaryPreparedStatements(t *testing.T) { + setupStmtSummary() + defer closeStmtSummary() + + store := testkit.CreateMockStore(t) + tk := newTestKitWithRoot(t, store) + + // Clear summaries. + tk.MustExec("set global tidb_enable_stmt_summary = 0") + tk.MustExec("set global tidb_enable_stmt_summary = 1") + + tk.MustExec("use test") + tk.MustExec("prepare stmt from 'select ?'") + tk.MustExec("set @number=1") + tk.MustExec("execute stmt using @number") + + tk.MustQuery(`select exec_count + from information_schema.statements_summary + where digest_text like "prepare%"`).Check(testkit.Rows()) + tk.MustQuery(`select exec_count + from information_schema.statements_summary + where digest_text like "select ?"`).Check(testkit.Rows("1")) +} + +func TestStmtSummarySensitiveQuery(t *testing.T) { + setupStmtSummary() + defer closeStmtSummary() + + store := testkit.CreateMockStore(t) + tk := newTestKitWithRoot(t, store) + + tk.MustExec("set global tidb_enable_stmt_summary = 0") + tk.MustExec("set global tidb_enable_stmt_summary = 1") + tk.MustExec("drop user if exists user_sensitive;") + tk.MustExec("create user user_sensitive identified by '123456789';") + tk.MustExec("alter user 'user_sensitive'@'%' identified by 'abcdefg';") + tk.MustExec("set password for 'user_sensitive'@'%' = 'xyzuvw';") + tk.MustQuery("select query_sample_text from `information_schema`.`STATEMENTS_SUMMARY` " + + "where query_sample_text like '%user_sensitive%' and " + + "(query_sample_text like 'set password%' or query_sample_text like 'create user%' or query_sample_text like 'alter user%') " + + "order by query_sample_text;"). + Check(testkit.Rows( + "alter user {user_sensitive@% password = ***}", + "create user {user_sensitive@% password = ***}", + "set password for user user_sensitive@%", + )) +} + +func TestStmtSummaryTableOther(t *testing.T) { + setupStmtSummary() + defer closeStmtSummary() + + store := testkit.CreateMockStore(t) + tk := newTestKitWithRoot(t, store) + + tk.MustExec("set global tidb_enable_stmt_summary=0") + tk.MustExec("set global tidb_enable_stmt_summary=1") + // set stmt size to 1 + // first sql + tk.MustExec("set global tidb_stmt_summary_max_stmt_count=1") + defer tk.MustExec("set global tidb_stmt_summary_max_stmt_count=100") + // second sql, evict first sql from stmt_summary + tk.MustExec("show databases;") + // third sql, evict second sql from stmt_summary + tk.MustQuery("SELECT DIGEST_TEXT, DIGEST FROM `INFORMATION_SCHEMA`.`STATEMENTS_SUMMARY`;"). + Check(testkit.Rows( + // digest in cache + // "show databases ;" + "show databases ; dcd020298c5f79e8dc9d63b3098083601614a04a52db458738347d15ea5712a1", + // digest evicted + " ", + )) + // forth sql, evict third sql from stmt_summary + tk.MustQuery("SELECT SCHEMA_NAME FROM `INFORMATION_SCHEMA`.`STATEMENTS_SUMMARY`;"). + Check(testkit.Rows( + // digest in cache + "test", // select xx from yy; + // digest evicted + "", + )) +} + +func TestStmtSummaryHistoryTableOther(t *testing.T) { + setupStmtSummary() + defer closeStmtSummary() + + store := testkit.CreateMockStore(t) + tk := newTestKitWithRoot(t, store) + + tk.MustExec("set global tidb_stmt_summary_max_stmt_count = 1") + defer tk.MustExec("set global tidb_stmt_summary_max_stmt_count = 100") + + tk.MustExec("set global tidb_enable_stmt_summary = 0") + tk.MustExec("set global tidb_enable_stmt_summary = 1") + // first sql + tk.MustExec("set global tidb_stmt_summary_max_stmt_count=1") + // second sql, evict first sql from stmt_summary + tk.MustExec("show databases;") + // third sql, evict second sql from stmt_summary + tk.MustQuery("SELECT DIGEST_TEXT, DIGEST FROM `INFORMATION_SCHEMA`.`STATEMENTS_SUMMARY_HISTORY`;"). + Check(testkit.Rows( + // digest in cache + // "show databases ;" + "show databases ; dcd020298c5f79e8dc9d63b3098083601614a04a52db458738347d15ea5712a1", + // digest evicted + " ", + )) + // forth sql, evict third sql from stmt_summary + tk.MustQuery("SELECT SCHEMA_NAME FROM `INFORMATION_SCHEMA`.`STATEMENTS_SUMMARY_HISTORY`;"). + Check(testkit.Rows( + // digest in cache + "test", // select xx from yy; + // digest evicted + "", + )) +} + +func TestPerformanceSchemaforPlanCache(t *testing.T) { + setupStmtSummary() + defer closeStmtSummary() + + store := testkit.CreateMockStore(t) + tmp := testkit.NewTestKit(t, store) + tmp.MustExec("set tidb_enable_prepared_plan_cache=ON") + tk := newTestKitWithPlanCache(t, store) + + // Clear summaries. + tk.MustExec("set global tidb_enable_stmt_summary = 0") + tk.MustExec("set global tidb_enable_stmt_summary = 1") + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int)") + tk.MustExec("prepare stmt from 'select * from t'") + tk.MustExec("execute stmt") + tk.MustQuery("select plan_cache_hits, plan_in_cache from information_schema.statements_summary where digest_text='select * from `t`'").Check( + testkit.Rows("0 0")) + tk.MustExec("execute stmt") + tk.MustExec("execute stmt") + tk.MustExec("execute stmt") + tk.MustQuery("select plan_cache_hits, plan_in_cache from information_schema.statements_summary where digest_text='select * from `t`'").Check( + testkit.Rows("3 1")) +} + +func newTestKit(t *testing.T, store kv.Storage) *testkit.TestKit { + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + return tk +} + +func newTestKitWithRoot(t *testing.T, store kv.Storage) *testkit.TestKit { + tk := newTestKit(t, store) + require.NoError(t, tk.Session().Auth(&auth.UserIdentity{Username: "root", Hostname: "%"}, nil, nil)) + return tk +} + +func newTestKitWithPlanCache(t *testing.T, store kv.Storage) *testkit.TestKit { + tk := testkit.NewTestKit(t, store) + se, err := session.CreateSession4TestWithOpt(store, &session.Opt{ + PreparedPlanCache: plannercore.NewLRUPlanCache(100, 0.1, math.MaxUint64, tk.Session()), + }) + require.NoError(t, err) + tk.SetSession(se) + tk.RefreshConnectionID() + require.NoError(t, tk.Session().Auth(&auth.UserIdentity{Username: "root", Hostname: "%"}, nil, nil)) + return tk +} + +func setupStmtSummary() { + stmtsummaryv2.Setup(&stmtsummaryv2.Config{ + Filename: "tidb-statements.log", + }) + config.UpdateGlobal(func(conf *config.Config) { + conf.Instance.StmtSummaryEnablePersistent = true + }) +} + +func closeStmtSummary() { + stmtsummaryv2.GlobalStmtSummary.Close() + stmtsummaryv2.GlobalStmtSummary = nil + config.UpdateGlobal(func(conf *config.Config) { + conf.Instance.StmtSummaryEnablePersistent = false + }) +} diff --git a/util/util.go b/util/util.go index e3a9cf25f750f..19db12aad1d4a 100644 --- a/util/util.go +++ b/util/util.go @@ -15,6 +15,7 @@ package util import ( + "bufio" "bytes" "encoding/json" "fmt" @@ -233,3 +234,52 @@ func (t *TCPConnWithIOCounter) Write(b []byte) (n int, err error) { t.c.Add(uint64(n)) return n, err } + +// ReadLine tries to read a complete line from bufio.Reader. +// maxLineSize specifies the maximum size of a single line. +func ReadLine(reader *bufio.Reader, maxLineSize int) ([]byte, error) { + var resByte []byte + lineByte, isPrefix, err := reader.ReadLine() + if isPrefix { + // Need to read more data. + resByte = make([]byte, len(lineByte), len(lineByte)*2) + } else { + resByte = make([]byte, len(lineByte)) + } + // Use copy here to avoid shallow copy problem. + copy(resByte, lineByte) + if err != nil { + return resByte, err + } + var tempLine []byte + for isPrefix { + tempLine, isPrefix, err = reader.ReadLine() + resByte = append(resByte, tempLine...) // nozero + // Use maxLineSize to check the single line length. + if len(resByte) > maxLineSize { + return resByte, errors.Errorf("single line length exceeds limit: %v", maxLineSize) + } + if err != nil { + return resByte, err + } + } + return resByte, err +} + +// ReadLines tries to read lines from bufio.Reader. +// count specifies the number of lines. +// maxLineSize specifies the maximum size of a single line. +func ReadLines(reader *bufio.Reader, count int, maxLineSize int) ([][]byte, error) { + lines := make([][]byte, 0, count) + for i := 0; i < count; i++ { + line, err := ReadLine(reader, maxLineSize) + if err == io.EOF && len(lines) > 0 { + return lines, nil + } + if err != nil { + return nil, err + } + lines = append(lines, line) + } + return lines, nil +} diff --git a/util/util_test.go b/util/util_test.go index ca68a55cd8ba6..cdd173e7a6c48 100644 --- a/util/util_test.go +++ b/util/util_test.go @@ -15,12 +15,16 @@ package util import ( + "bufio" + "io" + "strings" "testing" "time" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/util/memory" "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" ) func TestLogFormat(t *testing.T) { @@ -72,3 +76,21 @@ func TestLogFormat(t *testing.T) { logFields = GenLogFields(costTime, info, false) assert.Equal(t, len(logFields[6].String), len(mockTooLongQuery)) } + +func TestReadLine(t *testing.T) { + reader := bufio.NewReader(strings.NewReader(`line1 +line2 +line3`)) + line, err := ReadLine(reader, 1024) + require.NoError(t, err) + require.Equal(t, "line1", string(line)) + line, err = ReadLine(reader, 1024) + require.NoError(t, err) + require.Equal(t, "line2", string(line)) + line, err = ReadLine(reader, 1024) + require.NoError(t, err) + require.Equal(t, "line3", string(line)) + line, err = ReadLine(reader, 1024) + require.Equal(t, io.EOF, err) + require.Len(t, line, 0) +}