Skip to content

Commit

Permalink
*: milisecond test and infoschema fix (#24820)
Browse files Browse the repository at this point in the history
  • Loading branch information
xhebox committed May 24, 2021
1 parent b7c93e8 commit bfdad7c
Show file tree
Hide file tree
Showing 37 changed files with 135 additions and 137 deletions.
10 changes: 4 additions & 6 deletions distsql/request_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -229,10 +229,6 @@ func (builder *RequestBuilder) SetFromSessionVars(sv *variable.SessionVars) *Req
builder.Request.TaskID = sv.StmtCtx.TaskID
builder.Request.Priority = builder.getKVPriority(sv)
builder.Request.ReplicaRead = sv.GetReplicaRead()
// in tests, it may be null
if is, ok := sv.GetInfoSchema().(infoschema.InfoSchema); ok {
builder.Request.SchemaVar = is.SchemaMetaVersion()
}
builder.txnScope = sv.TxnCtx.TxnScope
builder.IsStaleness = sv.TxnCtx.IsStaleness
if builder.IsStaleness && builder.txnScope != kv.GlobalTxnScope {
Expand Down Expand Up @@ -268,11 +264,13 @@ func (builder *RequestBuilder) SetTiDBServerID(serverID uint64) *RequestBuilder

// SetFromInfoSchema sets the following fields from infoSchema:
// "bundles"
func (builder *RequestBuilder) SetFromInfoSchema(is infoschema.InfoSchema) *RequestBuilder {
if is == nil {
func (builder *RequestBuilder) SetFromInfoSchema(pis interface{}) *RequestBuilder {
is, ok := pis.(infoschema.InfoSchema)
if !ok {
return builder
}
builder.is = is
builder.Request.SchemaVar = is.SchemaMetaVersion()
return builder
}

Expand Down
4 changes: 1 addition & 3 deletions distsql/request_builder_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,6 @@ import (

. "github.com/pingcap/check"
"github.com/pingcap/parser/mysql"
"github.com/pingcap/tidb/infoschema"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/sessionctx/stmtctx"
Expand Down Expand Up @@ -612,7 +611,6 @@ func (s *testSuite) TestRequestBuilder7(c *C) {

func (s *testSuite) TestRequestBuilder8(c *C) {
sv := variable.NewSessionVars()
sv.SnapshotInfoschema = infoschema.MockInfoSchemaWithSchemaVer(nil, 10000)
actual, err := (&RequestBuilder{}).
SetFromSessionVars(sv).
Build()
Expand All @@ -625,7 +623,7 @@ func (s *testSuite) TestRequestBuilder8(c *C) {
IsolationLevel: 0,
Priority: 0,
MemTracker: (*memory.Tracker)(nil),
SchemaVar: 10000,
SchemaVar: 0,
}
c.Assert(actual, DeepEquals, expect)
}
Expand Down
2 changes: 1 addition & 1 deletion executor/adapter.go
Original file line number Diff line number Diff line change
Expand Up @@ -268,7 +268,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 := a.Ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema)
is := a.Ctx.GetInfoSchema().(infoschema.InfoSchema)
a.InfoSchema = is
if err := plannercore.Preprocess(a.Ctx, a.StmtNode, is, plannercore.InTxnRetry); err != nil {
return 0, err
Expand Down
3 changes: 3 additions & 0 deletions executor/admin.go
Original file line number Diff line number Diff line change
Expand Up @@ -126,6 +126,7 @@ func (e *CheckIndexRangeExec) Open(ctx context.Context) error {
SetStartTS(txn.StartTS()).
SetKeepOrder(true).
SetFromSessionVars(e.ctx.GetSessionVars()).
SetFromInfoSchema(e.ctx.GetInfoSchema()).
Build()
if err != nil {
return err
Expand Down Expand Up @@ -272,6 +273,7 @@ func (e *RecoverIndexExec) buildTableScan(ctx context.Context, txn kv.Transactio
SetStartTS(txn.StartTS()).
SetKeepOrder(true).
SetFromSessionVars(e.ctx.GetSessionVars()).
SetFromInfoSchema(e.ctx.GetInfoSchema()).
Build()
if err != nil {
return nil, err
Expand Down Expand Up @@ -735,6 +737,7 @@ func (e *CleanupIndexExec) buildIndexScan(ctx context.Context, txn kv.Transactio
SetStartTS(txn.StartTS()).
SetKeepOrder(true).
SetFromSessionVars(e.ctx.GetSessionVars()).
SetFromInfoSchema(e.ctx.GetInfoSchema()).
Build()
if err != nil {
return nil, err
Expand Down
4 changes: 2 additions & 2 deletions executor/analyze.go
Original file line number Diff line number Diff line change
Expand Up @@ -186,7 +186,7 @@ func (e *AnalyzeExec) Next(ctx context.Context, req *chunk.Chunk) error {
}
if needGlobalStats {
for globalStatsID, info := range globalStatsMap {
globalStats, err := statsHandle.MergePartitionStats2GlobalStatsByTableID(e.ctx, e.opts, e.ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema), globalStatsID.tableID, info.isIndex, info.idxID)
globalStats, err := statsHandle.MergePartitionStats2GlobalStatsByTableID(e.ctx, e.opts, e.ctx.GetInfoSchema().(infoschema.InfoSchema), globalStatsID.tableID, info.isIndex, info.idxID)
if err != nil {
if types.ErrPartitionStatsMissing.Equal(err) {
// When we find some partition-level stats are missing, we need to report warning.
Expand All @@ -205,7 +205,7 @@ func (e *AnalyzeExec) Next(ctx context.Context, req *chunk.Chunk) error {
}
}
}
return statsHandle.Update(e.ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema))
return statsHandle.Update(e.ctx.GetInfoSchema().(infoschema.InfoSchema))
}

func getBuildStatsConcurrency(ctx sessionctx.Context) (int, error) {
Expand Down
12 changes: 6 additions & 6 deletions executor/analyze_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -69,7 +69,7 @@ PARTITION BY RANGE ( a ) (
}
tk.MustExec("analyze table t")

is := tk.Se.(sessionctx.Context).GetSessionVars().GetInfoSchema().(infoschema.InfoSchema)
is := tk.Se.(sessionctx.Context).GetInfoSchema().(infoschema.InfoSchema)
table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t"))
c.Assert(err, IsNil)
pi := table.Meta().GetPartitionInfo()
Expand All @@ -96,7 +96,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 = tk.Se.(sessionctx.Context).GetSessionVars().GetInfoSchema().(infoschema.InfoSchema)
is = tk.Se.(sessionctx.Context).GetInfoSchema().(infoschema.InfoSchema)
table, err = is.TableByName(model.NewCIStr("test"), model.NewCIStr("t"))
c.Assert(err, IsNil)
pi = table.Meta().GetPartitionInfo()
Expand Down Expand Up @@ -176,7 +176,7 @@ func (s *testSuite1) TestAnalyzeParameters(c *C) {

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

tk.MustExec("analyze table t")
is := tk.Se.(sessionctx.Context).GetSessionVars().GetInfoSchema().(infoschema.InfoSchema)
is := tk.Se.(sessionctx.Context).GetInfoSchema().(infoschema.InfoSchema)
table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t"))
c.Assert(err, IsNil)
tableInfo := table.Meta()
Expand Down Expand Up @@ -259,7 +259,7 @@ func (s *testSuite1) TestAnalyzeIndexExtractTopN(c *C) {
tk.MustExec("set @@session.tidb_analyze_version=2")
tk.MustExec("analyze table t with 10 cmsketch width")

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

is := tk.Se.(sessionctx.Context).GetSessionVars().GetInfoSchema().(infoschema.InfoSchema)
is := tk.Se.(sessionctx.Context).GetInfoSchema().(infoschema.InfoSchema)
table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t"))
c.Assert(err, IsNil)
tableInfo := table.Meta()
Expand Down
2 changes: 1 addition & 1 deletion executor/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -3462,7 +3462,7 @@ func (builder *dataReaderBuilder) buildTableReaderBase(ctx context.Context, e *T
SetKeepOrder(e.keepOrder).
SetStreaming(e.streaming).
SetFromSessionVars(e.ctx.GetSessionVars()).
SetFromInfoSchema(e.ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema)).
SetFromInfoSchema(e.ctx.GetInfoSchema()).
Build()
if err != nil {
return nil, err
Expand Down
2 changes: 1 addition & 1 deletion executor/compiler.go
Original file line number Diff line number Diff line change
Expand Up @@ -53,7 +53,7 @@ func (c *Compiler) Compile(ctx context.Context, stmtNode ast.StmtNode) (*ExecStm
ctx = opentracing.ContextWithSpan(ctx, span1)
}

infoSchema := c.Ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema)
infoSchema := c.Ctx.GetInfoSchema().(infoschema.InfoSchema)
if err := plannercore.Preprocess(c.Ctx, stmtNode, infoSchema); err != nil {
return nil, err
}
Expand Down
2 changes: 1 addition & 1 deletion executor/coprocessor.go
Original file line number Diff line number Diff line change
Expand Up @@ -159,7 +159,7 @@ func (h *CoprocessorDAGHandler) buildDAGExecutor(req *coprocessor.Request) (Exec
return nil, errors.Trace(err)
}
h.dagReq = dagReq
is := h.sctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema)
is := h.sctx.GetInfoSchema().(infoschema.InfoSchema)
// Build physical plan.
bp := core.NewPBPlanBuilder(h.sctx, is, req.Ranges)
plan, err := bp.Build(dagReq.Executors)
Expand Down
11 changes: 2 additions & 9 deletions executor/distsql.go
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,6 @@ import (
"github.com/pingcap/parser/terror"
"github.com/pingcap/tidb/distsql"
"github.com/pingcap/tidb/expression"
"github.com/pingcap/tidb/infoschema"
"github.com/pingcap/tidb/kv"
plannercore "github.com/pingcap/tidb/planner/core"
"github.com/pingcap/tidb/sessionctx"
Expand Down Expand Up @@ -278,11 +277,8 @@ func (e *IndexReaderExecutor) open(ctx context.Context, kvRanges []kv.KeyRange)
SetKeepOrder(e.keepOrder).
SetStreaming(e.streaming).
SetFromSessionVars(e.ctx.GetSessionVars()).
SetFromInfoSchema(e.ctx.GetInfoSchema()).
SetMemTracker(e.memTracker)
// for tests, infoschema may be null
if is, ok := e.ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema); ok {
builder.SetFromInfoSchema(is)
}
kvReq, err := builder.Build()
if err != nil {
e.feedback.Invalidate()
Expand Down Expand Up @@ -530,11 +526,8 @@ func (e *IndexLookUpExecutor) startIndexWorker(ctx context.Context, workCh chan<
SetKeepOrder(e.keepOrder).
SetStreaming(e.indexStreaming).
SetFromSessionVars(e.ctx.GetSessionVars()).
SetFromInfoSchema(e.ctx.GetInfoSchema()).
SetMemTracker(tracker)
// for tests, infoschema may be null
if is, ok := e.ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema); ok {
builder.SetFromInfoSchema(is)
}

for partTblIdx, kvRange := range kvRanges {
// check if executor is closed
Expand Down
4 changes: 2 additions & 2 deletions executor/executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2346,7 +2346,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 := ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema)
infoSchema := ctx.GetInfoSchema().(infoschema.InfoSchema)

for sqlStr, result := range tests {
stmtNode, err := s.ParseOneStmt(sqlStr, "", "")
Expand Down Expand Up @@ -2378,7 +2378,7 @@ func (s *testSuiteP2) TestClusteredIndexIsPointGet(c *C) {
"select * from t where a='x' and c='x'": true,
"select * from t where a='x' and c='x' and b=1": false,
}
infoSchema := ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema)
infoSchema := ctx.GetInfoSchema().(infoschema.InfoSchema)
for sqlStr, result := range tests {
stmtNode, err := s.ParseOneStmt(sqlStr, "", "")
c.Check(err, IsNil)
Expand Down
2 changes: 1 addition & 1 deletion executor/grant.go
Original file line number Diff line number Diff line change
Expand Up @@ -74,7 +74,7 @@ func (e *GrantExec) Next(ctx context.Context, req *chunk.Chunk) error {
// Make sure the table exist.
if e.Level.Level == ast.GrantLevelTable {
dbNameStr := model.NewCIStr(dbName)
schema := e.ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema)
schema := e.ctx.GetInfoSchema().(infoschema.InfoSchema)
tbl, err := schema.TableByName(dbNameStr, model.NewCIStr(e.Level.TableName))
if err != nil {
return err
Expand Down
3 changes: 1 addition & 2 deletions executor/index_merge_reader.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,6 @@ import (
"github.com/pingcap/parser/model"
"github.com/pingcap/parser/terror"
"github.com/pingcap/tidb/distsql"
"github.com/pingcap/tidb/infoschema"
"github.com/pingcap/tidb/kv"
plannercore "github.com/pingcap/tidb/planner/core"
"github.com/pingcap/tidb/sessionctx"
Expand Down Expand Up @@ -245,7 +244,7 @@ func (e *IndexMergeReaderExecutor) startPartialIndexWorker(ctx context.Context,
SetStreaming(e.partialStreamings[workID]).
SetFromSessionVars(e.ctx.GetSessionVars()).
SetMemTracker(e.memTracker).
SetFromInfoSchema(e.ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema))
SetFromInfoSchema(e.ctx.GetInfoSchema())

worker := &partialIndexWorker{
stats: e.stats,
Expand Down
16 changes: 8 additions & 8 deletions executor/infoschema_reader.go
Original file line number Diff line number Diff line change
Expand Up @@ -80,7 +80,7 @@ func (e *memtableRetriever) retrieve(ctx context.Context, sctx sessionctx.Contex

// Cache the ret full rows in schemataRetriever
if !e.initialized {
is := sctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema)
is := sctx.GetInfoSchema().(infoschema.InfoSchema)
dbs := is.AllSchemas()
sort.Sort(infoschema.SchemasSorter(dbs))
var err error
Expand Down Expand Up @@ -304,7 +304,7 @@ func (c *statsCache) get(ctx sessionctx.Context) (map[int64]uint64, map[tableHis
}

func getAutoIncrementID(ctx sessionctx.Context, schema *model.DBInfo, tblInfo *model.TableInfo) (int64, error) {
is := ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema)
is := ctx.GetInfoSchema().(infoschema.InfoSchema)
tbl, err := is.TableByName(schema.Name, tblInfo.Name)
if err != nil {
return 0, err
Expand Down Expand Up @@ -592,7 +592,7 @@ func (e *hugeMemTableRetriever) setDataForColumns(ctx context.Context, sctx sess
}

func (e *hugeMemTableRetriever) dataForColumnsInTable(ctx context.Context, sctx sessionctx.Context, schema *model.DBInfo, tbl *model.TableInfo) {
if err := tryFillViewColumnType(ctx, sctx, sctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema), schema.Name, tbl); err != nil {
if err := tryFillViewColumnType(ctx, sctx, sctx.GetInfoSchema().(infoschema.InfoSchema), schema.Name, tbl); err != nil {
sctx.GetSessionVars().StmtCtx.AppendWarning(err)
return
}
Expand Down Expand Up @@ -1339,7 +1339,7 @@ func (e *memtableRetriever) setDataForTiKVRegionStatus(ctx sessionctx.Context) e
if err != nil {
return err
}
allSchemas := ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema).AllSchemas()
allSchemas := ctx.GetInfoSchema().(infoschema.InfoSchema).AllSchemas()
tableInfos := tikvHelper.GetRegionsTableInfo(regionsInfo, allSchemas)
for _, region := range regionsInfo.Regions {
tableList := tableInfos[region.ID]
Expand Down Expand Up @@ -1451,7 +1451,7 @@ func (e *memtableRetriever) setDataForTiDBHotRegions(ctx sessionctx.Context) err
if !ok {
return errors.New("Information about hot region can be gotten only when the storage is TiKV")
}
allSchemas := ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema).AllSchemas()
allSchemas := ctx.GetInfoSchema().(infoschema.InfoSchema).AllSchemas()
tikvHelper := &helper.Helper{
Store: tikvStore,
RegionCache: tikvStore.GetRegionCache(),
Expand Down Expand Up @@ -1600,7 +1600,7 @@ type initialTable struct {
}

func (e *tableStorageStatsRetriever) initialize(sctx sessionctx.Context) error {
is := sctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema)
is := sctx.GetInfoSchema().(infoschema.InfoSchema)
var databases []string
schemas := e.extractor.TableSchema
tables := e.extractor.TableName
Expand Down Expand Up @@ -1891,7 +1891,7 @@ func (e *memtableRetriever) setDataForStatementsSummary(ctx sessionctx.Context,

func (e *memtableRetriever) setDataForPlacementPolicy(ctx sessionctx.Context) error {
checker := privilege.GetPrivilegeManager(ctx)
is := ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema)
is := ctx.GetInfoSchema().(infoschema.InfoSchema)
var rows [][]types.Datum
for _, bundle := range is.RuleBundles() {
id, err := placement.ObjectIDFromGroupID(bundle.ID)
Expand Down Expand Up @@ -2099,7 +2099,7 @@ func (e *hugeMemTableRetriever) retrieve(ctx context.Context, sctx sessionctx.Co
}

if !e.initialized {
is := sctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema)
is := sctx.GetInfoSchema().(infoschema.InfoSchema)
dbs := is.AllSchemas()
sort.Sort(infoschema.SchemasSorter(dbs))
e.dbs = dbs
Expand Down
2 changes: 1 addition & 1 deletion executor/load_stats.go
Original file line number Diff line number Diff line change
Expand Up @@ -86,5 +86,5 @@ func (e *LoadStatsInfo) Update(data []byte) error {
if h == nil {
return errors.New("Load Stats: handle is nil")
}
return h.LoadStatsFromJSON(e.Ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema), jsonTbl)
return h.LoadStatsFromJSON(e.Ctx.GetInfoSchema().(infoschema.InfoSchema), jsonTbl)
}
2 changes: 1 addition & 1 deletion executor/metrics_reader_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -62,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 := tk.Se.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema)
is := tk.Se.GetInfoSchema().(infoschema.InfoSchema)
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/partition_table_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -284,7 +284,7 @@ func (s *partitionTableSuite) TestPartitionInfoDisable(c *C) {
PARTITION p202010 VALUES LESS THAN ("2020-11-01"),
PARTITION p202011 VALUES LESS THAN ("2020-12-01")
)`)
is := tk.Se.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema)
is := tk.Se.GetInfoSchema().(infoschema.InfoSchema)
tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t_info_null"))
c.Assert(err, IsNil)

Expand Down
2 changes: 1 addition & 1 deletion executor/point_get.go
Original file line number Diff line number Diff line change
Expand Up @@ -391,7 +391,7 @@ func (e *PointGetExecutor) verifyTxnScope() error {
var tblID int64
var tblName string
var partName string
is := e.ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema)
is := e.ctx.GetInfoSchema().(infoschema.InfoSchema)
if e.partInfo != nil {
tblID = e.partInfo.ID
tblInfo, _, partInfo := is.FindTableByPartitionID(tblID)
Expand Down
2 changes: 1 addition & 1 deletion executor/prepared.go
Original file line number Diff line number Diff line change
Expand Up @@ -320,7 +320,7 @@ func CompileExecutePreparedStmt(ctx context.Context, sctx sessionctx.Context,
return nil, false, false, err
}
execStmt.BinaryArgs = args
is := sctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema)
is := sctx.GetInfoSchema().(infoschema.InfoSchema)
execPlan, names, err := planner.Optimize(ctx, sctx, execStmt, is)
if err != nil {
return nil, false, false, err
Expand Down
3 changes: 2 additions & 1 deletion executor/simple.go
Original file line number Diff line number Diff line change
Expand Up @@ -1456,6 +1456,7 @@ func killRemoteConn(ctx context.Context, sctx sessionctx.Context, connID *util.G
kvReq, err := builder.
SetDAGRequest(dagReq).
SetFromSessionVars(sctx.GetSessionVars()).
SetFromInfoSchema(sctx.GetInfoSchema()).
SetStoreType(kv.TiDB).
SetTiDBServerID(connID.ServerID).
Build()
Expand Down Expand Up @@ -1544,7 +1545,7 @@ func (e *SimpleExec) executeDropStats(s *ast.DropStatsStmt) (err error) {
if err := h.DeleteTableStatsFromKV(statsIDs); err != nil {
return err
}
return h.Update(e.ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema))
return h.Update(e.ctx.GetInfoSchema().(infoschema.InfoSchema))
}

func (e *SimpleExec) autoNewTxn() bool {
Expand Down
Loading

0 comments on commit bfdad7c

Please sign in to comment.