Skip to content

Commit

Permalink
*: extend skyline pruning diagnostic info and add extra warnings (#39894
Browse files Browse the repository at this point in the history
)

ref #39893
  • Loading branch information
time-and-fate authored Dec 20, 2022
1 parent 6ab3803 commit 017901d
Show file tree
Hide file tree
Showing 11 changed files with 278 additions and 61 deletions.
23 changes: 16 additions & 7 deletions expression/expression.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
}
Expand Down Expand Up @@ -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
}
Expand Down
160 changes: 154 additions & 6 deletions infoschema/tables_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down
10 changes: 0 additions & 10 deletions planner/core/common_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -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,
Expand All @@ -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) {
Expand Down
5 changes: 4 additions & 1 deletion planner/core/exhaust_physical_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
}
Expand Down
10 changes: 6 additions & 4 deletions planner/core/find_best_task.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Expand Down Expand Up @@ -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
}
}()

Expand Down
52 changes: 28 additions & 24 deletions planner/core/stats.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
}
Expand Down Expand Up @@ -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
}
Expand Down
9 changes: 7 additions & 2 deletions planner/core/task.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
}
Expand Down
7 changes: 5 additions & 2 deletions planner/core/testdata/integration_suite_out.json
Original file line number Diff line number Diff line change
Expand Up @@ -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}"
]
},
{
Expand Down Expand Up @@ -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}"
]
},
{
Expand All @@ -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}"
]
}
Expand Down
Loading

0 comments on commit 017901d

Please sign in to comment.