Skip to content

Commit

Permalink
*: rename RECREATOR to REPLAYER (#28826)
Browse files Browse the repository at this point in the history
  • Loading branch information
rebelice authored Oct 15, 2021
1 parent 68440be commit cc96f59
Show file tree
Hide file tree
Showing 12 changed files with 610 additions and 610 deletions.
10 changes: 5 additions & 5 deletions executor/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -162,8 +162,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.PlanReplayerSingle:
return b.buildPlanReplayerSingle(v)
case *plannercore.PhysicalLimit:
return b.buildLimit(v)
case *plannercore.Prepare:
Expand Down Expand Up @@ -905,10 +905,10 @@ func (b *executorBuilder) buildIndexAdvise(v *plannercore.IndexAdvise) Executor
return e
}

func (b *executorBuilder) buildPlanRecreatorSingle(v *plannercore.PlanRecreatorSingle) Executor {
e := &PlanRecreatorSingleExec{
func (b *executorBuilder) buildPlanReplayerSingle(v *plannercore.PlanReplayerSingle) Executor {
e := &PlanReplayerSingleExec{
baseExecutor: newBaseExecutor(b.ctx, nil, v.ID()),
info: &PlanRecreatorSingleInfo{v.ExecStmt, v.Analyze, v.Load, v.File, b.ctx},
info: &PlanReplayerSingleInfo{v.ExecStmt, v.Analyze, v.Load, v.File, b.ctx},
}
return e
}
Expand Down
4 changes: 2 additions & 2 deletions executor/executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -282,12 +282,12 @@ func (s *testSuiteP1) TestLoadStats(c *C) {
c.Assert(tk.ExecToErr("load stats ./xxx.json"), NotNil)
}

func (s *testSuiteP1) TestPlanRecreator(c *C) {
func (s *testSuiteP1) TestPlanReplayer(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")
tk.MustExec("plan replayer dump explain select * from t where a=10")
}

func (s *testSuiteP1) TestShow(c *C) {
Expand Down
90 changes: 45 additions & 45 deletions executor/plan_recreator.go → executor/plan_replayer.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,25 +31,25 @@ import (
"github.com/pingcap/tidb/util/logutil"
)

const recreatorPath string = "/tmp/recreator"
const replayerPath string = "/tmp/replayer"

// TTL of plan recreator files
// TTL of plan replayer files
const remainedInterval float64 = 3

// PlanRecreatorInfo saves the information of plan recreator operation.
type PlanRecreatorInfo interface {
// Process does the export/import work for reproducing sql queries.
// PlanReplayerInfo saves the information of plan replayer operation.
type PlanReplayerInfo interface {
// Process dose the export/import work for reproducing sql queries.
Process() (string, error)
}

// PlanRecreatorSingleExec represents a plan recreator executor.
type PlanRecreatorSingleExec struct {
// PlanReplayerSingleExec represents a plan replayer executor.
type PlanReplayerSingleExec struct {
baseExecutor
info *PlanRecreatorSingleInfo
info *PlanReplayerSingleInfo
}

// PlanRecreatorSingleInfo saves the information of plan recreator operation.
type PlanRecreatorSingleInfo struct {
// PlanReplayerSingleInfo saves the information of plan replayer operation.
type PlanReplayerSingleInfo struct {
ExecStmt ast.StmtNode
Analyze bool
Load bool
Expand All @@ -67,86 +67,86 @@ type fileList struct {
TokenMap map[[16]byte]string
}

// planRecreatorVarKeyType is a dummy type to avoid naming collision in context.
type planRecreatorVarKeyType int
// planReplayerVarKeyType is a dummy type to avoid naming collision in context.
type planReplayerVarKeyType int

// String defines a Stringer function for debugging and pretty printing.
func (k planRecreatorVarKeyType) String() string {
return "plan_recreator_var"
func (k planReplayerVarKeyType) String() string {
return "plan_replayer_var"
}

// planRecreatorFileListType is a dummy type to avoid naming collision in context.
type planRecreatorFileListType int
// planReplayerFileListType is a dummy type to avoid naming collision in context.
type planReplayerFileListType int

// String defines a Stringer function for debugging and pretty printing.
func (k planRecreatorFileListType) String() string {
return "plan_recreator_file_list"
func (k planReplayerFileListType) String() string {
return "plan_replayer_file_list"
}

// PlanRecreatorVarKey is a variable key for plan recreator.
const PlanRecreatorVarKey planRecreatorVarKeyType = 0
// PlanReplayerVarKey is a variable key for plan replayer.
const PlanReplayerVarKey planReplayerVarKeyType = 0

// PlanRecreatorFileList is a variable key for plan recreator's file list.
const PlanRecreatorFileList planRecreatorFileListType = 0
// PlanReplayerFileList is a variable key for plan replayer's file list.
const PlanReplayerFileList planReplayerFileListType = 0

// Next implements the Executor Next interface.
func (e *PlanRecreatorSingleExec) Next(ctx context.Context, req *chunk.Chunk) error {
func (e *PlanReplayerSingleExec) 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 replayer: sql is empty")
}
val := e.ctx.Value(PlanRecreatorVarKey)
val := e.ctx.Value(PlanReplayerVarKey)
if val != nil {
e.ctx.SetValue(PlanRecreatorVarKey, nil)
return errors.New("plan recreator: previous plan recreator option isn't closed normally")
e.ctx.SetValue(PlanReplayerVarKey, nil)
return errors.New("plan replayer: previous plan replayer option isn't closed normally")
}
e.ctx.SetValue(PlanRecreatorVarKey, e.info)
e.ctx.SetValue(PlanReplayerVarKey, e.info)
return nil
}

// Close implements the Executor Close interface.
func (e *PlanRecreatorSingleExec) Close() error {
func (e *PlanReplayerSingleExec) Close() error {
return nil
}

// Open implements the Executor Open interface.
func (e *PlanRecreatorSingleExec) Open(ctx context.Context) error {
func (e *PlanReplayerSingleExec) Open(ctx context.Context) error {
return nil
}

// Process does the export/import work for reproducing sql queries.
func (e *PlanRecreatorSingleInfo) Process() (string, error) {
// TODO: plan recreator load will be developed later
// Process dose the export/import work for reproducing sql queries.
func (e *PlanReplayerSingleInfo) Process() (string, error) {
// TODO: plan replayer load will be developed later
if e.Load {
return "", nil
}
return e.dumpSingle()
}

func (e *PlanRecreatorSingleInfo) dumpSingle() (string, error) {
func (e *PlanReplayerSingleInfo) dumpSingle() (string, error) {
// Create path
err := os.MkdirAll(recreatorPath, os.ModePerm)
err := os.MkdirAll(replayerPath, os.ModePerm)
if err != nil {
return "", errors.New("plan recreator: cannot create plan recreator path")
return "", errors.New("plan replayer: cannot create plan replayer path")
}

// Create zip file
startTime := time.Now()
fileName := fmt.Sprintf("recreator_single_%v.zip", startTime.UnixNano())
zf, err := os.Create(recreatorPath + "/" + fileName)
fileName := fmt.Sprintf("replayer_single_%v.zip", startTime.UnixNano())
zf, err := os.Create(replayerPath + "/" + fileName)
if err != nil {
return "", errors.New("plan recreator: cannot create zip file")
return "", errors.New("plan replayer: cannot create zip file")
}
val := e.Ctx.Value(PlanRecreatorFileList)
val := e.Ctx.Value(PlanReplayerFileList)
if val == nil {
e.Ctx.SetValue(PlanRecreatorFileList, fileList{FileInfo: make(map[string]fileInfo), TokenMap: make(map[[16]byte]string)})
e.Ctx.SetValue(PlanReplayerFileList, fileList{FileInfo: make(map[string]fileInfo), TokenMap: make(map[[16]byte]string)})
} else {
// Clean outdated files
Flist := val.(fileList).FileInfo
TList := val.(fileList).TokenMap
for k, v := range Flist {
if time.Since(v.StartTime).Minutes() > remainedInterval {
err := os.Remove(recreatorPath + "/" + k)
err := os.Remove(replayerPath + "/" + k)
if err != nil {
logutil.BgLogger().Warn(fmt.Sprintf("Cleaning outdated file %s failed.", k))
}
Expand All @@ -157,8 +157,8 @@ func (e *PlanRecreatorSingleInfo) dumpSingle() (string, error) {
}
// Generate Token
token := md5.Sum([]byte(fmt.Sprintf("%s%d", fileName, rand.Int63()))) // #nosec G401 G404
e.Ctx.Value(PlanRecreatorFileList).(fileList).FileInfo[fileName] = fileInfo{StartTime: startTime, Token: token}
e.Ctx.Value(PlanRecreatorFileList).(fileList).TokenMap[token] = fileName
e.Ctx.Value(PlanReplayerFileList).(fileList).FileInfo[fileName] = fileInfo{StartTime: startTime, Token: token}
e.Ctx.Value(PlanReplayerFileList).(fileList).TokenMap[token] = fileName

// Create zip writer
zw := zip.NewWriter(zf)
Expand All @@ -173,6 +173,6 @@ func (e *PlanRecreatorSingleInfo) dumpSingle() (string, error) {
}
}()

// TODO: DUMP PLAN RECREATOR FILES IN ZIP WRITER
// TODO: DUMP PLAN REPLAYER FILES IN ZIP WRITER
return hex.EncodeToString(token[:]), nil
}
24 changes: 12 additions & 12 deletions parser/ast/misc.go
Original file line number Diff line number Diff line change
Expand Up @@ -54,7 +54,7 @@ var (
_ StmtNode = &RestartStmt{}
_ StmtNode = &RenameUserStmt{}
_ StmtNode = &HelpStmt{}
_ StmtNode = &PlanRecreatorStmt{}
_ StmtNode = &PlanReplayerStmt{}

_ Node = &PrivElem{}
_ Node = &VariableAssignment{}
Expand Down Expand Up @@ -243,8 +243,8 @@ func (n *ExplainStmt) Accept(v Visitor) (Node, bool) {
return v.Leave(n)
}

// PlanRecreatorStmt is a statement to dump or load information for recreating plans
type PlanRecreatorStmt struct {
// PlanReplayerStmt is a statement to dump or load information for recreating plans
type PlanReplayerStmt struct {
stmtNode

Stmt StmtNode
Expand All @@ -260,13 +260,13 @@ type PlanRecreatorStmt struct {
}

// Restore implements Node interface.
func (n *PlanRecreatorStmt) Restore(ctx *format.RestoreCtx) error {
func (n *PlanReplayerStmt) Restore(ctx *format.RestoreCtx) error {
if n.Load {
ctx.WriteKeyWord("PLAN RECREATOR LOAD ")
ctx.WriteKeyWord("PLAN REPLAYER LOAD ")
ctx.WriteString(n.File)
return nil
}
ctx.WriteKeyWord("PLAN RECREATOR DUMP EXPLAIN ")
ctx.WriteKeyWord("PLAN REPLAYER DUMP EXPLAIN ")
if n.Analyze {
ctx.WriteKeyWord("ANALYZE ")
}
Expand All @@ -275,37 +275,37 @@ func (n *PlanRecreatorStmt) Restore(ctx *format.RestoreCtx) error {
if n.Where != nil {
ctx.WriteKeyWord(" WHERE ")
if err := n.Where.Restore(ctx); err != nil {
return errors.Annotate(err, "An error occurred while restore PlanRecreatorStmt.Where")
return errors.Annotate(err, "An error occurred while restore PlanReplayerStmt.Where")
}
}
if n.OrderBy != nil {
ctx.WriteKeyWord(" ")
if err := n.OrderBy.Restore(ctx); err != nil {
return errors.Annotate(err, "An error occurred while restore PlanRecreatorStmt.OrderBy")
return errors.Annotate(err, "An error occurred while restore PlanReplayerStmt.OrderBy")
}
}
if n.Limit != nil {
ctx.WriteKeyWord(" ")
if err := n.Limit.Restore(ctx); err != nil {
return errors.Annotate(err, "An error occurred while restore PlanRecreatorStmt.Limit")
return errors.Annotate(err, "An error occurred while restore PlanReplayerStmt.Limit")
}
}
return nil
}
if err := n.Stmt.Restore(ctx); err != nil {
return errors.Annotate(err, "An error occurred while restore PlanRecreatorStmt.Stmt")
return errors.Annotate(err, "An error occurred while restore PlanReplayerStmt.Stmt")
}
return nil
}

// Accept implements Node Accept interface.
func (n *PlanRecreatorStmt) Accept(v Visitor) (Node, bool) {
func (n *PlanReplayerStmt) Accept(v Visitor) (Node, bool) {
newNode, skipChildren := v.Enter(n)
if skipChildren {
return v.Leave(newNode)
}

n = newNode.(*PlanRecreatorStmt)
n = newNode.(*PlanReplayerStmt)

if n.Load {
return v.Leave(n)
Expand Down
2 changes: 1 addition & 1 deletion parser/misc.go
Original file line number Diff line number Diff line change
Expand Up @@ -562,7 +562,6 @@ var tokenMap = map[string]int{
"REBUILD": rebuild,
"RECENT": recent,
"RECOVER": recover,
"RECREATOR": recreator,
"RECURSIVE": recursive,
"REDUNDANT": redundant,
"REFERENCES": references,
Expand All @@ -578,6 +577,7 @@ var tokenMap = map[string]int{
"REPEAT": repeat,
"REPEATABLE": repeatable,
"REPLACE": replace,
"REPLAYER": replayer,
"REPLICA": replica,
"REPLICAS": replicas,
"REPLICATION": replication,
Expand Down
Loading

0 comments on commit cc96f59

Please sign in to comment.