From 1de46ff6981276607345c25ab7f7a22df07dd8af Mon Sep 17 00:00:00 2001 From: zhuo-zhi <517770911@qq.com> Date: Tue, 20 Jul 2021 17:30:59 +0800 Subject: [PATCH 01/17] Modify tidb/executor to implement plan recreator --- executor/builder.go | 10 ++ executor/plan_recreator.go | 334 +++++++++++++++++++++++++++++++++++ planner/core/common_plans.go | 9 + planner/core/planbuilder.go | 7 + server/conn.go | 20 +++ session/session.go | 1 + 6 files changed, 381 insertions(+) create mode 100644 executor/plan_recreator.go diff --git a/executor/builder.go b/executor/builder.go index 1f64a98b356da..506a22facf631 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -161,6 +161,8 @@ func (b *executorBuilder) build(p plannercore.Plan) Executor { return b.buildLoadStats(v) case *plannercore.IndexAdvise: return b.buildIndexAdvise(v) + case *plannercore.PlanRecreatorSingle: + return b.buildPlanRecreatorSingle(v) case *plannercore.PhysicalLimit: return b.buildLimit(v) case *plannercore.Prepare: @@ -909,6 +911,14 @@ func (b *executorBuilder) buildIndexAdvise(v *plannercore.IndexAdvise) Executor return e } +func (b *executorBuilder) buildPlanRecreatorSingle(v *plannercore.PlanRecreatorSingle) Executor { + e := &PlanRecreatorSingleExec{ + baseExecutor: newBaseExecutor(b.ctx, nil, v.ID()), + info: &PlanRecreatorSingleInfo{v.ExecStmt, v.Analyze, v.Load, v.File, b.ctx}, + } + return e +} + func (b *executorBuilder) buildReplace(vals *InsertValues) Executor { replaceExec := &ReplaceExec{ InsertValues: vals, diff --git a/executor/plan_recreator.go b/executor/plan_recreator.go new file mode 100644 index 0000000000000..fa9115309f968 --- /dev/null +++ b/executor/plan_recreator.go @@ -0,0 +1,334 @@ +// Copyright 2018 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package executor + +import ( + "archive/zip" + "context" + "encoding/json" + "fmt" + "github.com/pingcap/errors" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/format" + "github.com/pingcap/parser/model" + "github.com/pingcap/tidb/domain" + "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/statistics/handle" + "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/sqlexec" + "os" + "strings" + "time" +) + +const recreatorPath string = "/tmp/recreator" + +// PlanRecreatorExec represents a plan recreator executor. +type PlanRecreatorSingleExec struct { + baseExecutor + info *PlanRecreatorSingleInfo +} + +// PlanRecreatorInfo saves the information of plan recreator operation. +type PlanRecreatorSingleInfo struct { + ExecStmt ast.StmtNode + Analyze bool + Load bool + File string + Ctx sessionctx.Context +} + +type tableNamePair struct { + DBName string + TableName string +} + +type tableNameExtractor struct { + curDB string + names map[tableNamePair]struct{} +} + +func (tne *tableNameExtractor) Enter(in ast.Node) (ast.Node, bool) { + if _, ok := in.(*ast.TableName); ok { + return in, true + } + return in, false +} + +func (tne *tableNameExtractor) Leave(in ast.Node) (ast.Node, bool) { + if t, ok := in.(*ast.TableName); ok { + tp := tableNamePair{DBName: t.Schema.L, TableName: t.Name.L} + if tp.DBName == "" { + tp.DBName = tne.curDB + } + if _, ok := tne.names[tp]; !ok { + tne.names[tp] = struct{}{} + } + } + return in, true +} + +// planRecreatorVarKeyType is a dummy type to avoid naming collision in context. +type planRecreatorVarKeyType int + +// String defines a Stringer function for debugging and pretty printing. +func (k planRecreatorVarKeyType) String() string { + return "plan_recreator_var" +} + +// PlanRecreatorVarKey is a variable key for load statistic. +const PlanRecreatorVarKey planRecreatorVarKeyType = 0 + +// Next implements the Executor Next interface. +func (e *PlanRecreatorSingleExec) Next(ctx context.Context, req *chunk.Chunk) error { + req.GrowAndReset(e.maxChunkSize) + if e.info.ExecStmt == nil { + return errors.New("Plan Recreator: sql is empty.") + } + val := e.ctx.Value(PlanRecreatorVarKey) + if val != nil { + e.ctx.SetValue(PlanRecreatorVarKey, nil) + return errors.New("Plan Recreator: previous plan recreator option isn't closed normally.") + } + e.ctx.SetValue(PlanRecreatorVarKey, e.info) + return nil +} + +// Close implements the Executor Close interface. +func (e *PlanRecreatorSingleExec) Close() error { + return nil +} + +// Open implements the Executor Open interface. +func (e *PlanRecreatorSingleExec) Open(ctx context.Context) error { + return nil +} + +func (e *PlanRecreatorSingleInfo) Process() error { + if e.Load { + return nil + } else { + return e.DumpSingle() + } +} + +// Process dose the export/import work for reproducing sql queries. +func (e *PlanRecreatorSingleInfo) DumpSingle() error { + // Create zip file + err := os.MkdirAll(recreatorPath, os.ModePerm) + if err != nil { + return errors.New("Plan Recreator: cannot create plan recreator path.") + } + zf, err := os.Create(recreatorPath + "/" + fmt.Sprintf("recreator_single_%v.zip", time.Now().UnixNano())) + if err != nil { + return errors.New("Plan Recreator: cannot create zip file.") + } + defer zf.Close() + zw := zip.NewWriter(zf) + + // Retrieve current DB + sessionVars := e.Ctx.GetSessionVars() + dbName := model.NewCIStr(sessionVars.CurrentDB) + do := domain.GetDomain(e.Ctx) + + // Retrieve all tables + pairs, err := extractTableNames(e.ExecStmt, dbName.L) + if err != nil { + return errors.New(fmt.Sprintf("Plan Recreator: invalid SQL text, err: %v", err)) + } + + // Dump stats + for pair := range pairs { + jsonTbl, err := getStatsForTable(do, pair) + if err != nil { + return err + } + statsFw, err := zw.Create(fmt.Sprintf("stats/%v.%v.json", pair.DBName, pair.TableName)) + if err != nil { + return err + } + data, err := json.Marshal(jsonTbl) + if err != nil { + return err + } + _, err = statsFw.Write(data) + if err != nil { + return err + } + } + + // Dump schema + for pair := range pairs { + err = getShowCreateTable(pair, zw, e.Ctx) + if err != nil { + return err + } + } + + // Dump explain + var sb strings.Builder + ctx := format.NewRestoreCtx(format.RestoreStringDoubleQuotes, &sb) + err = e.ExecStmt.Restore(ctx) + if err != nil { + return nil + } + if e.Analyze { + // Explain analyze + recordSets, err := e.Ctx.(sqlexec.SQLExecutor).Execute(context.TODO(), fmt.Sprintf("explain analyze %s", sb.String())) + if len(recordSets) > 0 { + defer recordSets[0].Close() + } + if err != nil { + return err + } + sRows, err := resultSetToStringSlice(context.Background(), recordSets[0]) + if err != nil { + return err + } + fw, err := zw.Create("explain.txt") + if err != nil { + return err + } + for _, row := range sRows { + fmt.Fprintf(fw, "%s\n", strings.Join(row, "\t")) + } + + err = zw.Close() + if err != nil { + return err + } + return nil + } else { + // Analyze + recordSets, err := e.Ctx.(sqlexec.SQLExecutor).Execute(context.TODO(), fmt.Sprintf("explain %s", sb.String())) + if len(recordSets) > 0 { + defer recordSets[0].Close() + } + if err != nil { + return err + } + sRows, err := resultSetToStringSlice(context.Background(), recordSets[0]) + if err != nil { + return err + } + fw, err := zw.Create("explain.txt") + if err != nil { + return err + } + for _, row := range sRows { + fmt.Fprintf(fw, "%s\n", strings.Join(row, "\t")) + } + + err = zw.Close() + if err != nil { + return err + } + return nil + } +} + +func extractTableNames(ExecStmt ast.StmtNode, curDB string) (map[tableNamePair]struct{}, error) { + extractor := &tableNameExtractor{ + curDB: curDB, + names: make(map[tableNamePair]struct{}), + } + ExecStmt.Accept(extractor) + return extractor.names, nil +} + +func getStatsForTable(do *domain.Domain, pair tableNamePair) (*handle.JSONTable, error) { + is := do.InfoSchema() + h := do.StatsHandle() + tbl, err := is.TableByName(model.NewCIStr(pair.DBName), model.NewCIStr(pair.TableName)) + if err != nil { + return nil, err + } + js, err := h.DumpStatsToJSON(pair.DBName, tbl.Meta(), nil) + return js, err +} + +func getShowCreateTable(pair tableNamePair, zw *zip.Writer, ctx sessionctx.Context) error { + recordSets, err := ctx.(sqlexec.SQLExecutor).Execute(context.TODO(), fmt.Sprintf("show create table `%v`.`%v`", pair.DBName, pair.TableName)) + if len(recordSets) > 0 { + defer recordSets[0].Close() + } + if err != nil { + return err + } + sRows, err := resultSetToStringSlice(context.Background(), recordSets[0]) + if err != nil { + return err + } + fw, err := zw.Create(fmt.Sprintf("schema/%v.%v.schema.txt", pair.DBName, pair.TableName)) + if err != nil { + return err + } + for _, row := range sRows { + fmt.Fprintf(fw, "%s\n", strings.Join(row, "\t")) + } + return nil +} + +func resultSetToStringSlice(ctx context.Context, rs sqlexec.RecordSet) ([][]string, error) { + rows, err := getRows4Test(ctx, rs) + if err != nil { + return nil, err + } + err = rs.Close() + if err != nil { + return nil, err + } + sRows := make([][]string, len(rows)) + for i := range rows { + row := rows[i] + iRow := make([]string, row.Len()) + for j := 0; j < row.Len(); j++ { + if row.IsNull(j) { + iRow[j] = "" + } else { + d := row.GetDatum(j, &rs.Fields()[j].Column.FieldType) + iRow[j], err = d.ToString() + if err != nil { + return nil, err + } + } + } + sRows[i] = iRow + } + return sRows, nil +} + +func getRows4Test(ctx context.Context, rs sqlexec.RecordSet) ([]chunk.Row, error) { + if rs == nil { + return nil, nil + } + var rows []chunk.Row + req := rs.NewChunk() + // Must reuse `req` for imitating server.(*clientConn).writeChunks + for { + err := rs.Next(ctx, req) + if err != nil { + return nil, err + } + if req.NumRows() == 0 { + break + } + + iter := chunk.NewIterator4Chunk(req.CopyConstruct()) + for row := iter.Begin(); row != iter.End(); row = iter.Next() { + rows = append(rows, row) + } + } + return rows, nil +} diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index 126cb17d7da1e..e555bfd9c1c9c 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -933,6 +933,15 @@ type LoadStats struct { Path string } +// PlanRecreatorSingle represents a plan recreator plan. +type PlanRecreatorSingle struct { + baseSchemaProducer + ExecStmt ast.StmtNode + Analyze bool + Load bool + File string +} + // IndexAdvise represents a index advise plan. type IndexAdvise struct { baseSchemaProducer diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 3e39d12bd58fb..f5e9696701be4 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -695,6 +695,8 @@ func (b *PlanBuilder) Build(ctx context.Context, node ast.Node) (Plan, error) { return b.buildLoadStats(x), nil case *ast.IndexAdviseStmt: return b.buildIndexAdvise(x), nil + case *ast.PlanRecreatorStmt: + return b.buildPlanRecreator(x), nil case *ast.PrepareStmt: return b.buildPrepare(x), nil case *ast.SelectStmt: @@ -4116,6 +4118,11 @@ func buildShowSchema(s *ast.ShowStmt, isView bool, isSequence bool) (schema *exp return } +func (b *PlanBuilder) buildPlanRecreator(pc *ast.PlanRecreatorStmt) Plan { + p := &PlanRecreatorSingle{ExecStmt: pc.Stmt, Analyze: pc.Analyze, Load: pc.Load, File: pc.File} + return p +} + func buildChecksumTableSchema() (*expression.Schema, []*types.FieldName) { schema := newColumnsWithNames(5) schema.Append(buildColumnWithName("", "Db_name", mysql.TypeVarchar, 128)) diff --git a/server/conn.go b/server/conn.go index b73e51543b13f..8c0cc5853607f 100644 --- a/server/conn.go +++ b/server/conn.go @@ -1617,6 +1617,16 @@ func (cc *clientConn) handleIndexAdvise(ctx context.Context, indexAdviseInfo *ex return nil } +// handlePlanRecreator dose the export/import work for reproducing sql queries. +func (cc *clientConn) handlePlanRecreator(ctx context.Context, planRecreatorInfo interface{}) error { + switch planRecreatorInfo.(type) { + case *executor.PlanRecreatorSingleInfo: + info := planRecreatorInfo.(*executor.PlanRecreatorSingleInfo) + return info.Process() + } + return errors.New("plan recreator: not supporting info type") +} + // handleQuery executes the sql query string and writes result set or result ok to the client. // As the execution time of this function represents the performance of TiDB, we do time log and metrics here. // There is a special query `load data` that does not return result, which is handled differently. @@ -1879,6 +1889,16 @@ func (cc *clientConn) handleQuerySpecial(ctx context.Context, status uint16) (bo return handled, err } } + + planRecreator := cc.ctx.Value(executor.PlanRecreatorVarKey) + if planRecreator != nil { + handled = true + defer cc.ctx.SetValue(executor.PlanRecreatorVarKey, nil) + if err := cc.handlePlanRecreator(ctx, planRecreator); err != nil { + return handled, err + } + } + return handled, cc.writeOkWith(ctx, cc.ctx.LastMessage(), cc.ctx.AffectedRows(), cc.ctx.LastInsertID(), status, cc.ctx.WarningCount()) } diff --git a/session/session.go b/session/session.go index 568fd0666fad7..ff22caefe4eb5 100644 --- a/session/session.go +++ b/session/session.go @@ -1672,6 +1672,7 @@ var querySpecialKeys = []fmt.Stringer{ executor.LoadDataVarKey, executor.LoadStatsVarKey, executor.IndexAdviseVarKey, + executor.PlanRecreatorVarKey, } func (s *session) hasQuerySpecial() bool { From 99fd2d703c54cf326282d06dd960a9778e93c98e Mon Sep 17 00:00:00 2001 From: zhuo-zhi <517770911@qq.com> Date: Tue, 20 Jul 2021 17:47:52 +0800 Subject: [PATCH 02/17] Refine some comments --- executor/plan_recreator.go | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) diff --git a/executor/plan_recreator.go b/executor/plan_recreator.go index fa9115309f968..961ab6b7a7053 100644 --- a/executor/plan_recreator.go +++ b/executor/plan_recreator.go @@ -18,6 +18,10 @@ import ( "context" "encoding/json" "fmt" + "os" + "strings" + "time" + "github.com/pingcap/errors" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/format" @@ -27,9 +31,6 @@ import ( "github.com/pingcap/tidb/statistics/handle" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/sqlexec" - "os" - "strings" - "time" ) const recreatorPath string = "/tmp/recreator" @@ -115,16 +116,17 @@ func (e *PlanRecreatorSingleExec) Open(ctx context.Context) error { return nil } +// Process dose the export/import work for reproducing sql queries. func (e *PlanRecreatorSingleInfo) Process() error { + // TODO: plan recreator load will be developed later if e.Load { return nil } else { - return e.DumpSingle() + return e.dumpSingle() } } -// Process dose the export/import work for reproducing sql queries. -func (e *PlanRecreatorSingleInfo) DumpSingle() error { +func (e *PlanRecreatorSingleInfo) dumpSingle() error { // Create zip file err := os.MkdirAll(recreatorPath, os.ModePerm) if err != nil { @@ -210,7 +212,7 @@ func (e *PlanRecreatorSingleInfo) DumpSingle() error { } return nil } else { - // Analyze + // Explain recordSets, err := e.Ctx.(sqlexec.SQLExecutor).Execute(context.TODO(), fmt.Sprintf("explain %s", sb.String())) if len(recordSets) > 0 { defer recordSets[0].Close() From b2b636e596340cfb3206b827dde8472148318fe3 Mon Sep 17 00:00:00 2001 From: zhuo-zhi <517770911@qq.com> Date: Wed, 21 Jul 2021 16:43:21 +0800 Subject: [PATCH 03/17] support dumping all files related to single sql --- executor/plan_recreator.go | 153 ++++++++++++++++++++++++------------- 1 file changed, 101 insertions(+), 52 deletions(-) diff --git a/executor/plan_recreator.go b/executor/plan_recreator.go index 961ab6b7a7053..03379c2ffb0a2 100644 --- a/executor/plan_recreator.go +++ b/executor/plan_recreator.go @@ -18,19 +18,22 @@ import ( "context" "encoding/json" "fmt" - "os" - "strings" - "time" - + "github.com/BurntSushi/toml" "github.com/pingcap/errors" + "github.com/pingcap/parser" "github.com/pingcap/parser/ast" - "github.com/pingcap/parser/format" "github.com/pingcap/parser/model" + "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/statistics/handle" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/logutil" + "github.com/pingcap/tidb/util/printer" "github.com/pingcap/tidb/util/sqlexec" + "os" + "strings" + "time" ) const recreatorPath string = "/tmp/recreator" @@ -136,8 +139,33 @@ func (e *PlanRecreatorSingleInfo) dumpSingle() error { if err != nil { return errors.New("Plan Recreator: cannot create zip file.") } - defer zf.Close() zw := zip.NewWriter(zf) + defer func() { + err = zw.Close() + if err != nil { + logutil.BgLogger().Warn("Closing zip writer failed.") + } + zf.Close() + }() + + // Dump config + cf, err := zw.Create("config.toml") + if err != nil { + return err + } + if err := toml.NewEncoder(cf).Encode(config.GetGlobalConfig()); err != nil { + return err + } + + // Dump meta + mt, err := zw.Create("meta.txt") + if err != nil { + return err + } + _, err = mt.Write([]byte(printer.GetTiDBInfo())) + if err != nil { + return err + } // Retrieve current DB sessionVars := e.Ctx.GetSessionVars() @@ -178,66 +206,87 @@ func (e *PlanRecreatorSingleInfo) dumpSingle() error { } } - // Dump explain - var sb strings.Builder - ctx := format.NewRestoreCtx(format.RestoreStringDoubleQuotes, &sb) - err = e.ExecStmt.Restore(ctx) + // Dump variables + varMap := make(map[string]string) + recordSets, err := e.Ctx.(sqlexec.SQLExecutor).Execute(context.TODO(), "show variables") + if len(recordSets) > 0 { + defer recordSets[0].Close() + } + if err != nil { + return err + } + sRows, err := resultSetToStringSlice(context.Background(), recordSets[0]) + if err != nil { + return err + } + vf, err := zw.Create("variables.toml") + if err != nil { + return err + } + for _, row := range sRows { + varMap[row[0]] = row[1] + } + if err := toml.NewEncoder(vf).Encode(varMap); err != nil { + return err + } + + // Dump sql + sql, err := zw.Create("sqls.sql") if err != nil { return nil } + sql.Write([]byte(e.ExecStmt.Text())) + + // Dump bindings + normSql, _ := parser.NormalizeDigest(e.ExecStmt.Text()) + recordSets, err = e.Ctx.(sqlexec.SQLExecutor).Execute(context.TODO(), fmt.Sprintf("show bindings where Original_sql='%s'", normSql)) + if len(recordSets) > 0 { + defer recordSets[0].Close() + } + if err != nil { + return err + } + sRows, err = resultSetToStringSlice(context.Background(), recordSets[0]) + if err != nil { + return err + } + bf, err := zw.Create("bindings.sql") + if err != nil { + return err + } + for _, row := range sRows { + fmt.Fprintf(bf, "%s\n", strings.Join(row, "\t")) + } + + // Dump explain if e.Analyze { // Explain analyze - recordSets, err := e.Ctx.(sqlexec.SQLExecutor).Execute(context.TODO(), fmt.Sprintf("explain analyze %s", sb.String())) - if len(recordSets) > 0 { - defer recordSets[0].Close() - } - if err != nil { - return err - } - sRows, err := resultSetToStringSlice(context.Background(), recordSets[0]) + recordSets, err = e.Ctx.(sqlexec.SQLExecutor).Execute(context.TODO(), fmt.Sprintf("explain analyze %s", e.ExecStmt.Text())) if err != nil { return err } - fw, err := zw.Create("explain.txt") - if err != nil { - return err - } - for _, row := range sRows { - fmt.Fprintf(fw, "%s\n", strings.Join(row, "\t")) - } - - err = zw.Close() - if err != nil { - return err - } - return nil } else { // Explain - recordSets, err := e.Ctx.(sqlexec.SQLExecutor).Execute(context.TODO(), fmt.Sprintf("explain %s", sb.String())) - if len(recordSets) > 0 { - defer recordSets[0].Close() - } - if err != nil { - return err - } - sRows, err := resultSetToStringSlice(context.Background(), recordSets[0]) + recordSets, err = e.Ctx.(sqlexec.SQLExecutor).Execute(context.TODO(), fmt.Sprintf("explain %s", e.ExecStmt.Text())) if err != nil { return err } - fw, err := zw.Create("explain.txt") - if err != nil { - return err - } - for _, row := range sRows { - fmt.Fprintf(fw, "%s\n", strings.Join(row, "\t")) - } - - err = zw.Close() - if err != nil { - return err - } - return nil } + if len(recordSets) > 0 { + defer recordSets[0].Close() + } + sRows, err = resultSetToStringSlice(context.Background(), recordSets[0]) + if err != nil { + return err + } + fw, err := zw.Create("explain.txt") + if err != nil { + return err + } + for _, row := range sRows { + fmt.Fprintf(fw, "%s\n", strings.Join(row, "\t")) + } + return nil } func extractTableNames(ExecStmt ast.StmtNode, curDB string) (map[tableNamePair]struct{}, error) { From c1bc141c949ee5ac4337e713737d84dacb07f0e5 Mon Sep 17 00:00:00 2001 From: zhuo-zhi <517770911@qq.com> Date: Wed, 21 Jul 2021 17:28:23 +0800 Subject: [PATCH 04/17] add plan recreator unit test --- executor/executor_test.go | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/executor/executor_test.go b/executor/executor_test.go index a21506ac16dc3..1aa93e10214f4 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -297,6 +297,14 @@ func (s *testSuiteP1) TestLoadStats(c *C) { c.Assert(tk.ExecToErr("load stats ./xxx.json"), NotNil) } +func (s *testSuiteP1) TestPlanRecreator(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int, b int, index idx_a(a))") + tk.MustExec("plan recreator dump explain select * from t where a=10") +} + func (s *testSuiteP1) TestShow(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("create database test_show;") From ef88000d7de7969840920378f7be56c66aac248d Mon Sep 17 00:00:00 2001 From: zhuo-zhi <517770911@qq.com> Date: Thu, 22 Jul 2021 18:18:11 +0800 Subject: [PATCH 05/17] Support cleaning outdated files --- executor/plan_recreator.go | 44 ++++++++++++++++++++++++++++++++++---- 1 file changed, 40 insertions(+), 4 deletions(-) diff --git a/executor/plan_recreator.go b/executor/plan_recreator.go index 03379c2ffb0a2..38cf8e050a56a 100644 --- a/executor/plan_recreator.go +++ b/executor/plan_recreator.go @@ -37,6 +37,7 @@ import ( ) const recreatorPath string = "/tmp/recreator" +const remainedInterval float64 = 3 // PlanRecreatorExec represents a plan recreator executor. type PlanRecreatorSingleExec struct { @@ -91,8 +92,17 @@ func (k planRecreatorVarKeyType) String() string { return "plan_recreator_var" } +// planRecreatorFileListType is a dummy type to avoid naming collision in context. +type planRecreatorFileListType int + +// String defines a Stringer function for debugging and pretty printing. +func (k planRecreatorFileListType) String() string { + return "plan_recreator_file_list" +} + // PlanRecreatorVarKey is a variable key for load statistic. const PlanRecreatorVarKey planRecreatorVarKeyType = 0 +const PlanRecreatorFileList planRecreatorFileListType = 0 // Next implements the Executor Next interface. func (e *PlanRecreatorSingleExec) Next(ctx context.Context, req *chunk.Chunk) error { @@ -130,22 +140,48 @@ func (e *PlanRecreatorSingleInfo) Process() error { } func (e *PlanRecreatorSingleInfo) dumpSingle() error { - // Create zip file + // Create path err := os.MkdirAll(recreatorPath, os.ModePerm) if err != nil { return errors.New("Plan Recreator: cannot create plan recreator path.") } - zf, err := os.Create(recreatorPath + "/" + fmt.Sprintf("recreator_single_%v.zip", time.Now().UnixNano())) + + // Create zip file + startTime := time.Now() + fileName := fmt.Sprintf("recreator_single_%v.zip", startTime.UnixNano()) + zf, err := os.Create(recreatorPath + "/" + fileName) if err != nil { return errors.New("Plan Recreator: cannot create zip file.") } + val := e.Ctx.Value(PlanRecreatorFileList) + if val == nil { + e.Ctx.SetValue(PlanRecreatorFileList, make(map[string]time.Time)) + } else { + // clean outdated files + Flist := val.(map[string]time.Time) + for k, v := range Flist { + if time.Since(v).Minutes() > remainedInterval { + err := os.Remove(recreatorPath + "/" + k) + if err != nil { + logutil.BgLogger().Warn(fmt.Sprintf("Cleaning outdated file %s failed.", k)) + } + delete(Flist, k) + } + } + } + e.Ctx.Value(PlanRecreatorFileList).(map[string]time.Time)[fileName] = startTime + + // Create zip writer zw := zip.NewWriter(zf) defer func() { - err = zw.Close() + err := zw.Close() if err != nil { logutil.BgLogger().Warn("Closing zip writer failed.") } - zf.Close() + err = zf.Close() + if err != nil { + logutil.BgLogger().Warn("Closing zip file failed.") + } }() // Dump config From ffb33ae4c6436b612341a04afa3c7ef7a74b942b Mon Sep 17 00:00:00 2001 From: zhuo-zhi <517770911@qq.com> Date: Mon, 26 Jul 2021 15:38:15 +0800 Subject: [PATCH 06/17] Support generating token --- executor/plan_recreator.go | 84 +++++++++++++++++++++++--------------- server/conn.go | 10 +++-- 2 files changed, 58 insertions(+), 36 deletions(-) diff --git a/executor/plan_recreator.go b/executor/plan_recreator.go index 38cf8e050a56a..a57772f2f77e5 100644 --- a/executor/plan_recreator.go +++ b/executor/plan_recreator.go @@ -16,6 +16,8 @@ package executor import ( "archive/zip" "context" + "crypto/md5" + "encoding/hex" "encoding/json" "fmt" "github.com/BurntSushi/toml" @@ -31,6 +33,7 @@ import ( "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/printer" "github.com/pingcap/tidb/util/sqlexec" + "math/rand" "os" "strings" "time" @@ -64,6 +67,16 @@ type tableNameExtractor struct { names map[tableNamePair]struct{} } +type fileInfo struct { + StartTime time.Time + Token [16]byte +} + +type fileList struct { + FileInfo map[string]fileInfo + TokenMap map[[16]byte]string +} + func (tne *tableNameExtractor) Enter(in ast.Node) (ast.Node, bool) { if _, ok := in.(*ast.TableName); ok { return in, true @@ -130,20 +143,20 @@ func (e *PlanRecreatorSingleExec) Open(ctx context.Context) error { } // Process dose the export/import work for reproducing sql queries. -func (e *PlanRecreatorSingleInfo) Process() error { +func (e *PlanRecreatorSingleInfo) Process() (interface{}, error) { // TODO: plan recreator load will be developed later if e.Load { - return nil + return nil, nil } else { return e.dumpSingle() } } -func (e *PlanRecreatorSingleInfo) dumpSingle() error { +func (e *PlanRecreatorSingleInfo) dumpSingle() (interface{}, error) { // Create path err := os.MkdirAll(recreatorPath, os.ModePerm) if err != nil { - return errors.New("Plan Recreator: cannot create plan recreator path.") + return nil, errors.New("Plan Recreator: cannot create plan recreator path.") } // Create zip file @@ -151,25 +164,30 @@ func (e *PlanRecreatorSingleInfo) dumpSingle() error { fileName := fmt.Sprintf("recreator_single_%v.zip", startTime.UnixNano()) zf, err := os.Create(recreatorPath + "/" + fileName) if err != nil { - return errors.New("Plan Recreator: cannot create zip file.") + return nil, errors.New("Plan Recreator: cannot create zip file.") } val := e.Ctx.Value(PlanRecreatorFileList) if val == nil { - e.Ctx.SetValue(PlanRecreatorFileList, make(map[string]time.Time)) + e.Ctx.SetValue(PlanRecreatorFileList, fileList{FileInfo: make(map[string]fileInfo), TokenMap: make(map[[16]byte]string)}) } else { - // clean outdated files - Flist := val.(map[string]time.Time) + // Clean outdated files + Flist := val.(fileList).FileInfo + TList := val.(fileList).TokenMap for k, v := range Flist { - if time.Since(v).Minutes() > remainedInterval { + if time.Since(v.StartTime).Minutes() > remainedInterval { err := os.Remove(recreatorPath + "/" + k) if err != nil { logutil.BgLogger().Warn(fmt.Sprintf("Cleaning outdated file %s failed.", k)) } delete(Flist, k) + delete(TList, v.Token) } } } - e.Ctx.Value(PlanRecreatorFileList).(map[string]time.Time)[fileName] = startTime + // Generate Token + token := md5.Sum([]byte(fmt.Sprintf("%s%d", fileName, rand.Int63()))) + e.Ctx.Value(PlanRecreatorFileList).(fileList).FileInfo[fileName] = fileInfo{StartTime: startTime, Token: token} + e.Ctx.Value(PlanRecreatorFileList).(fileList).TokenMap[token] = fileName // Create zip writer zw := zip.NewWriter(zf) @@ -187,20 +205,20 @@ func (e *PlanRecreatorSingleInfo) dumpSingle() error { // Dump config cf, err := zw.Create("config.toml") if err != nil { - return err + return nil, err } if err := toml.NewEncoder(cf).Encode(config.GetGlobalConfig()); err != nil { - return err + return nil, err } // Dump meta mt, err := zw.Create("meta.txt") if err != nil { - return err + return nil, err } _, err = mt.Write([]byte(printer.GetTiDBInfo())) if err != nil { - return err + return nil, err } // Retrieve current DB @@ -211,26 +229,26 @@ func (e *PlanRecreatorSingleInfo) dumpSingle() error { // Retrieve all tables pairs, err := extractTableNames(e.ExecStmt, dbName.L) if err != nil { - return errors.New(fmt.Sprintf("Plan Recreator: invalid SQL text, err: %v", err)) + return nil, errors.New(fmt.Sprintf("Plan Recreator: invalid SQL text, err: %v", err)) } // Dump stats for pair := range pairs { jsonTbl, err := getStatsForTable(do, pair) if err != nil { - return err + return nil, err } statsFw, err := zw.Create(fmt.Sprintf("stats/%v.%v.json", pair.DBName, pair.TableName)) if err != nil { - return err + return nil, err } data, err := json.Marshal(jsonTbl) if err != nil { - return err + return nil, err } _, err = statsFw.Write(data) if err != nil { - return err + return nil, err } } @@ -238,7 +256,7 @@ func (e *PlanRecreatorSingleInfo) dumpSingle() error { for pair := range pairs { err = getShowCreateTable(pair, zw, e.Ctx) if err != nil { - return err + return nil, err } } @@ -249,27 +267,27 @@ func (e *PlanRecreatorSingleInfo) dumpSingle() error { defer recordSets[0].Close() } if err != nil { - return err + return nil, err } sRows, err := resultSetToStringSlice(context.Background(), recordSets[0]) if err != nil { - return err + return nil, err } vf, err := zw.Create("variables.toml") if err != nil { - return err + return nil, err } for _, row := range sRows { varMap[row[0]] = row[1] } if err := toml.NewEncoder(vf).Encode(varMap); err != nil { - return err + return nil, err } // Dump sql sql, err := zw.Create("sqls.sql") if err != nil { - return nil + return nil, nil } sql.Write([]byte(e.ExecStmt.Text())) @@ -280,15 +298,15 @@ func (e *PlanRecreatorSingleInfo) dumpSingle() error { defer recordSets[0].Close() } if err != nil { - return err + return nil, err } sRows, err = resultSetToStringSlice(context.Background(), recordSets[0]) if err != nil { - return err + return nil, err } bf, err := zw.Create("bindings.sql") if err != nil { - return err + return nil, err } for _, row := range sRows { fmt.Fprintf(bf, "%s\n", strings.Join(row, "\t")) @@ -299,13 +317,13 @@ func (e *PlanRecreatorSingleInfo) dumpSingle() error { // Explain analyze recordSets, err = e.Ctx.(sqlexec.SQLExecutor).Execute(context.TODO(), fmt.Sprintf("explain analyze %s", e.ExecStmt.Text())) if err != nil { - return err + return nil, err } } else { // Explain recordSets, err = e.Ctx.(sqlexec.SQLExecutor).Execute(context.TODO(), fmt.Sprintf("explain %s", e.ExecStmt.Text())) if err != nil { - return err + return nil, err } } if len(recordSets) > 0 { @@ -313,16 +331,16 @@ func (e *PlanRecreatorSingleInfo) dumpSingle() error { } sRows, err = resultSetToStringSlice(context.Background(), recordSets[0]) if err != nil { - return err + return nil, err } fw, err := zw.Create("explain.txt") if err != nil { - return err + return nil, err } for _, row := range sRows { fmt.Fprintf(fw, "%s\n", strings.Join(row, "\t")) } - return nil + return hex.EncodeToString(token[:]), nil } func extractTableNames(ExecStmt ast.StmtNode, curDB string) (map[tableNamePair]struct{}, error) { diff --git a/server/conn.go b/server/conn.go index 8c0cc5853607f..33b89009dd8b6 100644 --- a/server/conn.go +++ b/server/conn.go @@ -1618,13 +1618,13 @@ func (cc *clientConn) handleIndexAdvise(ctx context.Context, indexAdviseInfo *ex } // handlePlanRecreator dose the export/import work for reproducing sql queries. -func (cc *clientConn) handlePlanRecreator(ctx context.Context, planRecreatorInfo interface{}) error { +func (cc *clientConn) handlePlanRecreator(ctx context.Context, planRecreatorInfo interface{}) (interface{}, error) { switch planRecreatorInfo.(type) { case *executor.PlanRecreatorSingleInfo: info := planRecreatorInfo.(*executor.PlanRecreatorSingleInfo) return info.Process() } - return errors.New("plan recreator: not supporting info type") + return nil, errors.New("plan recreator: not supporting info type") } // handleQuery executes the sql query string and writes result set or result ok to the client. @@ -1894,9 +1894,13 @@ func (cc *clientConn) handleQuerySpecial(ctx context.Context, status uint16) (bo if planRecreator != nil { handled = true defer cc.ctx.SetValue(executor.PlanRecreatorVarKey, nil) - if err := cc.handlePlanRecreator(ctx, planRecreator); err != nil { + token, err := cc.handlePlanRecreator(ctx, planRecreator) + if err != nil { return handled, err } + if token != nil { + return handled, cc.writeOkWith(ctx, token.(string), cc.ctx.AffectedRows(), cc.ctx.LastInsertID(), status, cc.ctx.WarningCount()) + } } return handled, cc.writeOkWith(ctx, cc.ctx.LastMessage(), cc.ctx.AffectedRows(), cc.ctx.LastInsertID(), status, cc.ctx.WarningCount()) From 6f72dc522e9ca45db562a067a6d543952ef8490f Mon Sep 17 00:00:00 2001 From: zhuo-zhi <517770911@qq.com> Date: Wed, 28 Jul 2021 15:01:36 +0800 Subject: [PATCH 07/17] update parser to version 20210728060616 --- executor/grant.go | 2 +- go.mod | 2 +- go.sum | 2 ++ 3 files changed, 4 insertions(+), 2 deletions(-) diff --git a/executor/grant.go b/executor/grant.go index d96e10d2729de..0d5a8cc2e46d4 100644 --- a/executor/grant.go +++ b/executor/grant.go @@ -360,7 +360,7 @@ func tlsOption2GlobalPriv(tlsOptions []*ast.TLSOption) (priv []byte, err error) gp := privileges.GlobalPrivValue{SSLType: privileges.SslTypeNotSpecified} for _, tlsOpt := range tlsOptions { switch tlsOpt.Type { - case ast.TslNone: + case ast.TlsNone: gp.SSLType = privileges.SslTypeNone case ast.Ssl: gp.SSLType = privileges.SslTypeAny diff --git a/go.mod b/go.mod index 2736597c2145f..326d329834caa 100644 --- a/go.mod +++ b/go.mod @@ -43,7 +43,7 @@ require ( github.com/pingcap/fn v0.0.0-20200306044125-d5540d389059 github.com/pingcap/kvproto v0.0.0-20210722091755-91a52cd9e8db github.com/pingcap/log v0.0.0-20210625125904-98ed8e2eb1c7 - github.com/pingcap/parser v0.0.0-20210707071004-31c87e37af5c + github.com/pingcap/parser v0.0.0-20210728060616-75cff0c906d2 github.com/pingcap/sysutil v0.0.0-20210315073920-cc0985d983a3 github.com/pingcap/tidb-tools v5.0.3+incompatible github.com/pingcap/tipb v0.0.0-20210708040514-0f154bb0dc0f diff --git a/go.sum b/go.sum index e39deeae4c432..60fb11a3a9d6e 100644 --- a/go.sum +++ b/go.sum @@ -447,6 +447,8 @@ github.com/pingcap/log v0.0.0-20210625125904-98ed8e2eb1c7/go.mod h1:8AanEdAHATuR github.com/pingcap/parser v0.0.0-20210525032559-c37778aff307/go.mod h1:xZC8I7bug4GJ5KtHhgAikjTfU4kBv1Sbo3Pf1MZ6lVw= github.com/pingcap/parser v0.0.0-20210707071004-31c87e37af5c h1:FPBMwDTtMW25zyLLBrIMUnM7Zl/WtLxR9LSEiaL5hII= github.com/pingcap/parser v0.0.0-20210707071004-31c87e37af5c/go.mod h1:Ek0mLKEqUGnQqBw1JnYrJQxsguU433DU68yUbsoeJ7s= +github.com/pingcap/parser v0.0.0-20210728060616-75cff0c906d2 h1:2cwX3RI5skZ4rEC1yz+VuS70DkLsCr2g9/Bur/UWBac= +github.com/pingcap/parser v0.0.0-20210728060616-75cff0c906d2/go.mod h1:Ek0mLKEqUGnQqBw1JnYrJQxsguU433DU68yUbsoeJ7s= github.com/pingcap/sysutil v0.0.0-20200206130906-2bfa6dc40bcd/go.mod h1:EB/852NMQ+aRKioCpToQ94Wl7fktV+FNnxf3CX/TTXI= github.com/pingcap/sysutil v0.0.0-20210315073920-cc0985d983a3 h1:A9KL9R+lWSVPH8IqUuH1QSTRJ5FGoY1bT2IcfPKsWD8= github.com/pingcap/sysutil v0.0.0-20210315073920-cc0985d983a3/go.mod h1:tckvA041UWP+NqYzrJ3fMgC/Hw9wnmQ/tUkp/JaHly8= From 6b8ade21583cdb8dc1a355f826f5aedd54863f4e Mon Sep 17 00:00:00 2001 From: zhuo-zhi <517770911@qq.com> Date: Wed, 28 Jul 2021 15:34:58 +0800 Subject: [PATCH 08/17] go mod tidy --- go.sum | 2 -- 1 file changed, 2 deletions(-) diff --git a/go.sum b/go.sum index 60fb11a3a9d6e..a2704fff9f651 100644 --- a/go.sum +++ b/go.sum @@ -445,8 +445,6 @@ github.com/pingcap/log v0.0.0-20210317133921-96f4fcab92a4/go.mod h1:4rbK1p9ILyIf github.com/pingcap/log v0.0.0-20210625125904-98ed8e2eb1c7 h1:k2BbABz9+TNpYRwsCCFS8pEEnFVOdbgEjL/kTlLuzZQ= github.com/pingcap/log v0.0.0-20210625125904-98ed8e2eb1c7/go.mod h1:8AanEdAHATuRurdGxZXBz0At+9avep+ub7U1AGYLIMM= github.com/pingcap/parser v0.0.0-20210525032559-c37778aff307/go.mod h1:xZC8I7bug4GJ5KtHhgAikjTfU4kBv1Sbo3Pf1MZ6lVw= -github.com/pingcap/parser v0.0.0-20210707071004-31c87e37af5c h1:FPBMwDTtMW25zyLLBrIMUnM7Zl/WtLxR9LSEiaL5hII= -github.com/pingcap/parser v0.0.0-20210707071004-31c87e37af5c/go.mod h1:Ek0mLKEqUGnQqBw1JnYrJQxsguU433DU68yUbsoeJ7s= github.com/pingcap/parser v0.0.0-20210728060616-75cff0c906d2 h1:2cwX3RI5skZ4rEC1yz+VuS70DkLsCr2g9/Bur/UWBac= github.com/pingcap/parser v0.0.0-20210728060616-75cff0c906d2/go.mod h1:Ek0mLKEqUGnQqBw1JnYrJQxsguU433DU68yUbsoeJ7s= github.com/pingcap/sysutil v0.0.0-20200206130906-2bfa6dc40bcd/go.mod h1:EB/852NMQ+aRKioCpToQ94Wl7fktV+FNnxf3CX/TTXI= From a6b99b7b03d95a73b5cafc6854ff94708d369096 Mon Sep 17 00:00:00 2001 From: zhuo-zhi <517770911@qq.com> Date: Wed, 28 Jul 2021 16:07:57 +0800 Subject: [PATCH 09/17] make fmt --- executor/plan_recreator.go | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/executor/plan_recreator.go b/executor/plan_recreator.go index a57772f2f77e5..2ae474f9754b7 100644 --- a/executor/plan_recreator.go +++ b/executor/plan_recreator.go @@ -20,6 +20,11 @@ import ( "encoding/hex" "encoding/json" "fmt" + "math/rand" + "os" + "strings" + "time" + "github.com/BurntSushi/toml" "github.com/pingcap/errors" "github.com/pingcap/parser" @@ -33,10 +38,6 @@ import ( "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/printer" "github.com/pingcap/tidb/util/sqlexec" - "math/rand" - "os" - "strings" - "time" ) const recreatorPath string = "/tmp/recreator" From 5db9741776c00cbe4cefaa133376b956dd95cb5d Mon Sep 17 00:00:00 2001 From: zhuo-zhi <517770911@qq.com> Date: Wed, 28 Jul 2021 16:22:47 +0800 Subject: [PATCH 10/17] reformat code --- executor/plan_recreator.go | 15 +++++++-------- 1 file changed, 7 insertions(+), 8 deletions(-) diff --git a/executor/plan_recreator.go b/executor/plan_recreator.go index 2ae474f9754b7..630c47dc55b0f 100644 --- a/executor/plan_recreator.go +++ b/executor/plan_recreator.go @@ -43,13 +43,13 @@ import ( const recreatorPath string = "/tmp/recreator" const remainedInterval float64 = 3 -// PlanRecreatorExec represents a plan recreator executor. +// PlanRecreatorSingleExec represents a plan recreator executor. type PlanRecreatorSingleExec struct { baseExecutor info *PlanRecreatorSingleInfo } -// PlanRecreatorInfo saves the information of plan recreator operation. +// PlanRecreatorSingleInfo saves the information of plan recreator operation. type PlanRecreatorSingleInfo struct { ExecStmt ast.StmtNode Analyze bool @@ -122,12 +122,12 @@ const PlanRecreatorFileList planRecreatorFileListType = 0 func (e *PlanRecreatorSingleExec) Next(ctx context.Context, req *chunk.Chunk) error { req.GrowAndReset(e.maxChunkSize) if e.info.ExecStmt == nil { - return errors.New("Plan Recreator: sql is empty.") + return errors.New("plan Recreator: sql is empty") } val := e.ctx.Value(PlanRecreatorVarKey) if val != nil { e.ctx.SetValue(PlanRecreatorVarKey, nil) - return errors.New("Plan Recreator: previous plan recreator option isn't closed normally.") + return errors.New("plan Recreator: previous plan recreator option isn't closed normally") } e.ctx.SetValue(PlanRecreatorVarKey, e.info) return nil @@ -148,16 +148,15 @@ func (e *PlanRecreatorSingleInfo) Process() (interface{}, error) { // TODO: plan recreator load will be developed later if e.Load { return nil, nil - } else { - return e.dumpSingle() } + return e.dumpSingle() } func (e *PlanRecreatorSingleInfo) dumpSingle() (interface{}, error) { // Create path err := os.MkdirAll(recreatorPath, os.ModePerm) if err != nil { - return nil, errors.New("Plan Recreator: cannot create plan recreator path.") + return nil, errors.New("plan Recreator: cannot create plan recreator path") } // Create zip file @@ -165,7 +164,7 @@ func (e *PlanRecreatorSingleInfo) dumpSingle() (interface{}, error) { fileName := fmt.Sprintf("recreator_single_%v.zip", startTime.UnixNano()) zf, err := os.Create(recreatorPath + "/" + fileName) if err != nil { - return nil, errors.New("Plan Recreator: cannot create zip file.") + return nil, errors.New("plan Recreator: cannot create zip file") } val := e.Ctx.Value(PlanRecreatorFileList) if val == nil { From 8332692cf8b8a7d48497e460922f2755ee376146 Mon Sep 17 00:00:00 2001 From: zhuo-zhi <517770911@qq.com> Date: Wed, 28 Jul 2021 16:47:39 +0800 Subject: [PATCH 11/17] reformat code --- executor/plan_recreator.go | 41 +++++++++++++++++++++++--------------- server/conn.go | 7 +++---- 2 files changed, 28 insertions(+), 20 deletions(-) diff --git a/executor/plan_recreator.go b/executor/plan_recreator.go index 630c47dc55b0f..21ddeaaeb790a 100644 --- a/executor/plan_recreator.go +++ b/executor/plan_recreator.go @@ -27,7 +27,6 @@ import ( "github.com/BurntSushi/toml" "github.com/pingcap/errors" - "github.com/pingcap/parser" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/model" "github.com/pingcap/tidb/config" @@ -263,9 +262,6 @@ func (e *PlanRecreatorSingleInfo) dumpSingle() (interface{}, error) { // Dump variables varMap := make(map[string]string) recordSets, err := e.Ctx.(sqlexec.SQLExecutor).Execute(context.TODO(), "show variables") - if len(recordSets) > 0 { - defer recordSets[0].Close() - } if err != nil { return nil, err } @@ -283,20 +279,24 @@ func (e *PlanRecreatorSingleInfo) dumpSingle() (interface{}, error) { if err := toml.NewEncoder(vf).Encode(varMap); err != nil { return nil, err } + if len(recordSets) > 0 { + if err := recordSets[0].Close(); err != nil { + return nil, err + } + } // Dump sql sql, err := zw.Create("sqls.sql") if err != nil { return nil, nil } - sql.Write([]byte(e.ExecStmt.Text())) + _, err = sql.Write([]byte(e.ExecStmt.Text())) + if err != nil { + return nil, err + } // Dump bindings - normSql, _ := parser.NormalizeDigest(e.ExecStmt.Text()) - recordSets, err = e.Ctx.(sqlexec.SQLExecutor).Execute(context.TODO(), fmt.Sprintf("show bindings where Original_sql='%s'", normSql)) - if len(recordSets) > 0 { - defer recordSets[0].Close() - } + recordSets, err = e.Ctx.(sqlexec.SQLExecutor).Execute(context.TODO(), fmt.Sprintf("show bindings")) if err != nil { return nil, err } @@ -311,6 +311,11 @@ func (e *PlanRecreatorSingleInfo) dumpSingle() (interface{}, error) { for _, row := range sRows { fmt.Fprintf(bf, "%s\n", strings.Join(row, "\t")) } + if len(recordSets) > 0 { + if err := recordSets[0].Close(); err != nil { + return nil, err + } + } // Dump explain if e.Analyze { @@ -326,9 +331,6 @@ func (e *PlanRecreatorSingleInfo) dumpSingle() (interface{}, error) { return nil, err } } - if len(recordSets) > 0 { - defer recordSets[0].Close() - } sRows, err = resultSetToStringSlice(context.Background(), recordSets[0]) if err != nil { return nil, err @@ -340,6 +342,11 @@ func (e *PlanRecreatorSingleInfo) dumpSingle() (interface{}, error) { for _, row := range sRows { fmt.Fprintf(fw, "%s\n", strings.Join(row, "\t")) } + if len(recordSets) > 0 { + if err := recordSets[0].Close(); err != nil { + return nil, err + } + } return hex.EncodeToString(token[:]), nil } @@ -365,9 +372,6 @@ func getStatsForTable(do *domain.Domain, pair tableNamePair) (*handle.JSONTable, func getShowCreateTable(pair tableNamePair, zw *zip.Writer, ctx sessionctx.Context) error { recordSets, err := ctx.(sqlexec.SQLExecutor).Execute(context.TODO(), fmt.Sprintf("show create table `%v`.`%v`", pair.DBName, pair.TableName)) - if len(recordSets) > 0 { - defer recordSets[0].Close() - } if err != nil { return err } @@ -382,6 +386,11 @@ func getShowCreateTable(pair tableNamePair, zw *zip.Writer, ctx sessionctx.Conte for _, row := range sRows { fmt.Fprintf(fw, "%s\n", strings.Join(row, "\t")) } + if len(recordSets) > 0 { + if err := recordSets[0].Close(); err != nil { + return err + } + } return nil } diff --git a/server/conn.go b/server/conn.go index 33b89009dd8b6..d9dd5339dcb17 100644 --- a/server/conn.go +++ b/server/conn.go @@ -1618,11 +1618,10 @@ func (cc *clientConn) handleIndexAdvise(ctx context.Context, indexAdviseInfo *ex } // handlePlanRecreator dose the export/import work for reproducing sql queries. -func (cc *clientConn) handlePlanRecreator(ctx context.Context, planRecreatorInfo interface{}) (interface{}, error) { - switch planRecreatorInfo.(type) { +func (cc *clientConn) handlePlanRecreator(ctx context.Context, info interface{}) (interface{}, error) { + switch info.(type) { case *executor.PlanRecreatorSingleInfo: - info := planRecreatorInfo.(*executor.PlanRecreatorSingleInfo) - return info.Process() + return info.(*executor.PlanRecreatorSingleInfo).Process() } return nil, errors.New("plan recreator: not supporting info type") } From e9dbe3b9f7506cc6babb6b9829cf6da4848f2583 Mon Sep 17 00:00:00 2001 From: zhuo-zhi <517770911@qq.com> Date: Wed, 28 Jul 2021 17:02:27 +0800 Subject: [PATCH 12/17] reformat code --- executor/plan_recreator.go | 4 +++- server/conn.go | 7 +++---- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/executor/plan_recreator.go b/executor/plan_recreator.go index 21ddeaaeb790a..648fba773d210 100644 --- a/executor/plan_recreator.go +++ b/executor/plan_recreator.go @@ -113,8 +113,10 @@ func (k planRecreatorFileListType) String() string { return "plan_recreator_file_list" } -// PlanRecreatorVarKey is a variable key for load statistic. +// PlanRecreatorVarKey is a variable key for plan recreator. const PlanRecreatorVarKey planRecreatorVarKeyType = 0 + +// PlanRecreatorFileList is a variable key for plan recreator's file list. const PlanRecreatorFileList planRecreatorFileListType = 0 // Next implements the Executor Next interface. diff --git a/server/conn.go b/server/conn.go index d9dd5339dcb17..0ac40e1899b64 100644 --- a/server/conn.go +++ b/server/conn.go @@ -1618,10 +1618,9 @@ func (cc *clientConn) handleIndexAdvise(ctx context.Context, indexAdviseInfo *ex } // handlePlanRecreator dose the export/import work for reproducing sql queries. -func (cc *clientConn) handlePlanRecreator(ctx context.Context, info interface{}) (interface{}, error) { - switch info.(type) { - case *executor.PlanRecreatorSingleInfo: - return info.(*executor.PlanRecreatorSingleInfo).Process() +func (cc *clientConn) handlePlanRecreator(ctx context.Context, PlanRecreatorInfo interface{}) (interface{}, error) { + if info, ok := PlanRecreatorInfo.(*executor.PlanRecreatorSingleInfo); ok { + return info.Process() } return nil, errors.New("plan recreator: not supporting info type") } From 97a927d7b7f91e95af840d792acb370989065b08 Mon Sep 17 00:00:00 2001 From: zhuo-zhi <517770911@qq.com> Date: Wed, 28 Jul 2021 17:09:05 +0800 Subject: [PATCH 13/17] reformat code --- executor/plan_recreator.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/executor/plan_recreator.go b/executor/plan_recreator.go index 648fba773d210..2d6bf4769f0a5 100644 --- a/executor/plan_recreator.go +++ b/executor/plan_recreator.go @@ -298,7 +298,7 @@ func (e *PlanRecreatorSingleInfo) dumpSingle() (interface{}, error) { } // Dump bindings - recordSets, err = e.Ctx.(sqlexec.SQLExecutor).Execute(context.TODO(), fmt.Sprintf("show bindings")) + recordSets, err = e.Ctx.(sqlexec.SQLExecutor).Execute(context.TODO(), "show bindings") if err != nil { return nil, err } From 6e0c56ce0033f6e90fdfc9e32197778fea3ebe80 Mon Sep 17 00:00:00 2001 From: zhuo-zhi <517770911@qq.com> Date: Fri, 30 Jul 2021 10:53:38 +0800 Subject: [PATCH 14/17] add PlanRecreatorInfo interface --- executor/plan_recreator.go | 70 +++++++++++++++++++++----------------- server/conn.go | 13 +++---- 2 files changed, 45 insertions(+), 38 deletions(-) diff --git a/executor/plan_recreator.go b/executor/plan_recreator.go index 2d6bf4769f0a5..2d48b3e205f78 100644 --- a/executor/plan_recreator.go +++ b/executor/plan_recreator.go @@ -42,13 +42,19 @@ import ( const recreatorPath string = "/tmp/recreator" const remainedInterval float64 = 3 +// PlanRecreatorInfo saves the information of plan recreator operation. +type PlanRecreatorInfo interface { + // Process dose the export/import work for reproducing sql queries. + Process() (string, error) +} + // PlanRecreatorSingleExec represents a plan recreator executor. type PlanRecreatorSingleExec struct { baseExecutor info *PlanRecreatorSingleInfo } -// PlanRecreatorSingleInfo saves the information of plan recreator operation. +// PlanRecreatorSingleInfo saves the information of plan recreator operation for single SQL statement. type PlanRecreatorSingleInfo struct { ExecStmt ast.StmtNode Analyze bool @@ -145,19 +151,19 @@ func (e *PlanRecreatorSingleExec) Open(ctx context.Context) error { } // Process dose the export/import work for reproducing sql queries. -func (e *PlanRecreatorSingleInfo) Process() (interface{}, error) { +func (e *PlanRecreatorSingleInfo) Process() (string, error) { // TODO: plan recreator load will be developed later if e.Load { - return nil, nil + return "", nil } return e.dumpSingle() } -func (e *PlanRecreatorSingleInfo) dumpSingle() (interface{}, error) { +func (e *PlanRecreatorSingleInfo) dumpSingle() (string, error) { // Create path err := os.MkdirAll(recreatorPath, os.ModePerm) if err != nil { - return nil, errors.New("plan Recreator: cannot create plan recreator path") + return "", errors.New("plan Recreator: cannot create plan recreator path") } // Create zip file @@ -165,7 +171,7 @@ func (e *PlanRecreatorSingleInfo) dumpSingle() (interface{}, error) { fileName := fmt.Sprintf("recreator_single_%v.zip", startTime.UnixNano()) zf, err := os.Create(recreatorPath + "/" + fileName) if err != nil { - return nil, errors.New("plan Recreator: cannot create zip file") + return "", errors.New("plan Recreator: cannot create zip file") } val := e.Ctx.Value(PlanRecreatorFileList) if val == nil { @@ -206,20 +212,20 @@ func (e *PlanRecreatorSingleInfo) dumpSingle() (interface{}, error) { // Dump config cf, err := zw.Create("config.toml") if err != nil { - return nil, err + return "", err } if err := toml.NewEncoder(cf).Encode(config.GetGlobalConfig()); err != nil { - return nil, err + return "", err } // Dump meta mt, err := zw.Create("meta.txt") if err != nil { - return nil, err + return "", err } _, err = mt.Write([]byte(printer.GetTiDBInfo())) if err != nil { - return nil, err + return "", err } // Retrieve current DB @@ -230,26 +236,26 @@ func (e *PlanRecreatorSingleInfo) dumpSingle() (interface{}, error) { // Retrieve all tables pairs, err := extractTableNames(e.ExecStmt, dbName.L) if err != nil { - return nil, errors.New(fmt.Sprintf("Plan Recreator: invalid SQL text, err: %v", err)) + return "", errors.New(fmt.Sprintf("Plan Recreator: invalid SQL text, err: %v", err)) } // Dump stats for pair := range pairs { jsonTbl, err := getStatsForTable(do, pair) if err != nil { - return nil, err + return "", err } statsFw, err := zw.Create(fmt.Sprintf("stats/%v.%v.json", pair.DBName, pair.TableName)) if err != nil { - return nil, err + return "", err } data, err := json.Marshal(jsonTbl) if err != nil { - return nil, err + return "", err } _, err = statsFw.Write(data) if err != nil { - return nil, err + return "", err } } @@ -257,7 +263,7 @@ func (e *PlanRecreatorSingleInfo) dumpSingle() (interface{}, error) { for pair := range pairs { err = getShowCreateTable(pair, zw, e.Ctx) if err != nil { - return nil, err + return "", err } } @@ -265,57 +271,57 @@ func (e *PlanRecreatorSingleInfo) dumpSingle() (interface{}, error) { varMap := make(map[string]string) recordSets, err := e.Ctx.(sqlexec.SQLExecutor).Execute(context.TODO(), "show variables") if err != nil { - return nil, err + return "", err } sRows, err := resultSetToStringSlice(context.Background(), recordSets[0]) if err != nil { - return nil, err + return "", err } vf, err := zw.Create("variables.toml") if err != nil { - return nil, err + return "", err } for _, row := range sRows { varMap[row[0]] = row[1] } if err := toml.NewEncoder(vf).Encode(varMap); err != nil { - return nil, err + return "", err } if len(recordSets) > 0 { if err := recordSets[0].Close(); err != nil { - return nil, err + return "", err } } // Dump sql sql, err := zw.Create("sqls.sql") if err != nil { - return nil, nil + return "", nil } _, err = sql.Write([]byte(e.ExecStmt.Text())) if err != nil { - return nil, err + return "", err } // Dump bindings recordSets, err = e.Ctx.(sqlexec.SQLExecutor).Execute(context.TODO(), "show bindings") if err != nil { - return nil, err + return "", err } sRows, err = resultSetToStringSlice(context.Background(), recordSets[0]) if err != nil { - return nil, err + return "", err } bf, err := zw.Create("bindings.sql") if err != nil { - return nil, err + return "", err } for _, row := range sRows { fmt.Fprintf(bf, "%s\n", strings.Join(row, "\t")) } if len(recordSets) > 0 { if err := recordSets[0].Close(); err != nil { - return nil, err + return "", err } } @@ -324,29 +330,29 @@ func (e *PlanRecreatorSingleInfo) dumpSingle() (interface{}, error) { // Explain analyze recordSets, err = e.Ctx.(sqlexec.SQLExecutor).Execute(context.TODO(), fmt.Sprintf("explain analyze %s", e.ExecStmt.Text())) if err != nil { - return nil, err + return "", err } } else { // Explain recordSets, err = e.Ctx.(sqlexec.SQLExecutor).Execute(context.TODO(), fmt.Sprintf("explain %s", e.ExecStmt.Text())) if err != nil { - return nil, err + return "", err } } sRows, err = resultSetToStringSlice(context.Background(), recordSets[0]) if err != nil { - return nil, err + return "", err } fw, err := zw.Create("explain.txt") if err != nil { - return nil, err + return "", err } for _, row := range sRows { fmt.Fprintf(fw, "%s\n", strings.Join(row, "\t")) } if len(recordSets) > 0 { if err := recordSets[0].Close(); err != nil { - return nil, err + return "", err } } return hex.EncodeToString(token[:]), nil diff --git a/server/conn.go b/server/conn.go index 0ac40e1899b64..259c04ba6828c 100644 --- a/server/conn.go +++ b/server/conn.go @@ -1618,11 +1618,12 @@ func (cc *clientConn) handleIndexAdvise(ctx context.Context, indexAdviseInfo *ex } // handlePlanRecreator dose the export/import work for reproducing sql queries. -func (cc *clientConn) handlePlanRecreator(ctx context.Context, PlanRecreatorInfo interface{}) (interface{}, error) { - if info, ok := PlanRecreatorInfo.(*executor.PlanRecreatorSingleInfo); ok { +func (cc *clientConn) handlePlanRecreator(ctx context.Context, info executor.PlanRecreatorInfo) (string, error) { + switch info.(type) { + case *executor.PlanRecreatorSingleInfo: return info.Process() } - return nil, errors.New("plan recreator: not supporting info type") + return "", errors.New("plan recreator: not supporting info type") } // handleQuery executes the sql query string and writes result set or result ok to the client. @@ -1892,12 +1893,12 @@ func (cc *clientConn) handleQuerySpecial(ctx context.Context, status uint16) (bo if planRecreator != nil { handled = true defer cc.ctx.SetValue(executor.PlanRecreatorVarKey, nil) - token, err := cc.handlePlanRecreator(ctx, planRecreator) + token, err := cc.handlePlanRecreator(ctx, planRecreator.(executor.PlanRecreatorInfo)) if err != nil { return handled, err } - if token != nil { - return handled, cc.writeOkWith(ctx, token.(string), cc.ctx.AffectedRows(), cc.ctx.LastInsertID(), status, cc.ctx.WarningCount()) + if token != "" { + return handled, cc.writeOkWith(ctx, token, cc.ctx.AffectedRows(), cc.ctx.LastInsertID(), status, cc.ctx.WarningCount()) } } From 109fe2eb4320a34fd430aa811724c906817416b4 Mon Sep 17 00:00:00 2001 From: rebelice Date: Thu, 5 Aug 2021 03:02:00 +0800 Subject: [PATCH 15/17] Update executor/plan_recreator.go Co-authored-by: Zhou Kunqin <25057648+time-and-fate@users.noreply.github.com> --- executor/plan_recreator.go | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/executor/plan_recreator.go b/executor/plan_recreator.go index 2d48b3e205f78..a6814d181a5de 100644 --- a/executor/plan_recreator.go +++ b/executor/plan_recreator.go @@ -412,8 +412,7 @@ func resultSetToStringSlice(ctx context.Context, rs sqlexec.RecordSet) ([][]stri return nil, err } sRows := make([][]string, len(rows)) - for i := range rows { - row := rows[i] + for i, row := range rows { iRow := make([]string, row.Len()) for j := 0; j < row.Len(); j++ { if row.IsNull(j) { From 986fe56305889bb78f66aebaccb5998ec5ed5bb2 Mon Sep 17 00:00:00 2001 From: rebelice Date: Thu, 5 Aug 2021 04:00:52 +0800 Subject: [PATCH 16/17] clean up files when closing session --- executor/plan_recreator.go | 46 ++++++++++++++++++++++++++++++++------ session/session.go | 3 +++ 2 files changed, 42 insertions(+), 7 deletions(-) diff --git a/executor/plan_recreator.go b/executor/plan_recreator.go index 4faed8aa52c98..108eb5be9e020 100644 --- a/executor/plan_recreator.go +++ b/executor/plan_recreator.go @@ -163,17 +163,18 @@ func (e *PlanRecreatorSingleInfo) Process() (string, error) { func (e *PlanRecreatorSingleInfo) dumpSingle() (string, error) { // Create path - err := os.MkdirAll(recreatorPath, os.ModePerm) + path := fmt.Sprintf("%s/%v", recreatorPath, e.Ctx.GetSessionVars().ConnectionID) + err := os.MkdirAll(path, os.ModePerm) if err != nil { - return "", errors.New("plan Recreator: cannot create plan recreator path") + return "", errors.New("Plan Recreator: cannot create plan recreator path") } // Create zip file startTime := time.Now() fileName := fmt.Sprintf("recreator_single_%v.zip", startTime.UnixNano()) - zf, err := os.Create(recreatorPath + "/" + fileName) + zf, err := os.Create(path + "/" + fileName) if err != nil { - return "", errors.New("plan Recreator: cannot create zip file") + return "", errors.New("Plan Recreator: cannot create zip file") } val := e.Ctx.Value(PlanRecreatorFileList) if val == nil { @@ -184,7 +185,7 @@ func (e *PlanRecreatorSingleInfo) dumpSingle() (string, error) { TList := val.(fileList).TokenMap for k, v := range Flist { if time.Since(v.StartTime).Minutes() > remainedInterval { - err := os.Remove(recreatorPath + "/" + k) + err := os.Remove(path + "/" + k) if err != nil { logutil.BgLogger().Warn(fmt.Sprintf("Cleaning outdated file %s failed.", k)) } @@ -305,7 +306,7 @@ func (e *PlanRecreatorSingleInfo) dumpSingle() (string, error) { return "", err } - // Dump bindings + // Dump session bindings recordSets, err = e.Ctx.(sqlexec.SQLExecutor).Execute(context.TODO(), "show bindings") if err != nil { return "", err @@ -314,7 +315,29 @@ func (e *PlanRecreatorSingleInfo) dumpSingle() (string, error) { if err != nil { return "", err } - bf, err := zw.Create("bindings.sql") + bf, err := zw.Create("session_bindings.sql") + if err != nil { + return "", err + } + for _, row := range sRows { + fmt.Fprintf(bf, "%s\n", strings.Join(row, "\t")) + } + if len(recordSets) > 0 { + if err := recordSets[0].Close(); err != nil { + return "", err + } + } + + // Dump global bindings + recordSets, err = e.Ctx.(sqlexec.SQLExecutor).Execute(context.TODO(), "show global bindings") + if err != nil { + return "", err + } + sRows, err = resultSetToStringSlice(context.Background(), recordSets[0]) + if err != nil { + return "", err + } + bf, err = zw.Create("global_bindings.sql") if err != nil { return "", err } @@ -455,3 +478,12 @@ func getRows4Test(ctx context.Context, rs sqlexec.RecordSet) ([]chunk.Row, error } return rows, nil } + +// CleanUpPlanRecreatorFile cleans files corresponding to the session. +func CleanUpPlanRecreatorFile(id uint64) { + path := fmt.Sprintf("%s/%v", recreatorPath, id) + err := os.RemoveAll(path) + if err != nil { + logutil.BgLogger().Warn(fmt.Sprintf("Cleaning up plan recreator file %s failed.", path)) + } +} diff --git a/session/session.go b/session/session.go index 50137fc3db09e..0252827a82036 100644 --- a/session/session.go +++ b/session/session.go @@ -2254,6 +2254,9 @@ func (s *session) Close() { if bindValue != nil { bindValue.(*bindinfo.SessionHandle).Close() } + if s.Value(executor.PlanRecreatorFileList) != nil { + executor.CleanUpPlanRecreatorFile(s.GetSessionVars().ConnectionID) + } ctx := context.WithValue(context.TODO(), inCloseSession{}, struct{}{}) s.RollbackTxn(ctx) if s.sessionVars != nil { From b99803fea3d9e51ae3926f1c62551494388599fb Mon Sep 17 00:00:00 2001 From: rebelice Date: Mon, 9 Aug 2021 17:43:24 +0800 Subject: [PATCH 17/17] remove plan recreator from server/conn.go --- executor/builder.go | 2 +- executor/plan_recreator.go | 32 ++++++++++++-------------------- planner/core/logical_plans.go | 14 ++++++++++++++ planner/core/planbuilder.go | 4 ++++ server/conn.go | 22 ---------------------- 5 files changed, 31 insertions(+), 43 deletions(-) diff --git a/executor/builder.go b/executor/builder.go index 1c6b18dc51d88..03c040fac2d5f 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -913,7 +913,7 @@ func (b *executorBuilder) buildIndexAdvise(v *plannercore.IndexAdvise) Executor func (b *executorBuilder) buildPlanRecreatorSingle(v *plannercore.PlanRecreatorSingle) Executor { e := &PlanRecreatorSingleExec{ - baseExecutor: newBaseExecutor(b.ctx, nil, v.ID()), + baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ID()), info: &PlanRecreatorSingleInfo{v.ExecStmt, v.Analyze, v.Load, v.File, b.ctx}, } return e diff --git a/executor/plan_recreator.go b/executor/plan_recreator.go index 108eb5be9e020..6e774d0c3d39a 100644 --- a/executor/plan_recreator.go +++ b/executor/plan_recreator.go @@ -44,16 +44,12 @@ const recreatorPath string = "/tmp/recreator" // TTL of plan recreator files const remainedInterval float64 = 3 -// PlanRecreatorInfo saves the information of plan recreator operation. -type PlanRecreatorInfo interface { - // Process dose the export/import work for reproducing sql queries. - Process() (string, error) -} - // PlanRecreatorSingleExec represents a plan recreator executor. type PlanRecreatorSingleExec struct { baseExecutor info *PlanRecreatorSingleInfo + + endFlag bool } // PlanRecreatorSingleInfo saves the information of plan recreator operation for single SQL statement. @@ -130,15 +126,20 @@ const PlanRecreatorFileList planRecreatorFileListType = 0 // Next implements the Executor Next interface. func (e *PlanRecreatorSingleExec) Next(ctx context.Context, req *chunk.Chunk) error { req.GrowAndReset(e.maxChunkSize) + if e.endFlag { + return nil + } if e.info.ExecStmt == nil { return errors.New("plan Recreator: sql is empty") } - val := e.ctx.Value(PlanRecreatorVarKey) - if val != nil { - e.ctx.SetValue(PlanRecreatorVarKey, nil) - return errors.New("plan Recreator: previous plan recreator option isn't closed normally") + res, err := e.info.dumpSingle() + if err != nil { + return err } - e.ctx.SetValue(PlanRecreatorVarKey, e.info) + result := newFirstChunk(e) + result.AppendString(0, res) + req.Append(result, 0, 1) + e.endFlag = true return nil } @@ -152,15 +153,6 @@ func (e *PlanRecreatorSingleExec) Open(ctx context.Context) error { return nil } -// Process dose the export/import work for reproducing sql queries. -func (e *PlanRecreatorSingleInfo) Process() (string, error) { - // TODO: plan recreator load will be developed later - if e.Load { - return "", nil - } - return e.dumpSingle() -} - func (e *PlanRecreatorSingleInfo) dumpSingle() (string, error) { // Create path path := fmt.Sprintf("%s/%v", recreatorPath, e.Ctx.GetSessionVars().ConnectionID) diff --git a/planner/core/logical_plans.go b/planner/core/logical_plans.go index b6084fb339466..d64475e0c05d4 100644 --- a/planner/core/logical_plans.go +++ b/planner/core/logical_plans.go @@ -1144,6 +1144,20 @@ func ExtractCorColumnsBySchema4PhysicalPlan(p PhysicalPlan, schema *expression.S return ExtractCorColumnsBySchema(corCols, schema, true) } +type PlanRecreatorContents struct { + baseSchemaProducer + ExecStmt ast.StmtNode + Analyze bool + Load bool + File string +} + +type LogicalPlanRecreator struct { + logicalSchemaProducer + + PlanRecreatorContents +} + // ShowContents stores the contents for the `SHOW` statement. type ShowContents struct { Tp ast.ShowStmtType // Databases/Tables/Columns/.... diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 2422fa301709f..6b9a598c62630 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -4141,6 +4141,10 @@ func buildShowSchema(s *ast.ShowStmt, isView bool, isSequence bool) (schema *exp func (b *PlanBuilder) buildPlanRecreator(pc *ast.PlanRecreatorStmt) Plan { p := &PlanRecreatorSingle{ExecStmt: pc.Stmt, Analyze: pc.Analyze, Load: pc.Load, File: pc.File} + schema := newColumnsWithNames(1) + schema.Append(buildColumnWithName("", "Dump_link", mysql.TypeVarchar, 128)) + p.SetSchema(schema.col2Schema()) + p.names = schema.names return p } diff --git a/server/conn.go b/server/conn.go index 26203c2044321..3acb2240db828 100644 --- a/server/conn.go +++ b/server/conn.go @@ -1617,15 +1617,6 @@ func (cc *clientConn) handleIndexAdvise(ctx context.Context, indexAdviseInfo *ex return nil } -// handlePlanRecreator dose the export/import work for reproducing sql queries. -func (cc *clientConn) handlePlanRecreator(ctx context.Context, info executor.PlanRecreatorInfo) (string, error) { - switch info.(type) { - case *executor.PlanRecreatorSingleInfo: - return info.Process() - } - return "", errors.New("plan recreator: not supporting info type") -} - // handleQuery executes the sql query string and writes result set or result ok to the client. // As the execution time of this function represents the performance of TiDB, we do time log and metrics here. // There is a special query `load data` that does not return result, which is handled differently. @@ -1889,19 +1880,6 @@ func (cc *clientConn) handleQuerySpecial(ctx context.Context, status uint16) (bo } } - planRecreator := cc.ctx.Value(executor.PlanRecreatorVarKey) - if planRecreator != nil { - handled = true - defer cc.ctx.SetValue(executor.PlanRecreatorVarKey, nil) - token, err := cc.handlePlanRecreator(ctx, planRecreator.(executor.PlanRecreatorInfo)) - if err != nil { - return handled, err - } - if token != "" { - return handled, cc.writeOkWith(ctx, token, cc.ctx.AffectedRows(), cc.ctx.LastInsertID(), status, cc.ctx.WarningCount()) - } - } - return handled, cc.writeOkWith(ctx, cc.ctx.LastMessage(), cc.ctx.AffectedRows(), cc.ctx.LastInsertID(), status, cc.ctx.WarningCount()) }