Skip to content

Commit

Permalink
resolved conflict and add more test cases
Browse files Browse the repository at this point in the history
  • Loading branch information
hsqlu committed Dec 6, 2019
2 parents c814120 + b1391ec commit 224f39a
Show file tree
Hide file tree
Showing 76 changed files with 1,550 additions and 762 deletions.
4 changes: 2 additions & 2 deletions cmd/explaintest/r/explain_easy.result
Original file line number Diff line number Diff line change
Expand Up @@ -79,9 +79,9 @@ TopN_7 1.00 root test.t2.c2:asc, offset:0, count:1
└─TableScan_13 10000.00 cop[tikv] table:t2, range:[-inf,+inf], keep order:false, stats:pseudo
explain select * from t1 where c1 > 1 and c2 = 1 and c3 < 1;
id count task operator info
IndexLookUp_11 1.11 root
IndexLookUp_11 11.08 root
├─IndexScan_8 33.33 cop[tikv] table:t1, index:c2, range:(1 1,1 +inf], keep order:false, stats:pseudo
└─Selection_10 1.11 cop[tikv] lt(test.t1.c3, 1)
└─Selection_10 11.08 cop[tikv] lt(test.t1.c3, 1)
└─TableScan_9 33.33 cop[tikv] table:t1, keep order:false, stats:pseudo
explain select * from t1 where c1 = 1 and c2 > 1;
id count task operator info
Expand Down
1 change: 1 addition & 0 deletions ddl/db_integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1043,6 +1043,7 @@ func (s *testIntegrationSuite5) TestBackwardCompatibility(c *C) {
err = txn.Commit(context.Background())
c.Assert(err, IsNil)
ticker := time.NewTicker(s.lease)
defer ticker.Stop()
for range ticker.C {
historyJob, err := s.getHistoryDDLJob(job.ID)
c.Assert(err, IsNil)
Expand Down
1 change: 1 addition & 0 deletions domain/schema_validator_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -128,6 +128,7 @@ func serverFunc(lease time.Duration, requireLease chan leaseGrantItem, oracleCh
var version int64
leaseTS := uint64(time.Now().UnixNano())
ticker := time.NewTicker(lease)
defer ticker.Stop()
for {
select {
case now := <-ticker.C:
Expand Down
2 changes: 1 addition & 1 deletion executor/adapter.go
Original file line number Diff line number Diff line change
Expand Up @@ -254,7 +254,7 @@ func (a *ExecStmt) IsReadOnly(vars *variable.SessionVars) bool {
// RebuildPlan rebuilds current execute statement plan.
// It returns the current information schema version that 'a' is using.
func (a *ExecStmt) RebuildPlan(ctx context.Context) (int64, error) {
is := GetInfoSchema(a.Ctx)
is := infoschema.GetInfoSchema(a.Ctx)
a.InfoSchema = is
if err := plannercore.Preprocess(a.Ctx, a.StmtNode, is, plannercore.InTxnRetry); err != nil {
return 0, err
Expand Down
3 changes: 2 additions & 1 deletion executor/analyze.go
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@ import (
"github.com/pingcap/parser/mysql"
"github.com/pingcap/tidb/distsql"
"github.com/pingcap/tidb/domain"
"github.com/pingcap/tidb/infoschema"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/metrics"
"github.com/pingcap/tidb/sessionctx"
Expand Down Expand Up @@ -125,7 +126,7 @@ func (e *AnalyzeExec) Next(ctx context.Context, req *chunk.Chunk) error {
if err != nil {
return err
}
return statsHandle.Update(GetInfoSchema(e.ctx))
return statsHandle.Update(infoschema.GetInfoSchema(e.ctx))
}

func getBuildStatsConcurrency(ctx sessionctx.Context) (int, error) {
Expand Down
11 changes: 6 additions & 5 deletions executor/analyze_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@ import (
"github.com/pingcap/parser/mysql"
"github.com/pingcap/tidb/domain"
"github.com/pingcap/tidb/executor"
"github.com/pingcap/tidb/infoschema"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/session"
"github.com/pingcap/tidb/sessionctx"
Expand Down Expand Up @@ -62,7 +63,7 @@ PARTITION BY RANGE ( a ) (
}
tk.MustExec("analyze table t")

is := executor.GetInfoSchema(tk.Se.(sessionctx.Context))
is := infoschema.GetInfoSchema(tk.Se.(sessionctx.Context))
table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t"))
c.Assert(err, IsNil)
pi := table.Meta().GetPartitionInfo()
Expand All @@ -89,7 +90,7 @@ PARTITION BY RANGE ( a ) (
tk.MustExec(fmt.Sprintf(`insert into t values (%d, %d, "hello")`, i, i))
}
tk.MustExec("alter table t analyze partition p0")
is = executor.GetInfoSchema(tk.Se.(sessionctx.Context))
is = infoschema.GetInfoSchema(tk.Se.(sessionctx.Context))
table, err = is.TableByName(model.NewCIStr("test"), model.NewCIStr("t"))
c.Assert(err, IsNil)
pi = table.Meta().GetPartitionInfo()
Expand Down Expand Up @@ -139,7 +140,7 @@ func (s *testSuite1) TestAnalyzeParameters(c *C) {

tk.MustExec("set @@tidb_enable_fast_analyze = 1")
tk.MustExec("analyze table t with 30 samples")
is := executor.GetInfoSchema(tk.Se.(sessionctx.Context))
is := infoschema.GetInfoSchema(tk.Se.(sessionctx.Context))
table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t"))
c.Assert(err, IsNil)
tableInfo := table.Meta()
Expand Down Expand Up @@ -170,7 +171,7 @@ func (s *testSuite1) TestAnalyzeTooLongColumns(c *C) {
tk.MustExec(fmt.Sprintf("insert into t values ('%s')", value))

tk.MustExec("analyze table t")
is := executor.GetInfoSchema(tk.Se.(sessionctx.Context))
is := infoschema.GetInfoSchema(tk.Se.(sessionctx.Context))
table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t"))
c.Assert(err, IsNil)
tableInfo := table.Meta()
Expand Down Expand Up @@ -311,7 +312,7 @@ func (s *testFastAnalyze) TestFastAnalyze(c *C) {
}
tk.MustExec("analyze table t with 5 buckets, 6 samples")

is := executor.GetInfoSchema(tk.Se.(sessionctx.Context))
is := infoschema.GetInfoSchema(tk.Se.(sessionctx.Context))
table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t"))
c.Assert(err, IsNil)
tableInfo := table.Meta()
Expand Down
9 changes: 5 additions & 4 deletions executor/bind.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ import (
"github.com/pingcap/parser/ast"
"github.com/pingcap/tidb/bindinfo"
"github.com/pingcap/tidb/domain"
"github.com/pingcap/tidb/infoschema"
plannercore "github.com/pingcap/tidb/planner/core"
"github.com/pingcap/tidb/util/chunk"
)
Expand Down Expand Up @@ -72,9 +73,9 @@ func (e *SQLBindExec) dropSQLBind() error {
}
if !e.isGlobal {
handle := e.ctx.Value(bindinfo.SessionBindInfoKeyType).(*bindinfo.SessionHandle)
return handle.DropBindRecord(e.ctx, GetInfoSchema(e.ctx), record)
return handle.DropBindRecord(e.ctx, infoschema.GetInfoSchema(e.ctx), record)
}
return domain.GetDomain(e.ctx).BindHandle().DropBindRecord(e.ctx, GetInfoSchema(e.ctx), record)
return domain.GetDomain(e.ctx).BindHandle().DropBindRecord(e.ctx, infoschema.GetInfoSchema(e.ctx), record)
}

func (e *SQLBindExec) createSQLBind() error {
Expand All @@ -91,9 +92,9 @@ func (e *SQLBindExec) createSQLBind() error {
}
if !e.isGlobal {
handle := e.ctx.Value(bindinfo.SessionBindInfoKeyType).(*bindinfo.SessionHandle)
return handle.AddBindRecord(e.ctx, GetInfoSchema(e.ctx), record)
return handle.AddBindRecord(e.ctx, infoschema.GetInfoSchema(e.ctx), record)
}
return domain.GetDomain(e.ctx).BindHandle().AddBindRecord(e.ctx, GetInfoSchema(e.ctx), record)
return domain.GetDomain(e.ctx).BindHandle().AddBindRecord(e.ctx, infoschema.GetInfoSchema(e.ctx), record)
}

func (e *SQLBindExec) flushBindings() error {
Expand Down
18 changes: 1 addition & 17 deletions executor/compiler.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,8 +25,6 @@ import (
"github.com/pingcap/tidb/planner"
plannercore "github.com/pingcap/tidb/planner/core"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/util/logutil"
"go.uber.org/zap"
)

var (
Expand Down Expand Up @@ -55,7 +53,7 @@ func (c *Compiler) Compile(ctx context.Context, stmtNode ast.StmtNode) (*ExecStm
ctx = opentracing.ContextWithSpan(ctx, span1)
}

infoSchema := GetInfoSchema(c.Ctx)
infoSchema := infoschema.GetInfoSchema(c.Ctx)
if err := plannercore.Preprocess(c.Ctx, stmtNode, infoSchema); err != nil {
return nil, err
}
Expand Down Expand Up @@ -347,17 +345,3 @@ func GetStmtLabel(stmtNode ast.StmtNode) string {
}
return "other"
}

// GetInfoSchema gets TxnCtx InfoSchema if snapshot schema is not set,
// Otherwise, snapshot schema is returned.
func GetInfoSchema(ctx sessionctx.Context) infoschema.InfoSchema {
sessVar := ctx.GetSessionVars()
var is infoschema.InfoSchema
if snap := sessVar.SnapshotInfoschema; snap != nil {
is = snap.(infoschema.InfoSchema)
logutil.BgLogger().Info("use snapshot schema", zap.Uint64("conn", sessVar.ConnectionID), zap.Int64("schemaVersion", is.SchemaMetaVersion()))
} else {
is = sessVar.TxnCtx.InfoSchema.(infoschema.InfoSchema)
}
return is
}
10 changes: 2 additions & 8 deletions executor/executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -1618,14 +1618,8 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) {
sc.PrevAffectedRows = -1
}
errCount, warnCount := vars.StmtCtx.NumErrorWarnings()
err = vars.SetSystemVar("warning_count", warnCount)
if err != nil {
return err
}
err = vars.SetSystemVar("error_count", errCount)
if err != nil {
return err
}
vars.SysErrorCount = errCount
vars.SysWarningCount = warnCount
vars.StmtCtx = sc
for _, warn := range hintWarns {
vars.StmtCtx.AppendWarning(warn)
Expand Down
2 changes: 1 addition & 1 deletion executor/executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2089,7 +2089,7 @@ func (s *testSuiteP2) TestIsPointGet(c *C) {
"select * from help_topic where help_topic_id=1": true,
"select * from help_topic where help_category_id=1": false,
}
infoSchema := executor.GetInfoSchema(ctx)
infoSchema := infoschema.GetInfoSchema(ctx)

for sqlStr, result := range tests {
stmtNode, err := s.ParseOneStmt(sqlStr, "", "")
Expand Down
3 changes: 2 additions & 1 deletion executor/load_stats.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ import (

"github.com/pingcap/errors"
"github.com/pingcap/tidb/domain"
"github.com/pingcap/tidb/infoschema"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/statistics/handle"
"github.com/pingcap/tidb/util/chunk"
Expand Down Expand Up @@ -85,5 +86,5 @@ func (e *LoadStatsInfo) Update(data []byte) error {
if h == nil {
return errors.New("Load Stats: handle is nil")
}
return h.LoadStatsFromJSON(GetInfoSchema(e.Ctx), jsonTbl)
return h.LoadStatsFromJSON(infoschema.GetInfoSchema(e.Ctx), jsonTbl)
}
3 changes: 2 additions & 1 deletion executor/metrics_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ import (
. "github.com/pingcap/check"
"github.com/pingcap/parser"
"github.com/pingcap/tidb/executor"
"github.com/pingcap/tidb/infoschema"
"github.com/pingcap/tidb/planner"
plannercore "github.com/pingcap/tidb/planner/core"
"github.com/pingcap/tidb/sessionctx"
Expand Down Expand Up @@ -61,7 +62,7 @@ func (s *testSuite7) TestStmtLabel(c *C) {
for _, tt := range tests {
stmtNode, err := parser.New().ParseOneStmt(tt.sql, "", "")
c.Check(err, IsNil)
is := executor.GetInfoSchema(tk.Se)
is := infoschema.GetInfoSchema(tk.Se)
err = plannercore.Preprocess(tk.Se.(sessionctx.Context), stmtNode, is)
c.Assert(err, IsNil)
_, _, err = planner.Optimize(context.TODO(), tk.Se, stmtNode, is)
Expand Down
2 changes: 1 addition & 1 deletion executor/prepared.go
Original file line number Diff line number Diff line change
Expand Up @@ -297,7 +297,7 @@ func CompileExecutePreparedStmt(ctx context.Context, sctx sessionctx.Context,
return nil, err
}
execStmt.BinaryArgs = args
is := GetInfoSchema(sctx)
is := infoschema.GetInfoSchema(sctx)
execPlan, names, err := planner.Optimize(ctx, sctx, execStmt, is)
if err != nil {
return nil, err
Expand Down
Loading

0 comments on commit 224f39a

Please sign in to comment.