Skip to content

Commit

Permalink
Merge branch 'release-4.0' into release-4.0-abbf3fe8e638
Browse files Browse the repository at this point in the history
  • Loading branch information
lysu authored Mar 8, 2021
2 parents f7e99ba + 31fd1f3 commit 4533f5c
Show file tree
Hide file tree
Showing 23 changed files with 278 additions and 124 deletions.
23 changes: 9 additions & 14 deletions bindinfo/handle.go
Original file line number Diff line number Diff line change
Expand Up @@ -661,16 +661,14 @@ func (h *BindHandle) CaptureBaselines() {
func getHintsForSQL(sctx sessionctx.Context, sql string) (string, error) {
oriVals := sctx.GetSessionVars().UsePlanBaselines
sctx.GetSessionVars().UsePlanBaselines = false
recordSets, err := sctx.(sqlexec.SQLExecutor).ExecuteInternal(context.TODO(), fmt.Sprintf("explain format='hint' %s", sql))
rs, err := sctx.(sqlexec.SQLExecutor).ExecuteInternal(context.TODO(), fmt.Sprintf("explain format='hint' %s", sql))
sctx.GetSessionVars().UsePlanBaselines = oriVals
if len(recordSets) > 0 {
defer terror.Log(recordSets[0].Close())
}
if err != nil {
return "", err
}
chk := recordSets[0].NewChunk()
err = recordSets[0].Next(context.TODO(), chk)
defer terror.Call(rs.Close)
chk := rs.NewChunk()
err = rs.Next(context.TODO(), chk)
if err != nil {
return "", err
}
Expand Down Expand Up @@ -873,23 +871,20 @@ func runSQL(ctx context.Context, sctx sessionctx.Context, sql string, resultChan
resultChan <- fmt.Errorf("run sql panicked: %v", string(buf))
}
}()
recordSets, err := sctx.(sqlexec.SQLExecutor).ExecuteInternal(ctx, sql)
rs, err := sctx.(sqlexec.SQLExecutor).ExecuteInternal(ctx, sql)
if err != nil {
if len(recordSets) > 0 {
terror.Call(recordSets[0].Close)
}
terror.Call(rs.Close)
resultChan <- err
return
}
recordSet := recordSets[0]
chk := recordSets[0].NewChunk()
chk := rs.NewChunk()
for {
err = recordSet.Next(ctx, chk)
err = rs.Next(ctx, chk)
if err != nil || chk.NumRows() == 0 {
break
}
}
terror.Call(recordSets[0].Close)
terror.Call(rs.Close)
resultChan <- err
}

Expand Down
9 changes: 6 additions & 3 deletions domain/domain.go
Original file line number Diff line number Diff line change
Expand Up @@ -1215,9 +1215,12 @@ func (do *Domain) NotifyUpdatePrivilege(ctx sessionctx.Context) {
}
}
// update locally
_, _, err := ctx.(sqlexec.RestrictedSQLExecutor).ExecRestrictedSQL(`FLUSH PRIVILEGES`)
if err != nil {
logutil.BgLogger().Error("unable to update privileges", zap.Error(err))
exec := ctx.(sqlexec.RestrictedSQLExecutor)
if stmt, err := exec.ParseWithParams(context.Background(), `FLUSH PRIVILEGES`); err == nil {
_, _, err := exec.ExecRestrictedStmt(context.Background(), stmt)
if err != nil {
logutil.BgLogger().Error("unable to update privileges", zap.Error(err))
}
}
}

Expand Down
2 changes: 2 additions & 0 deletions executor/adapter.go
Original file line number Diff line number Diff line change
Expand Up @@ -907,6 +907,8 @@ func (a *ExecStmt) LogSlowQuery(txnTS uint64, succ bool, hasMoreResults bool) {
TimeTotal: costTime,
TimeParse: sessVars.DurationParse,
TimeCompile: sessVars.DurationCompile,
TimeOptimize: sessVars.DurationOptimization,
TimeWaitTS: sessVars.DurationWaitTS,
IndexNames: indexNames,
StatsInfos: statsInfos,
CopTasks: copTaskInfo,
Expand Down
10 changes: 10 additions & 0 deletions executor/ddl_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -291,6 +291,16 @@ func (s *testSuite6) TestViewRecursion(c *C) {
tk.MustExec("drop view recursive_view1, t")
}

func (s *testSuite6) TestIssue23027(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec("drop table if exists t23027")
tk.MustExec("create table t23027(a char(10))")
tk.MustExec("insert into t23027 values ('a'), ('a')")
tk.MustExec("create definer='root'@'localhost' view v23027 as select group_concat(a) from t23027;")
tk.MustQuery("select * from v23027").Check(testkit.Rows("a,a"))
}

func (s *testSuite6) TestIssue16250(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
Expand Down
8 changes: 8 additions & 0 deletions executor/slow_query.go
Original file line number Diff line number Diff line change
Expand Up @@ -457,6 +457,8 @@ type slowQueryTuple struct {
rewriteTime float64
preprocSubqueries uint64
preprocSubQueryTime float64
optimizeTime float64
waitTSTime float64
preWriteTime float64
waitPrewriteBinlogTime float64
commitTime float64
Expand Down Expand Up @@ -563,6 +565,10 @@ func (st *slowQueryTuple) setFieldValue(tz *time.Location, field, value string,
st.parseTime, err = strconv.ParseFloat(value, 64)
case variable.SlowLogCompileTimeStr:
st.compileTime, err = strconv.ParseFloat(value, 64)
case variable.SlowLogOptimizeTimeStr:
st.optimizeTime, err = strconv.ParseFloat(value, 64)
case variable.SlowLogWaitTSTimeStr:
st.waitTSTime, err = strconv.ParseFloat(value, 64)
case execdetails.PreWriteTimeStr:
st.preWriteTime, err = strconv.ParseFloat(value, 64)
case execdetails.WaitPrewriteBinlogTimeStr:
Expand Down Expand Up @@ -687,6 +693,8 @@ func (st *slowQueryTuple) convertToDatumRow() []types.Datum {
record = append(record, types.NewFloat64Datum(st.rewriteTime))
record = append(record, types.NewUintDatum(st.preprocSubqueries))
record = append(record, types.NewFloat64Datum(st.preprocSubQueryTime))
record = append(record, types.NewFloat64Datum(st.optimizeTime))
record = append(record, types.NewFloat64Datum(st.waitTSTime))
record = append(record, types.NewFloat64Datum(st.preWriteTime))
record = append(record, types.NewFloat64Datum(st.waitPrewriteBinlogTime))
record = append(record, types.NewFloat64Datum(st.commitTime))
Expand Down
2 changes: 1 addition & 1 deletion executor/slow_query_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -130,7 +130,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.38,0.021,0,0,0,1,637,0,,,1,42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772,t1:1,t2:2,` +
`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,,,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,` +
`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,,60e9378c746d9a2be1c791047e008967cf252eb6de9167ad3aa6098fa2d523f4,` +
Expand Down
4 changes: 2 additions & 2 deletions go.mod
Original file line number Diff line number Diff line change
Expand Up @@ -46,7 +46,7 @@ require (
github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989
github.com/pingcap/kvproto v0.0.0-20201126113434-70db5fb4b0dc
github.com/pingcap/log v0.0.0-20201112100606-8f1e84a3abc8
github.com/pingcap/parser v0.0.0-20210107054750-53e33b4018fe
github.com/pingcap/parser v0.0.0-20210303062609-d1d977c9ceed
github.com/pingcap/sysutil v0.0.0-20201130064824-f0c8aa6a6966
github.com/pingcap/tidb-tools v4.0.9-0.20201127090955-2707c97b3853+incompatible
github.com/pingcap/tipb v0.0.0-20200618092958-4fad48b4c8c3
Expand Down Expand Up @@ -74,7 +74,7 @@ require (
golang.org/x/oauth2 v0.0.0-20200107190931-bf48bf16ab8d // indirect
golang.org/x/sync v0.0.0-20200625203802-6e8e738ad208
golang.org/x/sys v0.0.0-20200819171115-d785dc25833f
golang.org/x/text v0.3.4
golang.org/x/text v0.3.5
golang.org/x/time v0.0.0-20191024005414-555d28b269f0 // indirect
golang.org/x/tools v0.0.0-20200820010801-b793a1359eac
golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1 // indirect
Expand Down
8 changes: 4 additions & 4 deletions go.sum
Original file line number Diff line number Diff line change
Expand Up @@ -403,8 +403,8 @@ github.com/pingcap/log v0.0.0-20200117041106-d28c14d3b1cd/go.mod h1:4rbK1p9ILyIf
github.com/pingcap/log v0.0.0-20200511115504-543df19646ad/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8=
github.com/pingcap/log v0.0.0-20201112100606-8f1e84a3abc8 h1:M+DNpOu/I3uDmwee6vcnoPd6GgSMqND4gxvDQ/W584U=
github.com/pingcap/log v0.0.0-20201112100606-8f1e84a3abc8/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8=
github.com/pingcap/parser v0.0.0-20210107054750-53e33b4018fe h1:sukVKRva68HNGZ4nuPvQS/wMvH7NMxTXV2NIhmoYP4Y=
github.com/pingcap/parser v0.0.0-20210107054750-53e33b4018fe/go.mod h1:GbEr2PgY72/4XqPZzmzstlOU/+il/wrjeTNFs6ihsSE=
github.com/pingcap/parser v0.0.0-20210303062609-d1d977c9ceed h1:+ENLMPNRG8+/YGNJChC5QRgfrcmFnsrHl9WoVLXRZok=
github.com/pingcap/parser v0.0.0-20210303062609-d1d977c9ceed/go.mod h1:GbEr2PgY72/4XqPZzmzstlOU/+il/wrjeTNFs6ihsSE=
github.com/pingcap/sysutil v0.0.0-20200206130906-2bfa6dc40bcd/go.mod h1:EB/852NMQ+aRKioCpToQ94Wl7fktV+FNnxf3CX/TTXI=
github.com/pingcap/sysutil v0.0.0-20201130064824-f0c8aa6a6966 h1:JI0wOAb8aQML0vAVLHcxTEEC0VIwrk6gtw3WjbHvJLA=
github.com/pingcap/sysutil v0.0.0-20201130064824-f0c8aa6a6966/go.mod h1:EB/852NMQ+aRKioCpToQ94Wl7fktV+FNnxf3CX/TTXI=
Expand Down Expand Up @@ -693,8 +693,8 @@ golang.org/x/sys v0.0.0-20200819171115-d785dc25833f/go.mod h1:h1NjWce9XRLGQEsW7w
golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ=
golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ=
golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk=
golang.org/x/text v0.3.4 h1:0YWbFKbhXG/wIiuHDSKpS0Iy7FSA+u45VtBMfQcFTTc=
golang.org/x/text v0.3.4/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ=
golang.org/x/text v0.3.5 h1:i6eZZ+zk0SOf0xgBpEpPD18qWcJda6q1sxt3S0kzyUQ=
golang.org/x/text v0.3.5/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ=
golang.org/x/time v0.0.0-20180412165947-fbb02b2291d2/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ=
golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ=
golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ=
Expand Down
2 changes: 2 additions & 0 deletions infoschema/tables.go
Original file line number Diff line number Diff line change
Expand Up @@ -728,6 +728,8 @@ var slowQueryCols = []columnInfo{
{name: variable.SlowLogRewriteTimeStr, tp: mysql.TypeDouble, size: 22},
{name: variable.SlowLogPreprocSubQueriesStr, tp: mysql.TypeLonglong, size: 20, flag: mysql.UnsignedFlag},
{name: variable.SlowLogPreProcSubQueryTimeStr, tp: mysql.TypeDouble, size: 22},
{name: variable.SlowLogOptimizeTimeStr, tp: mysql.TypeDouble, size: 22},
{name: variable.SlowLogWaitTSTimeStr, tp: mysql.TypeDouble, size: 22},
{name: execdetails.PreWriteTimeStr, tp: mysql.TypeDouble, size: 22},
{name: execdetails.WaitPrewriteBinlogTimeStr, tp: mysql.TypeDouble, size: 22},
{name: execdetails.CommitTimeStr, tp: mysql.TypeDouble, size: 22},
Expand Down
6 changes: 4 additions & 2 deletions infoschema/tables_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -552,6 +552,8 @@ func prepareSlowLogfile(c *C, slowLogFileName string) {
# Parse_time: 0.4
# Compile_time: 0.2
# Rewrite_time: 0.000000003 Preproc_subqueries: 2 Preproc_subqueries_time: 0.000000002
# Optimize_time: 0.00000001
# Wait_TS: 0.000000003
# LockKeys_time: 1.71 Request_count: 1 Prewrite_time: 0.19 Wait_prewrite_binlog_time: 0.21 Commit_time: 0.01 Commit_backoff_time: 0.18 Backoff_types: [txnLock] Resolve_lock_time: 0.03 Write_keys: 15 Write_size: 480 Prewrite_region: 1 Txn_retry: 8
# Cop_time: 0.3824278 Process_time: 0.161 Request_count: 1 Total_keys: 100001 Process_keys: 100000
# Wait_time: 0.101
Expand Down Expand Up @@ -636,10 +638,10 @@ func (s *testTableSuite) TestSlowQuery(c *C) {
tk.MustExec("set time_zone = '+08:00';")
re := tk.MustQuery("select * from information_schema.slow_query")
re.Check(testutil.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.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|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||0|1|1|abcd|60e9378c746d9a2be1c791047e008967cf252eb6de9167ad3aa6098fa2d523f4|update t set i = 2;|select * from t_slim;"))
"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|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||0|1|1|abcd|60e9378c746d9a2be1c791047e008967cf252eb6de9167ad3aa6098fa2d523f4|update t set i = 2;|select * from t_slim;"))
tk.MustExec("set time_zone = '+00:00';")
re = tk.MustQuery("select * from information_schema.slow_query")
re.Check(testutil.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.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|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||0|1|1|abcd|60e9378c746d9a2be1c791047e008967cf252eb6de9167ad3aa6098fa2d523f4|update t set i = 2;|select * from t_slim;"))
re.Check(testutil.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|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||0|1|1|abcd|60e9378c746d9a2be1c791047e008967cf252eb6de9167ad3aa6098fa2d523f4|update t set i = 2;|select * from t_slim;"))

// Test for long query.
f, err := os.OpenFile(slowLogFileName, os.O_CREATE|os.O_WRONLY, 0644)
Expand Down
3 changes: 3 additions & 0 deletions planner/optimize.go
Original file line number Diff line number Diff line change
Expand Up @@ -248,7 +248,10 @@ func optimize(ctx context.Context, sctx sessionctx.Context, node ast.Node, is in
finalPlan, cost, err := cascades.DefaultOptimizer.FindBestPlan(sctx, logic)
return finalPlan, names, cost, err
}

beginOpt := time.Now()
finalPlan, cost, err := plannercore.DoOptimize(ctx, sctx, builder.GetOptFlag(), logic)
sctx.GetSessionVars().DurationOptimization = time.Since(beginOpt)
return finalPlan, names, cost, err
}

Expand Down
6 changes: 6 additions & 0 deletions server/conn_stmt.go
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,7 @@ import (
"github.com/pingcap/errors"
"github.com/pingcap/parser/mysql"
"github.com/pingcap/parser/terror"
"github.com/pingcap/tidb/metrics"
plannercore "github.com/pingcap/tidb/planner/core"
"github.com/pingcap/tidb/sessionctx/stmtctx"
"github.com/pingcap/tidb/types"
Expand Down Expand Up @@ -112,6 +113,11 @@ func (cc *clientConn) handleStmtPrepare(ctx context.Context, sql string) error {

func (cc *clientConn) handleStmtExecute(ctx context.Context, data []byte) (err error) {
defer trace.StartRegion(ctx, "HandleStmtExecute").End()
defer func() {
if err != nil {
metrics.ExecuteErrorCounter.WithLabelValues(metrics.ExecuteErrorToLabel(err)).Inc()
}
}()
if len(data) < 9 {
return mysql.ErrMalformPacket
}
Expand Down
2 changes: 1 addition & 1 deletion server/sql_info_fetcher.go
Original file line number Diff line number Diff line change
Expand Up @@ -88,7 +88,7 @@ func (sh *sqlInfoFetcher) zipInfoForSQL(w http.ResponseWriter, r *http.Request)
timeoutString := r.FormValue("timeout")
curDB := strings.ToLower(r.FormValue("current_db"))
if curDB != "" {
_, err = sh.s.Execute(reqCtx, fmt.Sprintf("use %v", curDB))
_, err = sh.s.ExecuteInternal(reqCtx, "use %n", curDB)
if err != nil {
serveError(w, http.StatusInternalServerError, fmt.Sprintf("use database %v failed, err: %v", curDB, err))
return
Expand Down
12 changes: 5 additions & 7 deletions session/bootstrap.go
Original file line number Diff line number Diff line change
Expand Up @@ -1160,24 +1160,22 @@ func upgradeToVer51(s Session, ver int64) {
mustExecute(s, "COMMIT")
}()
mustExecute(s, h.LockBindInfoSQL())
var recordSets []sqlexec.RecordSet
recordSets, err = s.ExecuteInternal(context.Background(),
var rs sqlexec.RecordSet
rs, err = s.ExecuteInternal(context.Background(),
`SELECT bind_sql, default_db, status, create_time, charset, collation, source
FROM mysql.bind_info
WHERE source != 'builtin'
ORDER BY update_time DESC`)
if err != nil {
logutil.BgLogger().Fatal("upgradeToVer61 error", zap.Error(err))
}
if len(recordSets) > 0 {
defer terror.Call(recordSets[0].Close)
}
req := recordSets[0].NewChunk()
defer terror.Call(rs.Close)
req := rs.NewChunk()
iter := chunk.NewIterator4Chunk(req)
p := parser.New()
now := types.NewTime(types.FromGoTime(time.Now()), mysql.TypeTimestamp, 3)
for {
err = recordSets[0].Next(context.TODO(), req)
err = rs.Next(context.TODO(), req)
if err != nil {
logutil.BgLogger().Fatal("upgradeToVer61 error", zap.Error(err))
}
Expand Down
Loading

0 comments on commit 4533f5c

Please sign in to comment.