diff --git a/expression/expression.go b/expression/expression.go index 352f105c52d65..6d7eb080b29fc 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_test.go b/infoschema/tables_test.go index ebf3e9b535893..d4d4c4fa588f7 100644 --- a/infoschema/tables_test.go +++ b/infoschema/tables_test.go @@ -531,18 +531,166 @@ func TestSlowQuery(t *testing.T) { slowLogFileName := "tidb_slow.log" prepareSlowLogfile(t, slowLogFileName) defer func() { require.NoError(t, os.Remove(slowLogFileName)) }() + expectedRes := [][]interface{}{ + {"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(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(expectedRes) + 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 ...;", - )) + expectedRes[0][0] = "2019-02-12 11:33:56.571953" + expectedRes[1][0] = "2021-09-08 06:39:54.506967" + re.Check(expectedRes) // Test for long query. f, err := os.OpenFile(slowLogFileName, os.O_CREATE|os.O_WRONLY, 0644) diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index f11f60b95cfe5..8feb357745853 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -914,11 +914,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, @@ -938,11 +933,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..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 { @@ -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.InVerboseExplain { + 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/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/planner/optimize.go b/planner/optimize.go index 0f37a59788a5c..1acd83d312516 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..47159dc8f8a60 100644 --- a/sessionctx/stmtctx/stmtctx.go +++ b/sessionctx/stmtctx/stmtctx.go @@ -209,8 +209,15 @@ type StatementContext struct { copied uint64 touched uint64 - message string - warnings []SQLWarn + message string + warnings []SQLWarn + // extraWarnings record the extra warnings and are only used by the slow log only now. + // If a warning is expected to be output only under some conditions (like in EXPLAIN or EXPLAIN VERBOSE) but it's + // not under such conditions now, it is considered as an extra warning. + // extraWarnings would not be printed through SHOW WARNINGS, 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 } @@ -299,8 +306,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 @@ -812,6 +817,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 fe4972fb5ff3a..695d0ad48a5a5 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -1482,8 +1482,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)) } }