From 82624b4f2ec3d4eec3871adc4d2ed4646cf27e59 Mon Sep 17 00:00:00 2001 From: time-and-fate <25057648+time-and-fate@users.noreply.github.com> Date: Tue, 13 Dec 2022 17:25:45 +0800 Subject: [PATCH 01/12] add --- executor/adapter.go | 26 +++++++++++++ executor/slow_query.go | 2 +- expression/expression.go | 23 ++++++++---- infoschema/tables.go | 1 + planner/core/common_plans.go | 10 ----- planner/core/exhaust_physical_plans.go | 5 ++- planner/core/find_best_task.go | 8 ++-- planner/core/stats.go | 52 ++++++++++++++------------ planner/core/task.go | 9 ++++- planner/optimize.go | 2 + sessionctx/stmtctx/stmtctx.go | 51 ++++++++++++++++++++++++- sessionctx/variable/session.go | 28 +++++++++++++- 12 files changed, 166 insertions(+), 51 deletions(-) diff --git a/executor/adapter.go b/executor/adapter.go index 5f01aa51e0378..e86238f2e5ff6 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -1527,6 +1527,7 @@ func (a *ExecStmt) LogSlowQuery(txnTS uint64, succ bool, hasMoreResults bool) { IsWriteCacheTable: stmtCtx.WaitLockLeaseTime > 0, StatsLoadStatus: convertStatusIntoString(a.Ctx, stmtCtx.StatsLoadStatus), IsSyncStatsFailed: stmtCtx.IsSyncStatsFailed, + Warnings: collectWarningsForSlowLog(stmtCtx), } failpoint.Inject("assertSyncStatsFailed", func(val failpoint.Value) { if val.(bool) { @@ -1582,6 +1583,31 @@ func (a *ExecStmt) LogSlowQuery(txnTS uint64, succ bool, hasMoreResults bool) { } } +func extractMsgFromSQLWarn(SQLWarn *stmtctx.SQLWarn) string { + warn := errors.Cause(SQLWarn.Err) + if x, ok := warn.(*terror.Error); ok { + sqlErr := terror.ToSQLError(x) + return sqlErr.Message + } + return warn.Error() +} + +func collectWarningsForSlowLog(stmtCtx *stmtctx.StatementContext) []variable.JSONSQLWarnForSlowLog { + warnings := stmtCtx.GetWarnings() + extraWarnings := stmtCtx.GetExtraWarnings() + res := make([]variable.JSONSQLWarnForSlowLog, len(warnings)+len(extraWarnings)) + for i := range warnings { + res[i].Level = warnings[i].Level + res[i].Message = extractMsgFromSQLWarn(&warnings[i]) + } + for i := range extraWarnings { + res[len(warnings)+i].Level = extraWarnings[i].Level + res[len(warnings)+i].Message = extractMsgFromSQLWarn(&extraWarnings[i]) + res[len(warnings)+i].IsExtra = true + } + return res +} + // GetResultRowsCount gets the count of the statement result rows. func GetResultRowsCount(stmtCtx *stmtctx.StatementContext, p plannercore.Plan) int64 { runtimeStatsColl := stmtCtx.RuntimeStatsColl diff --git a/executor/slow_query.go b/executor/slow_query.go index 395d8f4eba8ac..7b0bd9df07996 100644 --- a/executor/slow_query.go +++ b/executor/slow_query.go @@ -781,7 +781,7 @@ func getColumnValueFactoryByName(sctx sessionctx.Context, colName string, column }, nil case variable.SlowLogUserStr, variable.SlowLogHostStr, execdetails.BackoffTypesStr, variable.SlowLogDBStr, variable.SlowLogIndexNamesStr, variable.SlowLogDigestStr, variable.SlowLogStatsInfoStr, variable.SlowLogCopProcAddr, variable.SlowLogCopWaitAddr, variable.SlowLogPlanDigest, - variable.SlowLogPrevStmt, variable.SlowLogQuerySQLStr: + variable.SlowLogPrevStmt, variable.SlowLogQuerySQLStr, variable.SlowLogWarnings: return func(row []types.Datum, value string, tz *time.Location, checker *slowLogChecker) (valid bool, err error) { row[columnIdx] = types.NewStringDatum(value) return true, nil diff --git a/expression/expression.go b/expression/expression.go index 024bac00ef960..f3441d35cfade 100644 --- a/expression/expression.go +++ b/expression/expression.go @@ -1358,12 +1358,15 @@ func canScalarFuncPushDown(scalarFunc *ScalarFunction, pc PbConverter, storeType panic(errors.Errorf("unspecified PbCode: %T", scalarFunc.Function)) }) } + storageName := storeType.Name() + if storeType == kv.UnSpecified { + storageName = "storage layer" + } + warnErr := errors.New("Scalar function '" + scalarFunc.FuncName.L + "'(signature: " + scalarFunc.Function.PbCode().String() + ", return type: " + scalarFunc.RetType.CompactStr() + ") is not supported to push down to " + storageName + " now.") if pc.sc.InExplainStmt { - storageName := storeType.Name() - if storeType == kv.UnSpecified { - storageName = "storage layer" - } - pc.sc.AppendWarning(errors.New("Scalar function '" + scalarFunc.FuncName.L + "'(signature: " + scalarFunc.Function.PbCode().String() + ", return type: " + scalarFunc.RetType.CompactStr() + ") is not supported to push down to " + storageName + " now.")) + pc.sc.AppendWarning(warnErr) + } else { + pc.sc.AppendExtraWarning(warnErr) } return false } @@ -1393,14 +1396,20 @@ func canExprPushDown(expr Expression, pc PbConverter, storeType kv.StoreType, ca if expr.GetType().GetType() == mysql.TypeEnum && canEnumPush { break } + warnErr := errors.New("Expression about '" + expr.String() + "' can not be pushed to TiFlash because it contains unsupported calculation of type '" + types.TypeStr(expr.GetType().GetType()) + "'.") if pc.sc.InExplainStmt { - pc.sc.AppendWarning(errors.New("Expression about '" + expr.String() + "' can not be pushed to TiFlash because it contains unsupported calculation of type '" + types.TypeStr(expr.GetType().GetType()) + "'.")) + pc.sc.AppendWarning(warnErr) + } else { + pc.sc.AppendExtraWarning(warnErr) } return false case mysql.TypeNewDecimal: if !expr.GetType().IsDecimalValid() { + warnErr := errors.New("Expression about '" + expr.String() + "' can not be pushed to TiFlash because it contains invalid decimal('" + strconv.Itoa(expr.GetType().GetFlen()) + "','" + strconv.Itoa(expr.GetType().GetDecimal()) + "').") if pc.sc.InExplainStmt { - pc.sc.AppendWarning(errors.New("Expression about '" + expr.String() + "' can not be pushed to TiFlash because it contains invalid decimal('" + strconv.Itoa(expr.GetType().GetFlen()) + "','" + strconv.Itoa(expr.GetType().GetDecimal()) + "').")) + pc.sc.AppendWarning(warnErr) + } else { + pc.sc.AppendExtraWarning(warnErr) } return false } diff --git a/infoschema/tables.go b/infoschema/tables.go index d76d8b8be60b0..519b3764bf4a6 100644 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -890,6 +890,7 @@ var slowQueryCols = []columnInfo{ {name: variable.SlowLogBackoffTotal, tp: mysql.TypeDouble, size: 22}, {name: variable.SlowLogWriteSQLRespTotal, tp: mysql.TypeDouble, size: 22}, {name: variable.SlowLogResultRows, tp: mysql.TypeLonglong, size: 22}, + {name: variable.SlowLogWarnings, tp: mysql.TypeLongBlob, size: types.UnspecifiedLength}, {name: variable.SlowLogBackoffDetail, tp: mysql.TypeVarchar, size: 4096}, {name: variable.SlowLogPrepared, tp: mysql.TypeTiny, size: 1}, {name: variable.SlowLogSucc, tp: mysql.TypeTiny, size: 1}, diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index 73bb25e0d874c..045696cf76f06 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -898,11 +898,6 @@ func (e *Explain) explainOpRecursivelyInJSONFormat(flatOp *FlatOperator, flats F textTreeExplainID := texttree.PrettyIdentifier(explainID, flatOp.TextTreeIndent, flatOp.IsLastChild) cur := e.prepareOperatorInfoForJSONFormat(flatOp.Origin, taskTp, textTreeExplainID, explainID) - if e.ctx != nil && e.ctx.GetSessionVars() != nil && e.ctx.GetSessionVars().StmtCtx != nil { - if optimInfo, ok := e.ctx.GetSessionVars().StmtCtx.OptimInfo[flatOp.Origin.ID()]; ok { - e.ctx.GetSessionVars().StmtCtx.AppendNote(errors.New(optimInfo)) - } - } for _, idx := range flatOp.ChildrenIdx { cur.SubOperators = append(cur.SubOperators, @@ -922,11 +917,6 @@ func (e *Explain) explainFlatOpInRowFormat(flatOp *FlatOperator) { flatOp.TextTreeIndent, flatOp.IsLastChild) e.prepareOperatorInfo(flatOp.Origin, taskTp, textTreeExplainID) - if e.ctx != nil && e.ctx.GetSessionVars() != nil && e.ctx.GetSessionVars().StmtCtx != nil { - if optimInfo, ok := e.ctx.GetSessionVars().StmtCtx.OptimInfo[flatOp.Origin.ID()]; ok { - e.ctx.GetSessionVars().StmtCtx.AppendNote(errors.New(optimInfo)) - } - } } func getRuntimeInfoStr(ctx sessionctx.Context, p Plan, runtimeStatsColl *execdetails.RuntimeStatsColl) (actRows, analyzeInfo, memoryInfo, diskInfo string) { diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index 768a8c20fc0b5..2817f370ffcec 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -2730,8 +2730,11 @@ func (la *LogicalAggregation) checkCanPushDownToMPP() bool { } } if hasUnsupportedDistinct { + warnErr := errors.New("Aggregation can not be pushed to storage layer in mpp mode because it contains agg function with distinct") if la.ctx.GetSessionVars().StmtCtx.InExplainStmt { - la.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.New("Aggregation can not be pushed to storage layer in mpp mode because it contains agg function with distinct")) + la.ctx.GetSessionVars().StmtCtx.AppendWarning(warnErr) + } else { + la.ctx.GetSessionVars().StmtCtx.AppendExtraWarning(warnErr) } return false } diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index afc5223b9be94..bc869c62184ad 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -889,10 +889,12 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, planCounter pruningInfo := ds.getPruningInfo(candidates, prop) defer func() { if err == nil && t != nil && !t.invalid() && pruningInfo != "" { - if ds.ctx.GetSessionVars().StmtCtx.OptimInfo == nil { - ds.ctx.GetSessionVars().StmtCtx.OptimInfo = make(map[int]string) + warnErr := errors.New(pruningInfo) + if ds.ctx.GetSessionVars().StmtCtx.InExplainStmt { + ds.ctx.GetSessionVars().StmtCtx.AppendNote(warnErr) + } else { + ds.ctx.GetSessionVars().StmtCtx.AppendExtraNote(warnErr) } - ds.ctx.GetSessionVars().StmtCtx.OptimInfo[t.plan().ID()] = pruningInfo } }() diff --git a/planner/core/stats.go b/planner/core/stats.go index f377feac91030..71e1037c52c76 100644 --- a/planner/core/stats.go +++ b/planner/core/stats.go @@ -343,35 +343,37 @@ func (ds *DataSource) derivePathStatsAndTryHeuristics() error { if selected != nil { ds.possibleAccessPaths[0] = selected ds.possibleAccessPaths = ds.possibleAccessPaths[:1] - if ds.ctx.GetSessionVars().StmtCtx.InVerboseExplain { - var tableName string - if ds.TableAsName.O == "" { - tableName = ds.tableInfo.Name.O + var tableName string + if ds.TableAsName.O == "" { + tableName = ds.tableInfo.Name.O + } else { + tableName = ds.TableAsName.O + } + var sb strings.Builder + if selected.IsTablePath() { + // TODO: primary key / handle / real name? + sb.WriteString(fmt.Sprintf("handle of %s is selected since the path only has point ranges", tableName)) + } else { + if selected.Index.Unique { + sb.WriteString("unique ") + } + sb.WriteString(fmt.Sprintf("index %s of %s is selected since the path", selected.Index.Name.O, tableName)) + if isRefinedPath { + sb.WriteString(" only fetches limited number of rows") } else { - tableName = ds.TableAsName.O + sb.WriteString(" only has point ranges") } - if selected.IsTablePath() { - // TODO: primary key / handle / real name? - ds.ctx.GetSessionVars().StmtCtx.AppendNote(fmt.Errorf("handle of %s is selected since the path only has point ranges", tableName)) + if selected.IsSingleScan { + sb.WriteString(" with single scan") } else { - var sb strings.Builder - if selected.Index.Unique { - sb.WriteString("unique ") - } - sb.WriteString(fmt.Sprintf("index %s of %s is selected since the path", selected.Index.Name.O, tableName)) - if isRefinedPath { - sb.WriteString(" only fetches limited number of rows") - } else { - sb.WriteString(" only has point ranges") - } - if selected.IsSingleScan { - sb.WriteString(" with single scan") - } else { - sb.WriteString(" with double scan") - } - ds.ctx.GetSessionVars().StmtCtx.AppendNote(errors.New(sb.String())) + sb.WriteString(" with double scan") } } + if ds.ctx.GetSessionVars().StmtCtx.InVerboseExplain { + ds.ctx.GetSessionVars().StmtCtx.AppendNote(errors.New(sb.String())) + } else { + ds.ctx.GetSessionVars().StmtCtx.AppendExtraNote(errors.New(sb.String())) + } } return nil } @@ -435,8 +437,10 @@ func (ds *DataSource) DeriveStats(_ []*property.StatsInfo, _ *expression.Schema, if needConsiderIndexMerge { // PushDownExprs() will append extra warnings, which is annoying. So we reset warnings here. warnings := stmtCtx.GetWarnings() + extraWarnings := stmtCtx.GetExtraWarnings() _, remaining := expression.PushDownExprs(stmtCtx, indexMergeConds, ds.ctx.GetClient(), kv.UnSpecified) stmtCtx.SetWarnings(warnings) + stmtCtx.SetExtraWarnings(extraWarnings) if len(remaining) != 0 { needConsiderIndexMerge = false } diff --git a/planner/core/task.go b/planner/core/task.go index cc27029d83c8e..99952038688fe 100644 --- a/planner/core/task.go +++ b/planner/core/task.go @@ -1239,12 +1239,17 @@ func CheckAggCanPushCop(sctx sessionctx.Context, aggFuncs []*aggregation.AggFunc ret = false } - if !ret && sc.InExplainStmt { + if !ret { storageName := storeType.Name() if storeType == kv.UnSpecified { storageName = "storage layer" } - sc.AppendWarning(errors.New("Aggregation can not be pushed to " + storageName + " because " + reason)) + warnErr := errors.New("Aggregation can not be pushed to " + storageName + " because " + reason) + if sc.InExplainStmt { + sc.AppendWarning(warnErr) + } else { + sc.AppendExtraWarning(warnErr) + } } return ret } diff --git a/planner/optimize.go b/planner/optimize.go index d5ee997057180..2a0bdf0ab3d6c 100644 --- a/planner/optimize.go +++ b/planner/optimize.go @@ -234,6 +234,8 @@ func Optimize(ctx context.Context, sctx sessionctx.Context, node ast.Node, is in sessVars.FoundInBinding = true if sessVars.StmtCtx.InVerboseExplain { sessVars.StmtCtx.AppendNote(errors.Errorf("Using the bindSQL: %v", chosenBinding.BindSQL)) + } else { + sessVars.StmtCtx.AppendExtraNote(errors.Errorf("Using the bindSQL: %v", chosenBinding.BindSQL)) } } // Restore the hint to avoid changing the stmt node. diff --git a/sessionctx/stmtctx/stmtctx.go b/sessionctx/stmtctx/stmtctx.go index 6f9a276691149..7dbe3f78e6daa 100644 --- a/sessionctx/stmtctx/stmtctx.go +++ b/sessionctx/stmtctx/stmtctx.go @@ -209,8 +209,14 @@ type StatementContext struct { copied uint64 touched uint64 - message string - warnings []SQLWarn + message string + warnings []SQLWarn + // extraWarnings record some extra warnings and are only used by the slow log only now. + // extraWarnings record warnings that would not be printed through SHOW WARNINGS because they are expected to + // be printed only under some conditions (like in EXPLAIN or EXPLAIN VERBOSE), but we want to always output them + // through the slow log to help diagnostics, so we store them here separately. + extraWarnings []SQLWarn + execDetails execdetails.ExecDetails allExecDetails []*execdetails.DetailsNeedP90 } @@ -812,6 +818,47 @@ func (sc *StatementContext) AppendError(warn error) { } } +// GetExtraWarnings gets extra warnings. +func (sc *StatementContext) GetExtraWarnings() []SQLWarn { + sc.mu.Lock() + defer sc.mu.Unlock() + return sc.mu.extraWarnings +} + +// SetExtraWarnings sets extra warnings. +func (sc *StatementContext) SetExtraWarnings(warns []SQLWarn) { + sc.mu.Lock() + defer sc.mu.Unlock() + sc.mu.extraWarnings = warns +} + +// AppendExtraWarning appends an extra warning with level 'Warning'. +func (sc *StatementContext) AppendExtraWarning(warn error) { + sc.mu.Lock() + defer sc.mu.Unlock() + if len(sc.mu.extraWarnings) < math.MaxUint16 { + sc.mu.extraWarnings = append(sc.mu.extraWarnings, SQLWarn{WarnLevelWarning, warn}) + } +} + +// AppendExtraNote appends an extra warning with level 'Note'. +func (sc *StatementContext) AppendExtraNote(warn error) { + sc.mu.Lock() + defer sc.mu.Unlock() + if len(sc.mu.extraWarnings) < math.MaxUint16 { + sc.mu.extraWarnings = append(sc.mu.extraWarnings, SQLWarn{WarnLevelNote, warn}) + } +} + +// AppendExtraError appends an extra warning with level 'Error'. +func (sc *StatementContext) AppendExtraError(warn error) { + sc.mu.Lock() + defer sc.mu.Unlock() + if len(sc.mu.extraWarnings) < math.MaxUint16 { + sc.mu.extraWarnings = append(sc.mu.extraWarnings, SQLWarn{WarnLevelError, warn}) + } +} + // HandleTruncate ignores or returns the error based on the StatementContext state. func (sc *StatementContext) HandleTruncate(err error) error { // TODO: At present we have not checked whether the error can be ignored or treated as warning. diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 8c977aa832579..e7b0a848a9a71 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -19,6 +19,7 @@ import ( "context" "crypto/tls" "encoding/binary" + "encoding/json" "fmt" "math" "math/rand" @@ -1440,8 +1441,13 @@ func (s *SessionVars) IsMPPEnforced() bool { // TODO: Confirm whether this function will be inlined and // omit the overhead of string construction when calling with false condition. func (s *SessionVars) RaiseWarningWhenMPPEnforced(warning string) { - if s.IsMPPEnforced() && s.StmtCtx.InExplainStmt { + if !s.IsMPPEnforced() { + return + } + if s.StmtCtx.InExplainStmt { s.StmtCtx.AppendWarning(errors.New(warning)) + } else { + s.StmtCtx.AppendExtraWarning(errors.New(warning)) } } @@ -2726,6 +2732,7 @@ const ( SlowLogBackoffDetail = "Backoff_Detail" // SlowLogResultRows is the row count of the SQL result. SlowLogResultRows = "Result_rows" + SlowLogWarnings = "Warnings" // SlowLogIsExplicitTxn is used to indicate whether this sql execute in explicit transaction or not. SlowLogIsExplicitTxn = "IsExplicitTxn" // SlowLogIsWriteCacheTable is used to indicate whether writing to the cache table need to wait for the read lock to expire. @@ -2738,6 +2745,15 @@ const ( // It's controlled by the global variable `tidb_generate_binary_plan`. var GenerateBinaryPlan atomic2.Bool +// JSONSQLWarnForSlowLog helps to print the SQLWarn through the slow log in JSON format. +type JSONSQLWarnForSlowLog struct { + Level string + Message string + // IsExtra means this SQL Warn is expected to be recorded only under some conditions (like in EXPLAIN) and should + // haven't been recorded as a warning now, but we recorded it anyway to help diagnostics. + IsExtra bool `json:",omitempty"` +} + // SlowQueryLogItems is a collection of items that should be included in the // slow query log. type SlowQueryLogItems struct { @@ -2777,6 +2793,7 @@ type SlowQueryLogItems struct { // table -> name -> status StatsLoadStatus map[string]map[string]string IsSyncStatsFailed bool + Warnings []JSONSQLWarnForSlowLog } // SlowLogFormat uses for formatting slow log. @@ -2942,6 +2959,15 @@ func (s *SessionVars) SlowLogFormat(logItems *SlowQueryLogItems) string { writeSlowLogItem(&buf, SlowLogBackoffTotal, strconv.FormatFloat(logItems.BackoffTotal.Seconds(), 'f', -1, 64)) writeSlowLogItem(&buf, SlowLogWriteSQLRespTotal, strconv.FormatFloat(logItems.WriteSQLRespTotal.Seconds(), 'f', -1, 64)) writeSlowLogItem(&buf, SlowLogResultRows, strconv.FormatInt(logItems.ResultRows, 10)) + if len(logItems.Warnings) > 0 { + buf.WriteString(SlowLogRowPrefixStr + SlowLogWarnings + SlowLogSpaceMarkStr) + jsonEncoder := json.NewEncoder(&buf) + jsonEncoder.SetEscapeHTML(false) + err := jsonEncoder.Encode(logItems.Warnings) + if err != nil { + buf.WriteString(err.Error()) + } + } writeSlowLogItem(&buf, SlowLogSucc, strconv.FormatBool(logItems.Succ)) writeSlowLogItem(&buf, SlowLogIsExplicitTxn, strconv.FormatBool(logItems.IsExplicitTxn)) writeSlowLogItem(&buf, SlowLogIsSyncStatsFailed, strconv.FormatBool(logItems.IsSyncStatsFailed)) From e350c35fa6e6e80858d8c11d8df1966a9269bf58 Mon Sep 17 00:00:00 2001 From: time-and-fate <25057648+time-and-fate@users.noreply.github.com> Date: Tue, 13 Dec 2022 17:50:33 +0800 Subject: [PATCH 02/12] update test result --- executor/slow_query_test.go | 4 +- infoschema/tables_test.go | 303 +++++++++++++++++++++++++++++++++++- 2 files changed, 301 insertions(+), 6 deletions(-) diff --git a/executor/slow_query_test.go b/executor/slow_query_test.go index 32412d2f1ac70..069d0ee41275c 100644 --- a/executor/slow_query_test.go +++ b/executor/slow_query_test.go @@ -160,7 +160,7 @@ select * from t;` expectRecordString := `2019-04-28 15:24:04.309074,` + `405888132465033227,root,localhost,0,57,0.12,0.216905,` + `0,0,0,0,0,0,0,0,0,0,0,0,,0,0,0,0,0,0,0.38,0.021,0,0,0,1,637,0,10,10,10,10,100,,,1,42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772,t1:1,t2:2,` + - `0.1,0.2,0.03,127.0.0.1:20160,0.05,0.6,0.8,0.0.0.0:20160,70724,65536,0,0,0,0,0,` + + `0.1,0.2,0.03,127.0.0.1:20160,0.05,0.6,0.8,0.0.0.0:20160,70724,65536,0,0,0,0,0,,` + `Cop_backoff_regionMiss_total_times: 200 Cop_backoff_regionMiss_total_time: 0.2 Cop_backoff_regionMiss_max_time: 0.2 Cop_backoff_regionMiss_max_addr: 127.0.0.1 Cop_backoff_regionMiss_avg_time: 0.2 Cop_backoff_regionMiss_p90_time: 0.2 Cop_backoff_rpcPD_total_times: 200 Cop_backoff_rpcPD_total_time: 0.2 Cop_backoff_rpcPD_max_time: 0.2 Cop_backoff_rpcPD_max_addr: 127.0.0.1 Cop_backoff_rpcPD_avg_time: 0.2 Cop_backoff_rpcPD_p90_time: 0.2 Cop_backoff_rpcTiKV_total_times: 200 Cop_backoff_rpcTiKV_total_time: 0.2 Cop_backoff_rpcTiKV_max_time: 0.2 Cop_backoff_rpcTiKV_max_addr: 127.0.0.1 Cop_backoff_rpcTiKV_avg_time: 0.2 Cop_backoff_rpcTiKV_p90_time: 0.2,` + `0,0,1,0,1,1,0,,60e9378c746d9a2be1c791047e008967cf252eb6de9167ad3aa6098fa2d523f4,` + `,update t set i = 1;,select * from t;` @@ -183,7 +183,7 @@ select * from t;` expectRecordString = `2019-04-28 15:24:04.309074,` + `405888132465033227,root,localhost,0,57,0.12,0.216905,` + `0,0,0,0,0,0,0,0,0,0,0,0,,0,0,0,0,0,0,0.38,0.021,0,0,0,1,637,0,10,10,10,10,100,,,1,42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772,t1:1,t2:2,` + - `0.1,0.2,0.03,127.0.0.1:20160,0.05,0.6,0.8,0.0.0.0:20160,70724,65536,0,0,0,0,0,` + + `0.1,0.2,0.03,127.0.0.1:20160,0.05,0.6,0.8,0.0.0.0:20160,70724,65536,0,0,0,0,0,,` + `Cop_backoff_regionMiss_total_times: 200 Cop_backoff_regionMiss_total_time: 0.2 Cop_backoff_regionMiss_max_time: 0.2 Cop_backoff_regionMiss_max_addr: 127.0.0.1 Cop_backoff_regionMiss_avg_time: 0.2 Cop_backoff_regionMiss_p90_time: 0.2 Cop_backoff_rpcPD_total_times: 200 Cop_backoff_rpcPD_total_time: 0.2 Cop_backoff_rpcPD_max_time: 0.2 Cop_backoff_rpcPD_max_addr: 127.0.0.1 Cop_backoff_rpcPD_avg_time: 0.2 Cop_backoff_rpcPD_p90_time: 0.2 Cop_backoff_rpcTiKV_total_times: 200 Cop_backoff_rpcTiKV_total_time: 0.2 Cop_backoff_rpcTiKV_max_time: 0.2 Cop_backoff_rpcTiKV_max_addr: 127.0.0.1 Cop_backoff_rpcTiKV_avg_time: 0.2 Cop_backoff_rpcTiKV_p90_time: 0.2,` + `0,0,1,0,1,1,0,,60e9378c746d9a2be1c791047e008967cf252eb6de9167ad3aa6098fa2d523f4,` + `,update t set i = 1;,select * from t;` diff --git a/infoschema/tables_test.go b/infoschema/tables_test.go index bd6168d035873..3655e07263c7b 100644 --- a/infoschema/tables_test.go +++ b/infoschema/tables_test.go @@ -535,13 +535,308 @@ func TestSlowQuery(t *testing.T) { tk.MustExec(fmt.Sprintf("set @@tidb_slow_query_file='%v'", slowLogFileName)) tk.MustExec("set time_zone = '+08:00';") re := tk.MustQuery("select * from information_schema.slow_query") - re.Check(testkit.RowsWithSep("|", "2019-02-12 19:33:56.571953|406315658548871171|root|localhost|6|57|0.12|4.895492|0.4|0.2|0.000000003|2|0.000000002|0.00000001|0.000000003|0.19|0.21|0.01|0|0.18|[txnLock]|0.03|0|15|480|1|8|0.3824278|0.161|0.101|0.092|1.71|1|100001|100000|100|10|10|10|100|test||0|42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772|t1:1,t2:2|0.1|0.2|0.03|127.0.0.1:20160|0.05|0.6|0.8|0.0.0.0:20160|70724|65536|0|0|0|0|10||0|1|0|0|1|0|0|abcd|60e9378c746d9a2be1c791047e008967cf252eb6de9167ad3aa6098fa2d523f4||update t set i = 2;|select * from t_slim;", - "2021-09-08|14:39:54.506967|427578666238083075|root|172.16.0.0|40507|0|0|25.571605962|0.002923536|0.006800973|0.002100764|0|0|0|0.000015801|25.542014572|0|0.002294647|0.000605473|12.483|[tikvRPC regionMiss tikvRPC regionMiss regionMiss]|0|0|624|172064|60|0|0|0|0|0|0|0|0|0|0|0|0|0|0|rtdb||0|124acb3a0bec903176baca5f9da00b4e7512a41c93b417923f26502edeb324cc||0|0|0||0|0|0||856544|0|86.635049185|0.015486658|100.054|0|0||0|1|0|0|0|0|0|||||INSERT INTO ...;", + re.Check(testkit.RowsWithSep("|", + "2019-02-12 19:33:56.571953|"+ + "406315658548871171|"+ + "root|"+ + "localhost|"+ + "6|"+ + "57|"+ + "0.12|"+ + "4.895492|"+ + "0.4|"+ + "0.2|"+ + "0.000000003|"+ + "2|"+ + "0.000000002|"+ + "0.00000001|"+ + "0.000000003|"+ + "0.19|"+ + "0.21|"+ + "0.01|"+ + "0|"+ + "0.18|"+ + "[txnLock]|"+ + "0.03|"+ + "0|"+ + "15|"+ + "480|"+ + "1|"+ + "8|"+ + "0.3824278|"+ + "0.161|"+ + "0.101|"+ + "0.092|"+ + "1.71|"+ + "1|"+ + "100001|"+ + "100000|"+ + "100|"+ + "10|"+ + "10|"+ + "10|"+ + "100|"+ + "test|"+ + "|"+ + "0|"+ + "42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772|"+ + "t1:1,t2:2|"+ + "0.1|"+ + "0.2|"+ + "0.03|"+ + "127.0.0.1:20160|"+ + "0.05|"+ + "0.6|"+ + "0.8|"+ + "0.0.0.0:20160|"+ + "70724|"+ + "65536|"+ + "0|"+ + "0|"+ + "0|"+ + "0|"+ + "10|"+ + "|"+ + "|"+ + "0|"+ + "1|"+ + "0|"+ + "0|"+ + "1|"+ + "0|"+ + "0|"+ + "abcd|"+ + "60e9378c746d9a2be1c791047e008967cf252eb6de9167ad3aa6098fa2d523f4|"+ + "|"+ + "update t set i = 2;|"+ + "select * from t_slim;", + "2021-09-08|"+ + "14:39:54.506967|"+ + "427578666238083075|"+ + "root|"+ + "172.16.0.0|"+ + "40507|"+ + "0|"+ + "0|"+ + "25.571605962|"+ + "0.002923536|"+ + "0.006800973|"+ + "0.002100764|"+ + "0|"+ + "0|"+ + "0|"+ + "0.000015801|"+ + "25.542014572|"+ + "0|"+ + "0.002294647|"+ + "0.000605473|"+ + "12.483|"+ + "[tikvRPC regionMiss tikvRPC regionMiss regionMiss]|"+ + "0|"+ + "0|"+ + "624|"+ + "172064|"+ + "60|"+ + "0|"+ + "0|"+ + "0|"+ + "0|"+ + "0|"+ + "0|"+ + "0|"+ + "0|"+ + "0|"+ + "0|"+ + "0|"+ + "0|"+ + "0|"+ + "0|"+ + "rtdb|"+ + "|"+ + "0|"+ + "124acb3a0bec903176baca5f9da00b4e7512a41c93b417923f26502edeb324cc|"+ + "|"+ + "0|"+ + "0|"+ + "0|"+ + "|"+ + "0|"+ + "0|"+ + "0|"+ + "|"+ + "856544|"+ + "0|"+ + "86.635049185|"+ + "0.015486658|"+ + "100.054|"+ + "0|"+ + "0|"+ + "|"+ + "|"+ + "0|"+ + "1|"+ + "0|"+ + "0|"+ + "0|"+ + "0|"+ + "0|"+ + "|"+ + "|"+ + "|"+ + "|"+ + "INSERT INTO ...;", )) tk.MustExec("set time_zone = '+00:00';") re = tk.MustQuery("select * from information_schema.slow_query") - re.Check(testkit.RowsWithSep("|", "2019-02-12 11:33:56.571953|406315658548871171|root|localhost|6|57|0.12|4.895492|0.4|0.2|0.000000003|2|0.000000002|0.00000001|0.000000003|0.19|0.21|0.01|0|0.18|[txnLock]|0.03|0|15|480|1|8|0.3824278|0.161|0.101|0.092|1.71|1|100001|100000|100|10|10|10|100|test||0|42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772|t1:1,t2:2|0.1|0.2|0.03|127.0.0.1:20160|0.05|0.6|0.8|0.0.0.0:20160|70724|65536|0|0|0|0|10||0|1|0|0|1|0|0|abcd|60e9378c746d9a2be1c791047e008967cf252eb6de9167ad3aa6098fa2d523f4||update t set i = 2;|select * from t_slim;", - "2021-09-08|06:39:54.506967|427578666238083075|root|172.16.0.0|40507|0|0|25.571605962|0.002923536|0.006800973|0.002100764|0|0|0|0.000015801|25.542014572|0|0.002294647|0.000605473|12.483|[tikvRPC regionMiss tikvRPC regionMiss regionMiss]|0|0|624|172064|60|0|0|0|0|0|0|0|0|0|0|0|0|0|0|rtdb||0|124acb3a0bec903176baca5f9da00b4e7512a41c93b417923f26502edeb324cc||0|0|0||0|0|0||856544|0|86.635049185|0.015486658|100.054|0|0||0|1|0|0|0|0|0|||||INSERT INTO ...;", + re.Check(testkit.RowsWithSep("|", "2019-02-12 11:33:56.571953|"+ + "406315658548871171|"+ + "root|"+ + "localhost|"+ + "6|"+ + "57|"+ + "0.12|"+ + "4.895492|"+ + "0.4|"+ + "0.2|"+ + "0.000000003|"+ + "2|"+ + "0.000000002|"+ + "0.00000001|"+ + "0.000000003|"+ + "0.19|"+ + "0.21|"+ + "0.01|"+ + "0|"+ + "0.18|"+ + "[txnLock]|"+ + "0.03|"+ + "0|"+ + "15|"+ + "480|"+ + "1|"+ + "8|"+ + "0.3824278|"+ + "0.161|"+ + "0.101|"+ + "0.092|"+ + "1.71|"+ + "1|"+ + "100001|"+ + "100000|"+ + "100|"+ + "10|"+ + "10|"+ + "10|"+ + "100|"+ + "test|"+ + "|"+ + "0|"+ + "42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772|"+ + "t1:1,t2:2|"+ + "0.1|"+ + "0.2|"+ + "0.03|"+ + "127.0.0.1:20160|"+ + "0.05|"+ + "0.6|"+ + "0.8|"+ + "0.0.0.0:20160|"+ + "70724|"+ + "65536|"+ + "0|"+ + "0|"+ + "0|"+ + "0|"+ + "10|"+ + "|"+ + "|"+ + "0|"+ + "1|"+ + "0|"+ + "0|"+ + "1|"+ + "0|"+ + "0|"+ + "abcd|"+ + "60e9378c746d9a2be1c791047e008967cf252eb6de9167ad3aa6098fa2d523f4|"+ + "|"+ + "update t set i = 2;|"+ + "select * from t_slim;", + "2021-09-08|"+ + "06:39:54.506967|"+ + "427578666238083075|"+ + "root|"+ + "172.16.0.0|"+ + "40507|"+ + "0|"+ + "0|"+ + "25.571605962|"+ + "0.002923536|"+ + "0.006800973|"+ + "0.002100764|"+ + "0|"+ + "0|"+ + "0|"+ + "0.000015801|"+ + "25.542014572|"+ + "0|"+ + "0.002294647|"+ + "0.000605473|"+ + "12.483|"+ + "[tikvRPC regionMiss tikvRPC regionMiss regionMiss]|"+ + "0|"+ + "0|"+ + "624|"+ + "172064|"+ + "60|"+ + "0|"+ + "0|"+ + "0|"+ + "0|"+ + "0|"+ + "0|"+ + "0|"+ + "0|"+ + "0|"+ + "0|"+ + "0|"+ + "0|"+ + "0|"+ + "0|"+ + "rtdb|"+ + "|"+ + "0|"+ + "124acb3a0bec903176baca5f9da00b4e7512a41c93b417923f26502edeb324cc|"+ + "|"+ + "0|"+ + "0|"+ + "0|"+ + "|"+ + "0|"+ + "0|"+ + "0|"+ + "|"+ + "856544|"+ + "0|"+ + "86.635049185|"+ + "0.015486658|"+ + "100.054|"+ + "0|"+ + "0|"+ + "|"+ + "|"+ + "0|"+ + "1|"+ + "0|"+ + "0|"+ + "0|"+ + "0|"+ + "0|"+ + "|"+ + "|"+ + "|"+ + "|"+ + "INSERT INTO ...;", )) // Test for long query. From b3ad2f5d39e75a9d8aa13d72fe153720a7229919 Mon Sep 17 00:00:00 2001 From: time-and-fate <25057648+time-and-fate@users.noreply.github.com> Date: Tue, 13 Dec 2022 19:04:39 +0800 Subject: [PATCH 03/12] add comments and update test result --- planner/core/testdata/integration_suite_out.json | 7 +++++-- sessionctx/variable/session.go | 2 ++ 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/planner/core/testdata/integration_suite_out.json b/planner/core/testdata/integration_suite_out.json index 14c04c6cfb0ab..d61124de927bd 100644 --- a/planner/core/testdata/integration_suite_out.json +++ b/planner/core/testdata/integration_suite_out.json @@ -1969,7 +1969,8 @@ " └─TableRangeScan_8 3333.33 923531.15 cop[tikv] table:t range:(1,+inf], keep order:false, stats:pseudo" ], "Warnings": [ - "Note 1105 [t] remain after pruning paths for t given Prop{SortItems: [], TaskTp: rootTask}" + "Note 1105 [t] remain after pruning paths for t given Prop{SortItems: [], TaskTp: rootTask}", + "Note 1105 [t,f,f_g] remain after pruning paths for t given Prop{SortItems: [{test.t.f asc}], TaskTp: rootTask}" ] }, { @@ -2014,7 +2015,8 @@ " └─TableRowIDScan_12(Probe) 10.00 2770.59 cop[tikv] table:t keep order:false, stats:pseudo" ], "Warnings": [ - "Note 1105 [t,g] remain after pruning paths for t given Prop{SortItems: [], TaskTp: rootTask}" + "Note 1105 [t,g] remain after pruning paths for t given Prop{SortItems: [], TaskTp: rootTask}", + "Note 1105 [t,f_g,g] remain after pruning paths for t given Prop{SortItems: [{test.t.f asc}], TaskTp: rootTask}" ] }, { @@ -2026,6 +2028,7 @@ "└─TableRowIDScan_13(Probe) 10.00 2770.59 cop[tikv] table:t keep order:false, stats:pseudo" ], "Warnings": [ + "Note 1105 [t] remain after pruning paths for t given Prop{SortItems: [], TaskTp: rootTask}", "Note 1105 [t,c_d_e] remain after pruning paths for t given Prop{SortItems: [{test.t.c asc} {test.t.e asc}], TaskTp: rootTask}" ] } diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index e7b0a848a9a71..872d864d7ae86 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -2732,6 +2732,8 @@ const ( SlowLogBackoffDetail = "Backoff_Detail" // SlowLogResultRows is the row count of the SQL result. SlowLogResultRows = "Result_rows" + // SlowLogWarnings is the warnings generated during executing the statement. + // Note that some extra warnings would also be printed through slow log. SlowLogWarnings = "Warnings" // SlowLogIsExplicitTxn is used to indicate whether this sql execute in explicit transaction or not. SlowLogIsExplicitTxn = "IsExplicitTxn" From b2e6bdc38a3c4d8986d63b231f84cad143450b6d Mon Sep 17 00:00:00 2001 From: time-and-fate <25057648+time-and-fate@users.noreply.github.com> Date: Tue, 13 Dec 2022 19:10:47 +0800 Subject: [PATCH 04/12] fmt --- sessionctx/variable/session.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 872d864d7ae86..7388d046a656e 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -2734,7 +2734,7 @@ const ( SlowLogResultRows = "Result_rows" // SlowLogWarnings is the warnings generated during executing the statement. // Note that some extra warnings would also be printed through slow log. - SlowLogWarnings = "Warnings" + SlowLogWarnings = "Warnings" // SlowLogIsExplicitTxn is used to indicate whether this sql execute in explicit transaction or not. SlowLogIsExplicitTxn = "IsExplicitTxn" // SlowLogIsWriteCacheTable is used to indicate whether writing to the cache table need to wait for the read lock to expire. From a2caa09e5e349cb68c0d6b2ac939a341ce119656 Mon Sep 17 00:00:00 2001 From: time-and-fate <25057648+time-and-fate@users.noreply.github.com> Date: Tue, 13 Dec 2022 20:28:39 +0800 Subject: [PATCH 05/12] better fmt --- infoschema/tables_test.go | 149 +++++++++++++++++++------------------- 1 file changed, 75 insertions(+), 74 deletions(-) diff --git a/infoschema/tables_test.go b/infoschema/tables_test.go index 3655e07263c7b..043cad3207340 100644 --- a/infoschema/tables_test.go +++ b/infoschema/tables_test.go @@ -688,80 +688,81 @@ func TestSlowQuery(t *testing.T) { )) tk.MustExec("set time_zone = '+00:00';") re = tk.MustQuery("select * from information_schema.slow_query") - re.Check(testkit.RowsWithSep("|", "2019-02-12 11:33:56.571953|"+ - "406315658548871171|"+ - "root|"+ - "localhost|"+ - "6|"+ - "57|"+ - "0.12|"+ - "4.895492|"+ - "0.4|"+ - "0.2|"+ - "0.000000003|"+ - "2|"+ - "0.000000002|"+ - "0.00000001|"+ - "0.000000003|"+ - "0.19|"+ - "0.21|"+ - "0.01|"+ - "0|"+ - "0.18|"+ - "[txnLock]|"+ - "0.03|"+ - "0|"+ - "15|"+ - "480|"+ - "1|"+ - "8|"+ - "0.3824278|"+ - "0.161|"+ - "0.101|"+ - "0.092|"+ - "1.71|"+ - "1|"+ - "100001|"+ - "100000|"+ - "100|"+ - "10|"+ - "10|"+ - "10|"+ - "100|"+ - "test|"+ - "|"+ - "0|"+ - "42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772|"+ - "t1:1,t2:2|"+ - "0.1|"+ - "0.2|"+ - "0.03|"+ - "127.0.0.1:20160|"+ - "0.05|"+ - "0.6|"+ - "0.8|"+ - "0.0.0.0:20160|"+ - "70724|"+ - "65536|"+ - "0|"+ - "0|"+ - "0|"+ - "0|"+ - "10|"+ - "|"+ - "|"+ - "0|"+ - "1|"+ - "0|"+ - "0|"+ - "1|"+ - "0|"+ - "0|"+ - "abcd|"+ - "60e9378c746d9a2be1c791047e008967cf252eb6de9167ad3aa6098fa2d523f4|"+ - "|"+ - "update t set i = 2;|"+ - "select * from t_slim;", + re.Check(testkit.RowsWithSep("|", + "2019-02-12 11:33:56.571953|"+ + "406315658548871171|"+ + "root|"+ + "localhost|"+ + "6|"+ + "57|"+ + "0.12|"+ + "4.895492|"+ + "0.4|"+ + "0.2|"+ + "0.000000003|"+ + "2|"+ + "0.000000002|"+ + "0.00000001|"+ + "0.000000003|"+ + "0.19|"+ + "0.21|"+ + "0.01|"+ + "0|"+ + "0.18|"+ + "[txnLock]|"+ + "0.03|"+ + "0|"+ + "15|"+ + "480|"+ + "1|"+ + "8|"+ + "0.3824278|"+ + "0.161|"+ + "0.101|"+ + "0.092|"+ + "1.71|"+ + "1|"+ + "100001|"+ + "100000|"+ + "100|"+ + "10|"+ + "10|"+ + "10|"+ + "100|"+ + "test|"+ + "|"+ + "0|"+ + "42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772|"+ + "t1:1,t2:2|"+ + "0.1|"+ + "0.2|"+ + "0.03|"+ + "127.0.0.1:20160|"+ + "0.05|"+ + "0.6|"+ + "0.8|"+ + "0.0.0.0:20160|"+ + "70724|"+ + "65536|"+ + "0|"+ + "0|"+ + "0|"+ + "0|"+ + "10|"+ + "|"+ + "|"+ + "0|"+ + "1|"+ + "0|"+ + "0|"+ + "1|"+ + "0|"+ + "0|"+ + "abcd|"+ + "60e9378c746d9a2be1c791047e008967cf252eb6de9167ad3aa6098fa2d523f4|"+ + "|"+ + "update t set i = 2;|"+ + "select * from t_slim;", "2021-09-08|"+ "06:39:54.506967|"+ "427578666238083075|"+ From 87d3badbfddeba4141308c26442749f98071c945 Mon Sep 17 00:00:00 2001 From: time-and-fate <25057648+time-and-fate@users.noreply.github.com> Date: Tue, 13 Dec 2022 20:57:38 +0800 Subject: [PATCH 06/12] remove optiminfo --- sessionctx/stmtctx/stmtctx.go | 2 -- 1 file changed, 2 deletions(-) diff --git a/sessionctx/stmtctx/stmtctx.go b/sessionctx/stmtctx/stmtctx.go index 7dbe3f78e6daa..3142588dcc1fc 100644 --- a/sessionctx/stmtctx/stmtctx.go +++ b/sessionctx/stmtctx/stmtctx.go @@ -305,8 +305,6 @@ type StatementContext struct { LogOnExceed [2]memory.LogOnExceed } - // OptimInfo maps Plan.ID() to optimization information when generating Plan. - OptimInfo map[int]string // InVerboseExplain indicates the statement is "explain format='verbose' ...". InVerboseExplain bool From d3d78c6156311963f98d31b63384039ccf224034 Mon Sep 17 00:00:00 2001 From: time-and-fate <25057648+time-and-fate@users.noreply.github.com> Date: Wed, 14 Dec 2022 20:23:59 +0800 Subject: [PATCH 07/12] fix and add test --- executor/main_test.go | 3 ++ executor/slow_query.go | 3 ++ executor/slow_query_sql_test.go | 59 +++++++++++++++++++++ executor/testdata/slow_query_suite_in.json | 18 +++++++ executor/testdata/slow_query_suite_out.json | 51 ++++++++++++++++++ planner/core/find_best_task.go | 4 +- 6 files changed, 136 insertions(+), 2 deletions(-) create mode 100644 executor/testdata/slow_query_suite_in.json create mode 100644 executor/testdata/slow_query_suite_out.json diff --git a/executor/main_test.go b/executor/main_test.go index 512eced5b50fa..4bf8cfb8f1bac 100644 --- a/executor/main_test.go +++ b/executor/main_test.go @@ -33,6 +33,7 @@ var prepareMergeSuiteData testdata.TestData var aggMergeSuiteData testdata.TestData var executorSuiteData testdata.TestData var pointGetSuiteData testdata.TestData +var slowQuerySuiteData testdata.TestData func TestMain(m *testing.M) { testsetup.SetupForCommonTest() @@ -40,10 +41,12 @@ func TestMain(m *testing.M) { testDataMap.LoadTestSuiteData("testdata", "executor_suite") testDataMap.LoadTestSuiteData("testdata", "prepare_suite") testDataMap.LoadTestSuiteData("testdata", "point_get_suite") + testDataMap.LoadTestSuiteData("testdata", "slow_query_suite") aggMergeSuiteData = testDataMap["agg_suite"] executorSuiteData = testDataMap["executor_suite"] prepareMergeSuiteData = testDataMap["prepare_suite"] pointGetSuiteData = testDataMap["point_get_suite"] + slowQuerySuiteData = testDataMap["slow_query_suite"] autoid.SetStep(5000) config.UpdateGlobal(func(conf *config.Config) { diff --git a/executor/slow_query.go b/executor/slow_query.go index 7b0bd9df07996..b83a480f85857 100644 --- a/executor/slow_query.go +++ b/executor/slow_query.go @@ -622,6 +622,9 @@ func (e *slowQueryRetriever) parseLog(ctx context.Context, sctx sessionctx.Conte valid = e.setColumnValue(sctx, row, tz, variable.SlowLogHostStr, host, e.checker, fileLine) } else if strings.HasPrefix(line, variable.SlowLogCopBackoffPrefix) { valid = e.setColumnValue(sctx, row, tz, variable.SlowLogBackoffDetail, line, e.checker, fileLine) + } else if strings.HasPrefix(line, variable.SlowLogWarnings) { + line = line[len(variable.SlowLogWarnings+variable.SlowLogSpaceMarkStr):] + valid = e.setColumnValue(sctx, row, tz, variable.SlowLogWarnings, line, e.checker, fileLine) } else { fields, values := splitByColon(line) for i := 0; i < len(fields); i++ { diff --git a/executor/slow_query_sql_test.go b/executor/slow_query_sql_test.go index f7029454023fa..94ed928846812 100644 --- a/executor/slow_query_sql_test.go +++ b/executor/slow_query_sql_test.go @@ -16,6 +16,7 @@ package executor_test import ( "fmt" + "github.com/pingcap/tidb/parser/model" "math" "os" "testing" @@ -283,3 +284,61 @@ func TestIssue37066(t *testing.T) { } } } + +func TestWarningsInSlowQuery(t *testing.T) { + // Prepare the slow log + originCfg := config.GetGlobalConfig() + newCfg := *originCfg + f, err := os.CreateTemp("", "tidb-slow-*.log") + require.NoError(t, err) + newCfg.Log.SlowQueryFile = f.Name() + config.StoreGlobalConfig(&newCfg) + defer func() { + config.StoreGlobalConfig(originCfg) + require.NoError(t, f.Close()) + require.NoError(t, os.Remove(newCfg.Log.SlowQueryFile)) + }() + require.NoError(t, logutil.InitLogger(newCfg.Log.ToLogConfig())) + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec(fmt.Sprintf("set @@tidb_slow_query_file='%v'", f.Name())) + tk.MustExec("set tidb_slow_log_threshold=0;") + defer func() { + tk.MustExec("set tidb_slow_log_threshold=300;") + }() + + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int, b int, c int, d int, e int, f int, g int, h set('11', '22', '33')," + + "primary key (a), unique key c_d_e (c, d, e), unique key f (f), unique key f_g (f, g), key g (g))") + tbl, err := dom.InfoSchema().TableByName(model.CIStr{O: "test", L: "test"}, model.CIStr{O: "t", L: "t"}) + require.NoError(t, err) + tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{Count: 1, Available: true} + + var input []string + var output []struct { + SQL string + Result string + } + slowQuerySuiteData.LoadTestCases(t, &input, &output) + for i, test := range input { + comment := fmt.Sprintf("case:%v sql:%s", i, test) + if len(test) < 6 || test[:6] != "select" { + tk.MustExec(test) + } else { + tk.MustQuery(test) + } + res := testdata.ConvertRowsToStrings( + tk.MustQuery("select warnings from information_schema.slow_query " + + `where query = "` + test + `;" ` + + "order by time desc limit 1").Rows(), + ) + require.Lenf(t, res, 1, comment) + + testdata.OnRecord(func() { + output[i].SQL = test + output[i].Result = res[0] + }) + require.Equal(t, output[i].Result, res[0]) + } +} diff --git a/executor/testdata/slow_query_suite_in.json b/executor/testdata/slow_query_suite_in.json new file mode 100644 index 0000000000000..6a5ef203de542 --- /dev/null +++ b/executor/testdata/slow_query_suite_in.json @@ -0,0 +1,18 @@ +[ + { + "name": "TestWarningsInSlowQuery", + "cases": [ + "insert into t(a) value (1)", + "create binding for select * from t where c = 10 using select * from t use index (c_d_e) where c = 10", + "select * from t where c = 10", + "select /*+ READ_FROM_STORAGE(TIKV[t]) */ * from t where a > 1 order by f", + "select /*+ READ_FROM_STORAGE(TIKV[t]) */ * from t where f > 1", + "select /*+ READ_FROM_STORAGE(TIKV[t]) */ f from t where f > 1", + "select /*+ READ_FROM_STORAGE(TIKV[t]) */ * from t where f > 3 and g = 5", + "select /*+ READ_FROM_STORAGE(TIKV[t]) */ * from t where g = 5 order by f", + "select /*+ READ_FROM_STORAGE(TIKV[t]) */ * from t where d = 3 order by c, e", + "select /*+ READ_FROM_STORAGE(TIKV[t]) */ * from t where h = '11,22'", + "select /*+ READ_FROM_STORAGE(TIKV[t]) */ * from t where a > rand()*100" + ] + } +] diff --git a/executor/testdata/slow_query_suite_out.json b/executor/testdata/slow_query_suite_out.json new file mode 100644 index 0000000000000..4f3f7dbb4fc1a --- /dev/null +++ b/executor/testdata/slow_query_suite_out.json @@ -0,0 +1,51 @@ +[ + { + "Name": "TestWarningsInSlowQuery", + "Cases": [ + { + "SQL": "insert into t(a) value (1)", + "Result": "" + }, + { + "SQL": "create binding for select * from t where c = 10 using select * from t use index (c_d_e) where c = 10", + "Result": "" + }, + { + "SQL": "select * from t where c = 10", + "Result": "[{\"Level\":\"Note\",\"Message\":\"Using the bindSQL: SELECT * FROM `test`.`t` USE INDEX (`c_d_e`) WHERE `c` = 10\",\"IsExtra\":true}]" + }, + { + "SQL": "select /*+ READ_FROM_STORAGE(TIKV[t]) */ * from t where a > 1 order by f", + "Result": "[{\"Level\":\"Warning\",\"Message\":\"Expression about 'test.t.h' can not be pushed to TiFlash because it contains unsupported calculation of type 'set'.\",\"IsExtra\":true},{\"Level\":\"Note\",\"Message\":\"[t,t(tiflash)] remain after pruning paths for t given Prop{SortItems: [], TaskTp: rootTask}\",\"IsExtra\":true},{\"Level\":\"Warning\",\"Message\":\"Expression about 'test.t.h' can not be pushed to TiFlash because it contains unsupported calculation of type 'set'.\",\"IsExtra\":true},{\"Level\":\"Note\",\"Message\":\"[t,t(tiflash),f,f_g] remain after pruning paths for t given Prop{SortItems: [{test.t.f asc}], TaskTp: rootTask}\",\"IsExtra\":true}]" + }, + { + "SQL": "select /*+ READ_FROM_STORAGE(TIKV[t]) */ * from t where f > 1", + "Result": "[{\"Level\":\"Warning\",\"Message\":\"Expression about 'test.t.h' can not be pushed to TiFlash because it contains unsupported calculation of type 'set'.\",\"IsExtra\":true},{\"Level\":\"Note\",\"Message\":\"[t,t(tiflash),f,f_g] remain after pruning paths for t given Prop{SortItems: [], TaskTp: rootTask}\",\"IsExtra\":true}]" + }, + { + "SQL": "select /*+ READ_FROM_STORAGE(TIKV[t]) */ f from t where f > 1", + "Result": "[{\"Level\":\"Note\",\"Message\":\"[t(tiflash),f,f_g] remain after pruning paths for t given Prop{SortItems: [], TaskTp: rootTask}\",\"IsExtra\":true}]" + }, + { + "SQL": "select /*+ READ_FROM_STORAGE(TIKV[t]) */ * from t where f > 3 and g = 5", + "Result": "[{\"Level\":\"Warning\",\"Message\":\"Expression about 'test.t.h' can not be pushed to TiFlash because it contains unsupported calculation of type 'set'.\",\"IsExtra\":true},{\"Level\":\"Note\",\"Message\":\"[t,t(tiflash),f_g,g] remain after pruning paths for t given Prop{SortItems: [], TaskTp: rootTask}\",\"IsExtra\":true}]" + }, + { + "SQL": "select /*+ READ_FROM_STORAGE(TIKV[t]) */ * from t where g = 5 order by f", + "Result": "[{\"Level\":\"Warning\",\"Message\":\"Expression about 'test.t.h' can not be pushed to TiFlash because it contains unsupported calculation of type 'set'.\",\"IsExtra\":true},{\"Level\":\"Note\",\"Message\":\"[t,t(tiflash),g] remain after pruning paths for t given Prop{SortItems: [], TaskTp: rootTask}\",\"IsExtra\":true},{\"Level\":\"Warning\",\"Message\":\"Expression about 'test.t.h' can not be pushed to TiFlash because it contains unsupported calculation of type 'set'.\",\"IsExtra\":true},{\"Level\":\"Note\",\"Message\":\"[t,t(tiflash),f_g,g] remain after pruning paths for t given Prop{SortItems: [{test.t.f asc}], TaskTp: rootTask}\",\"IsExtra\":true}]" + }, + { + "SQL": "select /*+ READ_FROM_STORAGE(TIKV[t]) */ * from t where d = 3 order by c, e", + "Result": "[{\"Level\":\"Warning\",\"Message\":\"Expression about 'test.t.h' can not be pushed to TiFlash because it contains unsupported calculation of type 'set'.\",\"IsExtra\":true},{\"Level\":\"Note\",\"Message\":\"[t,t(tiflash)] remain after pruning paths for t given Prop{SortItems: [], TaskTp: rootTask}\",\"IsExtra\":true},{\"Level\":\"Warning\",\"Message\":\"Expression about 'test.t.h' can not be pushed to TiFlash because it contains unsupported calculation of type 'set'.\",\"IsExtra\":true},{\"Level\":\"Note\",\"Message\":\"[t,t(tiflash),c_d_e] remain after pruning paths for t given Prop{SortItems: [{test.t.c asc} {test.t.e asc}], TaskTp: rootTask}\",\"IsExtra\":true}]" + }, + { + "SQL": "select /*+ READ_FROM_STORAGE(TIKV[t]) */ * from t where h = '11,22'", + "Result": "[{\"Level\":\"Warning\",\"Message\":\"Expression about 'test.t.h' can not be pushed to TiFlash because it contains unsupported calculation of type 'set'.\",\"IsExtra\":true},{\"Level\":\"Warning\",\"Message\":\"Expression about 'test.t.h' can not be pushed to TiFlash because it contains unsupported calculation of type 'set'.\",\"IsExtra\":true},{\"Level\":\"Note\",\"Message\":\"[t,t(tiflash)] remain after pruning paths for t given Prop{SortItems: [], TaskTp: rootTask}\",\"IsExtra\":true}]" + }, + { + "SQL": "select /*+ READ_FROM_STORAGE(TIKV[t]) */ * from t where a > rand()*100", + "Result": "[{\"Level\":\"Warning\",\"Message\":\"Scalar function 'rand'(signature: Rand, return type: double) is not supported to push down to storage layer now.\",\"IsExtra\":true},{\"Level\":\"Warning\",\"Message\":\"Expression about 'test.t.h' can not be pushed to TiFlash because it contains unsupported calculation of type 'set'.\",\"IsExtra\":true},{\"Level\":\"Warning\",\"Message\":\"Scalar function 'rand'(signature: Rand, return type: double) is not supported to push down to tiflash now.\",\"IsExtra\":true},{\"Level\":\"Note\",\"Message\":\"[t,t(tiflash)] remain after pruning paths for t given Prop{SortItems: [], TaskTp: rootTask}\",\"IsExtra\":true}]" + } + ] + } +] diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index bc869c62184ad..639bc15dbdc98 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -742,7 +742,7 @@ func (ds *DataSource) skylinePruning(prop *property.PhysicalProperty) []*candida } func (ds *DataSource) getPruningInfo(candidates []*candidatePath, prop *property.PhysicalProperty) string { - if !ds.ctx.GetSessionVars().StmtCtx.InVerboseExplain || len(candidates) == len(ds.possibleAccessPaths) { + if len(candidates) == len(ds.possibleAccessPaths) { return "" } if len(candidates) == 1 && len(candidates[0].path.Ranges) == 0 { @@ -890,7 +890,7 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, planCounter defer func() { if err == nil && t != nil && !t.invalid() && pruningInfo != "" { warnErr := errors.New(pruningInfo) - if ds.ctx.GetSessionVars().StmtCtx.InExplainStmt { + if ds.ctx.GetSessionVars().StmtCtx.InVerboseExplain { ds.ctx.GetSessionVars().StmtCtx.AppendNote(warnErr) } else { ds.ctx.GetSessionVars().StmtCtx.AppendExtraNote(warnErr) From 04549a61e87d5a97078638b9f46f68363272c436 Mon Sep 17 00:00:00 2001 From: time-and-fate <25057648+time-and-fate@users.noreply.github.com> Date: Wed, 14 Dec 2022 22:03:14 +0800 Subject: [PATCH 08/12] fix fmt --- executor/slow_query_sql_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/executor/slow_query_sql_test.go b/executor/slow_query_sql_test.go index 94ed928846812..cdf13f2a9ccf9 100644 --- a/executor/slow_query_sql_test.go +++ b/executor/slow_query_sql_test.go @@ -16,7 +16,6 @@ package executor_test import ( "fmt" - "github.com/pingcap/tidb/parser/model" "math" "os" "testing" @@ -24,6 +23,7 @@ import ( "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/parser/auth" + "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/testkit/testdata" "github.com/pingcap/tidb/util/logutil" From b63cfef6f160a5641f4edd1ada59dfad66d18128 Mon Sep 17 00:00:00 2001 From: time-and-fate <25057648+time-and-fate@users.noreply.github.com> Date: Tue, 20 Dec 2022 16:17:07 +0800 Subject: [PATCH 09/12] add comments and add test case --- executor/testdata/slow_query_suite_in.json | 1 + executor/testdata/slow_query_suite_out.json | 4 ++++ sessionctx/variable/session.go | 1 + 3 files changed, 6 insertions(+) diff --git a/executor/testdata/slow_query_suite_in.json b/executor/testdata/slow_query_suite_in.json index 6a5ef203de542..4b2c95b6aa378 100644 --- a/executor/testdata/slow_query_suite_in.json +++ b/executor/testdata/slow_query_suite_in.json @@ -3,6 +3,7 @@ "name": "TestWarningsInSlowQuery", "cases": [ "insert into t(a) value (1)", + "select /*+ READ_FROM_STORAGE(TIKV[t]) */ a/0 from t", "create binding for select * from t where c = 10 using select * from t use index (c_d_e) where c = 10", "select * from t where c = 10", "select /*+ READ_FROM_STORAGE(TIKV[t]) */ * from t where a > 1 order by f", diff --git a/executor/testdata/slow_query_suite_out.json b/executor/testdata/slow_query_suite_out.json index 4f3f7dbb4fc1a..adf6606709859 100644 --- a/executor/testdata/slow_query_suite_out.json +++ b/executor/testdata/slow_query_suite_out.json @@ -6,6 +6,10 @@ "SQL": "insert into t(a) value (1)", "Result": "" }, + { + "SQL": "select /*+ READ_FROM_STORAGE(TIKV[t]) */ a/0 from t", + "Result": "[{\"Level\":\"Warning\",\"Message\":\"Division by 0\"}]" + }, { "SQL": "create binding for select * from t where c = 10 using select * from t use index (c_d_e) where c = 10", "Result": "" diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index bff2af3294812..44f28fd587001 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -2965,6 +2965,7 @@ func (s *SessionVars) SlowLogFormat(logItems *SlowQueryLogItems) string { buf.WriteString(SlowLogRowPrefixStr + SlowLogWarnings + SlowLogSpaceMarkStr) jsonEncoder := json.NewEncoder(&buf) jsonEncoder.SetEscapeHTML(false) + // Note that the Encode() will append a '\n' so we don't need to add another. err := jsonEncoder.Encode(logItems.Warnings) if err != nil { buf.WriteString(err.Error()) From ece243ef817118f6e66453d2af48d7987bbd3e09 Mon Sep 17 00:00:00 2001 From: time-and-fate <25057648+time-and-fate@users.noreply.github.com> Date: Tue, 20 Dec 2022 19:55:51 +0800 Subject: [PATCH 10/12] update test result --- infoschema/tables_test.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/infoschema/tables_test.go b/infoschema/tables_test.go index d4d4c4fa588f7..d584296069bbf 100644 --- a/infoschema/tables_test.go +++ b/infoschema/tables_test.go @@ -593,6 +593,7 @@ func TestSlowQuery(t *testing.T) { "0", "10", "", + "", "0", "1", "0", @@ -666,6 +667,7 @@ func TestSlowQuery(t *testing.T) { "0", "0", "", + "", "0", "1", "0", From 70760b8b1021e6d558ba7446c89c4776df3c200c Mon Sep 17 00:00:00 2001 From: time-and-fate <25057648+time-and-fate@users.noreply.github.com> Date: Wed, 4 Jan 2023 14:57:55 +0800 Subject: [PATCH 11/12] add nil check --- executor/adapter.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/executor/adapter.go b/executor/adapter.go index afbee5c989a8e..ed2c0566e0f81 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -1611,8 +1611,10 @@ func (a *ExecStmt) LogSlowQuery(txnTS uint64, succ bool, hasMoreResults bool) { } func extractMsgFromSQLWarn(SQLWarn *stmtctx.SQLWarn) string { + // Currently, this function is only used in extractMsgFromSQLWarn(). + // extractMsgFromSQLWarn() can make sure SQLWarn is not nil so no need to add a nil check here. warn := errors.Cause(SQLWarn.Err) - if x, ok := warn.(*terror.Error); ok { + if x, ok := warn.(*terror.Error); ok && x != nil { sqlErr := terror.ToSQLError(x) return sqlErr.Message } From 0531b23b0f01dc7e0e68cb92659a23d1a48a4dbd Mon Sep 17 00:00:00 2001 From: Zhou Kunqin <25057648+time-and-fate@users.noreply.github.com> Date: Thu, 5 Jan 2023 14:25:30 +0800 Subject: [PATCH 12/12] Apply suggestions from code review Co-authored-by: Yifan Xu <30385241+xuyifangreeneyes@users.noreply.github.com> --- executor/adapter.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/executor/adapter.go b/executor/adapter.go index ed2c0566e0f81..444c358f96c97 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -1611,8 +1611,8 @@ func (a *ExecStmt) LogSlowQuery(txnTS uint64, succ bool, hasMoreResults bool) { } func extractMsgFromSQLWarn(SQLWarn *stmtctx.SQLWarn) string { - // Currently, this function is only used in extractMsgFromSQLWarn(). - // extractMsgFromSQLWarn() can make sure SQLWarn is not nil so no need to add a nil check here. + // Currently, this function is only used in collectWarningsForSlowLog. + // collectWarningsForSlowLog can make sure SQLWarn is not nil so no need to add a nil check here. warn := errors.Cause(SQLWarn.Err) if x, ok := warn.(*terror.Error); ok && x != nil { sqlErr := terror.ToSQLError(x)