Skip to content

Commit 592326d

Browse files
authored
Merge branch 'master' into non-prep-param-err
2 parents 90df07d + 017901d commit 592326d

File tree

12 files changed

+280
-62
lines changed

12 files changed

+280
-62
lines changed

executor/oomtest/oom_test.go

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -223,7 +223,8 @@ func (h *oomCapture) Write(entry zapcore.Entry, fields []zapcore.Field) error {
223223
return nil
224224
}
225225
// They are just common background task and not related to the oom.
226-
if entry.Message == "SetTiFlashGroupConfig" {
226+
if entry.Message == "SetTiFlashGroupConfig" ||
227+
entry.Message == "record table item load status failed due to not finding item" {
227228
return nil
228229
}
229230

expression/expression.go

Lines changed: 16 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -1358,12 +1358,15 @@ func canScalarFuncPushDown(scalarFunc *ScalarFunction, pc PbConverter, storeType
13581358
panic(errors.Errorf("unspecified PbCode: %T", scalarFunc.Function))
13591359
})
13601360
}
1361+
storageName := storeType.Name()
1362+
if storeType == kv.UnSpecified {
1363+
storageName = "storage layer"
1364+
}
1365+
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.")
13611366
if pc.sc.InExplainStmt {
1362-
storageName := storeType.Name()
1363-
if storeType == kv.UnSpecified {
1364-
storageName = "storage layer"
1365-
}
1366-
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."))
1367+
pc.sc.AppendWarning(warnErr)
1368+
} else {
1369+
pc.sc.AppendExtraWarning(warnErr)
13671370
}
13681371
return false
13691372
}
@@ -1393,14 +1396,20 @@ func canExprPushDown(expr Expression, pc PbConverter, storeType kv.StoreType, ca
13931396
if expr.GetType().GetType() == mysql.TypeEnum && canEnumPush {
13941397
break
13951398
}
1399+
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()) + "'.")
13961400
if pc.sc.InExplainStmt {
1397-
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()) + "'."))
1401+
pc.sc.AppendWarning(warnErr)
1402+
} else {
1403+
pc.sc.AppendExtraWarning(warnErr)
13981404
}
13991405
return false
14001406
case mysql.TypeNewDecimal:
14011407
if !expr.GetType().IsDecimalValid() {
1408+
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()) + "').")
14021409
if pc.sc.InExplainStmt {
1403-
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()) + "')."))
1410+
pc.sc.AppendWarning(warnErr)
1411+
} else {
1412+
pc.sc.AppendExtraWarning(warnErr)
14041413
}
14051414
return false
14061415
}

infoschema/tables_test.go

Lines changed: 154 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -531,18 +531,166 @@ func TestSlowQuery(t *testing.T) {
531531
slowLogFileName := "tidb_slow.log"
532532
prepareSlowLogfile(t, slowLogFileName)
533533
defer func() { require.NoError(t, os.Remove(slowLogFileName)) }()
534+
expectedRes := [][]interface{}{
535+
{"2019-02-12 19:33:56.571953",
536+
"406315658548871171",
537+
"root",
538+
"localhost",
539+
"6",
540+
"57",
541+
"0.12",
542+
"4.895492",
543+
"0.4",
544+
"0.2",
545+
"0.000000003",
546+
"2",
547+
"0.000000002",
548+
"0.00000001",
549+
"0.000000003",
550+
"0.19",
551+
"0.21",
552+
"0.01",
553+
"0",
554+
"0.18",
555+
"[txnLock]",
556+
"0.03",
557+
"0",
558+
"15",
559+
"480",
560+
"1",
561+
"8",
562+
"0.3824278",
563+
"0.161",
564+
"0.101",
565+
"0.092",
566+
"1.71",
567+
"1",
568+
"100001",
569+
"100000",
570+
"100",
571+
"10",
572+
"10",
573+
"10",
574+
"100",
575+
"test",
576+
"",
577+
"0",
578+
"42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772",
579+
"t1:1,t2:2",
580+
"0.1",
581+
"0.2",
582+
"0.03",
583+
"127.0.0.1:20160",
584+
"0.05",
585+
"0.6",
586+
"0.8",
587+
"0.0.0.0:20160",
588+
"70724",
589+
"65536",
590+
"0",
591+
"0",
592+
"0",
593+
"0",
594+
"10",
595+
"",
596+
"0",
597+
"1",
598+
"0",
599+
"0",
600+
"1",
601+
"0",
602+
"0",
603+
"abcd",
604+
"60e9378c746d9a2be1c791047e008967cf252eb6de9167ad3aa6098fa2d523f4",
605+
"",
606+
"update t set i = 2;",
607+
"select * from t_slim;"},
608+
{"2021-09-08 14:39:54.506967",
609+
"427578666238083075",
610+
"root",
611+
"172.16.0.0",
612+
"40507",
613+
"0",
614+
"0",
615+
"25.571605962",
616+
"0.002923536",
617+
"0.006800973",
618+
"0.002100764",
619+
"0",
620+
"0",
621+
"0",
622+
"0.000015801",
623+
"25.542014572",
624+
"0",
625+
"0.002294647",
626+
"0.000605473",
627+
"12.483",
628+
"[tikvRPC regionMiss tikvRPC regionMiss regionMiss]",
629+
"0",
630+
"0",
631+
"624",
632+
"172064",
633+
"60",
634+
"0",
635+
"0",
636+
"0",
637+
"0",
638+
"0",
639+
"0",
640+
"0",
641+
"0",
642+
"0",
643+
"0",
644+
"0",
645+
"0",
646+
"0",
647+
"0",
648+
"rtdb",
649+
"",
650+
"0",
651+
"124acb3a0bec903176baca5f9da00b4e7512a41c93b417923f26502edeb324cc",
652+
"",
653+
"0",
654+
"0",
655+
"0",
656+
"",
657+
"0",
658+
"0",
659+
"0",
660+
"",
661+
"856544",
662+
"0",
663+
"86.635049185",
664+
"0.015486658",
665+
"100.054",
666+
"0",
667+
"0",
668+
"",
669+
"0",
670+
"1",
671+
"0",
672+
"0",
673+
"0",
674+
"0",
675+
"0",
676+
"",
677+
"",
678+
"",
679+
"",
680+
"INSERT INTO ...;",
681+
},
682+
}
534683

535684
tk.MustExec(fmt.Sprintf("set @@tidb_slow_query_file='%v'", slowLogFileName))
536685
tk.MustExec("set time_zone = '+08:00';")
537686
re := tk.MustQuery("select * from information_schema.slow_query")
538-
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;",
539-
"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 ...;",
540-
))
687+
re.Check(expectedRes)
688+
541689
tk.MustExec("set time_zone = '+00:00';")
542690
re = tk.MustQuery("select * from information_schema.slow_query")
543-
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;",
544-
"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 ...;",
545-
))
691+
expectedRes[0][0] = "2019-02-12 11:33:56.571953"
692+
expectedRes[1][0] = "2021-09-08 06:39:54.506967"
693+
re.Check(expectedRes)
546694

547695
// Test for long query.
548696
f, err := os.OpenFile(slowLogFileName, os.O_CREATE|os.O_WRONLY, 0644)

planner/core/common_plans.go

Lines changed: 0 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -914,11 +914,6 @@ func (e *Explain) explainOpRecursivelyInJSONFormat(flatOp *FlatOperator, flats F
914914
textTreeExplainID := texttree.PrettyIdentifier(explainID, flatOp.TextTreeIndent, flatOp.IsLastChild)
915915

916916
cur := e.prepareOperatorInfoForJSONFormat(flatOp.Origin, taskTp, textTreeExplainID, explainID)
917-
if e.ctx != nil && e.ctx.GetSessionVars() != nil && e.ctx.GetSessionVars().StmtCtx != nil {
918-
if optimInfo, ok := e.ctx.GetSessionVars().StmtCtx.OptimInfo[flatOp.Origin.ID()]; ok {
919-
e.ctx.GetSessionVars().StmtCtx.AppendNote(errors.New(optimInfo))
920-
}
921-
}
922917

923918
for _, idx := range flatOp.ChildrenIdx {
924919
cur.SubOperators = append(cur.SubOperators,
@@ -938,11 +933,6 @@ func (e *Explain) explainFlatOpInRowFormat(flatOp *FlatOperator) {
938933
flatOp.TextTreeIndent,
939934
flatOp.IsLastChild)
940935
e.prepareOperatorInfo(flatOp.Origin, taskTp, textTreeExplainID)
941-
if e.ctx != nil && e.ctx.GetSessionVars() != nil && e.ctx.GetSessionVars().StmtCtx != nil {
942-
if optimInfo, ok := e.ctx.GetSessionVars().StmtCtx.OptimInfo[flatOp.Origin.ID()]; ok {
943-
e.ctx.GetSessionVars().StmtCtx.AppendNote(errors.New(optimInfo))
944-
}
945-
}
946936
}
947937

948938
func getRuntimeInfoStr(ctx sessionctx.Context, p Plan, runtimeStatsColl *execdetails.RuntimeStatsColl) (actRows, analyzeInfo, memoryInfo, diskInfo string) {

planner/core/exhaust_physical_plans.go

Lines changed: 4 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -2730,8 +2730,11 @@ func (la *LogicalAggregation) checkCanPushDownToMPP() bool {
27302730
}
27312731
}
27322732
if hasUnsupportedDistinct {
2733+
warnErr := errors.New("Aggregation can not be pushed to storage layer in mpp mode because it contains agg function with distinct")
27332734
if la.ctx.GetSessionVars().StmtCtx.InExplainStmt {
2734-
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"))
2735+
la.ctx.GetSessionVars().StmtCtx.AppendWarning(warnErr)
2736+
} else {
2737+
la.ctx.GetSessionVars().StmtCtx.AppendExtraWarning(warnErr)
27352738
}
27362739
return false
27372740
}

planner/core/find_best_task.go

Lines changed: 6 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -742,7 +742,7 @@ func (ds *DataSource) skylinePruning(prop *property.PhysicalProperty) []*candida
742742
}
743743

744744
func (ds *DataSource) getPruningInfo(candidates []*candidatePath, prop *property.PhysicalProperty) string {
745-
if !ds.ctx.GetSessionVars().StmtCtx.InVerboseExplain || len(candidates) == len(ds.possibleAccessPaths) {
745+
if len(candidates) == len(ds.possibleAccessPaths) {
746746
return ""
747747
}
748748
if len(candidates) == 1 && len(candidates[0].path.Ranges) == 0 {
@@ -889,10 +889,12 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, planCounter
889889
pruningInfo := ds.getPruningInfo(candidates, prop)
890890
defer func() {
891891
if err == nil && t != nil && !t.invalid() && pruningInfo != "" {
892-
if ds.ctx.GetSessionVars().StmtCtx.OptimInfo == nil {
893-
ds.ctx.GetSessionVars().StmtCtx.OptimInfo = make(map[int]string)
892+
warnErr := errors.New(pruningInfo)
893+
if ds.ctx.GetSessionVars().StmtCtx.InVerboseExplain {
894+
ds.ctx.GetSessionVars().StmtCtx.AppendNote(warnErr)
895+
} else {
896+
ds.ctx.GetSessionVars().StmtCtx.AppendExtraNote(warnErr)
894897
}
895-
ds.ctx.GetSessionVars().StmtCtx.OptimInfo[t.plan().ID()] = pruningInfo
896898
}
897899
}()
898900

planner/core/stats.go

Lines changed: 28 additions & 24 deletions
Original file line numberDiff line numberDiff line change
@@ -343,35 +343,37 @@ func (ds *DataSource) derivePathStatsAndTryHeuristics() error {
343343
if selected != nil {
344344
ds.possibleAccessPaths[0] = selected
345345
ds.possibleAccessPaths = ds.possibleAccessPaths[:1]
346-
if ds.ctx.GetSessionVars().StmtCtx.InVerboseExplain {
347-
var tableName string
348-
if ds.TableAsName.O == "" {
349-
tableName = ds.tableInfo.Name.O
346+
var tableName string
347+
if ds.TableAsName.O == "" {
348+
tableName = ds.tableInfo.Name.O
349+
} else {
350+
tableName = ds.TableAsName.O
351+
}
352+
var sb strings.Builder
353+
if selected.IsTablePath() {
354+
// TODO: primary key / handle / real name?
355+
sb.WriteString(fmt.Sprintf("handle of %s is selected since the path only has point ranges", tableName))
356+
} else {
357+
if selected.Index.Unique {
358+
sb.WriteString("unique ")
359+
}
360+
sb.WriteString(fmt.Sprintf("index %s of %s is selected since the path", selected.Index.Name.O, tableName))
361+
if isRefinedPath {
362+
sb.WriteString(" only fetches limited number of rows")
350363
} else {
351-
tableName = ds.TableAsName.O
364+
sb.WriteString(" only has point ranges")
352365
}
353-
if selected.IsTablePath() {
354-
// TODO: primary key / handle / real name?
355-
ds.ctx.GetSessionVars().StmtCtx.AppendNote(fmt.Errorf("handle of %s is selected since the path only has point ranges", tableName))
366+
if selected.IsSingleScan {
367+
sb.WriteString(" with single scan")
356368
} else {
357-
var sb strings.Builder
358-
if selected.Index.Unique {
359-
sb.WriteString("unique ")
360-
}
361-
sb.WriteString(fmt.Sprintf("index %s of %s is selected since the path", selected.Index.Name.O, tableName))
362-
if isRefinedPath {
363-
sb.WriteString(" only fetches limited number of rows")
364-
} else {
365-
sb.WriteString(" only has point ranges")
366-
}
367-
if selected.IsSingleScan {
368-
sb.WriteString(" with single scan")
369-
} else {
370-
sb.WriteString(" with double scan")
371-
}
372-
ds.ctx.GetSessionVars().StmtCtx.AppendNote(errors.New(sb.String()))
369+
sb.WriteString(" with double scan")
373370
}
374371
}
372+
if ds.ctx.GetSessionVars().StmtCtx.InVerboseExplain {
373+
ds.ctx.GetSessionVars().StmtCtx.AppendNote(errors.New(sb.String()))
374+
} else {
375+
ds.ctx.GetSessionVars().StmtCtx.AppendExtraNote(errors.New(sb.String()))
376+
}
375377
}
376378
return nil
377379
}
@@ -435,8 +437,10 @@ func (ds *DataSource) DeriveStats(_ []*property.StatsInfo, _ *expression.Schema,
435437
if needConsiderIndexMerge {
436438
// PushDownExprs() will append extra warnings, which is annoying. So we reset warnings here.
437439
warnings := stmtCtx.GetWarnings()
440+
extraWarnings := stmtCtx.GetExtraWarnings()
438441
_, remaining := expression.PushDownExprs(stmtCtx, indexMergeConds, ds.ctx.GetClient(), kv.UnSpecified)
439442
stmtCtx.SetWarnings(warnings)
443+
stmtCtx.SetExtraWarnings(extraWarnings)
440444
if len(remaining) != 0 {
441445
needConsiderIndexMerge = false
442446
}

planner/core/task.go

Lines changed: 7 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -1239,12 +1239,17 @@ func CheckAggCanPushCop(sctx sessionctx.Context, aggFuncs []*aggregation.AggFunc
12391239
ret = false
12401240
}
12411241

1242-
if !ret && sc.InExplainStmt {
1242+
if !ret {
12431243
storageName := storeType.Name()
12441244
if storeType == kv.UnSpecified {
12451245
storageName = "storage layer"
12461246
}
1247-
sc.AppendWarning(errors.New("Aggregation can not be pushed to " + storageName + " because " + reason))
1247+
warnErr := errors.New("Aggregation can not be pushed to " + storageName + " because " + reason)
1248+
if sc.InExplainStmt {
1249+
sc.AppendWarning(warnErr)
1250+
} else {
1251+
sc.AppendExtraWarning(warnErr)
1252+
}
12481253
}
12491254
return ret
12501255
}

0 commit comments

Comments
 (0)