Skip to content

Commit

Permalink
table: introduce MutateContext and AllocatorContext for `table.Ta…
Browse files Browse the repository at this point in the history
…ble` (#50862)

close #50861
  • Loading branch information
lcwangchao authored Feb 1, 2024
1 parent cac449b commit 6e022a5
Show file tree
Hide file tree
Showing 33 changed files with 234 additions and 194 deletions.
1 change: 0 additions & 1 deletion br/pkg/lightning/backend/kv/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -70,7 +70,6 @@ go_test(
"//pkg/parser/model",
"//pkg/parser/mysql",
"//pkg/planner/core",
"//pkg/sessionctx",
"//pkg/sessionctx/variable",
"//pkg/table",
"//pkg/table/tables",
Expand Down
4 changes: 2 additions & 2 deletions br/pkg/lightning/backend/kv/base.go
Original file line number Diff line number Diff line change
Expand Up @@ -218,14 +218,14 @@ func (e *BaseKVEncoder) ProcessColDatum(col *table.Column, rowID int64, inputDat
meta := e.Table.Meta()
shardFmt := autoid.NewShardIDFormat(&col.FieldType, meta.AutoRandomBits, meta.AutoRandomRangeBits)
// this allocator is the same as the allocator in table importer, i.e. PanickingAllocators. below too.
alloc := e.Table.Allocators(e.SessionCtx).Get(autoid.AutoRandomType)
alloc := e.Table.Allocators(e.SessionCtx.GetSessionVars()).Get(autoid.AutoRandomType)
if err := alloc.Rebase(context.Background(), value.GetInt64()&shardFmt.IncrementalMask(), false); err != nil {
return value, errors.Trace(err)
}
}
if IsAutoIncCol(col.ToInfo()) {
// same as RowIDAllocType, since SepAutoInc is always false when initializing allocators of Table.
alloc := e.Table.Allocators(e.SessionCtx).Get(autoid.AutoIncrementType)
alloc := e.Table.Allocators(e.SessionCtx.GetSessionVars()).Get(autoid.AutoIncrementType)
if err := alloc.Rebase(context.Background(), GetAutoRecordID(value, &col.FieldType), false); err != nil {
return value, errors.Trace(err)
}
Expand Down
2 changes: 1 addition & 1 deletion br/pkg/lightning/backend/kv/sql2kv.go
Original file line number Diff line number Diff line change
Expand Up @@ -227,7 +227,7 @@ func (kvcodec *tableKVEncoder) Encode(row []types.Datum,
return nil, kvcodec.LogKVConvertFailed(row, j, ExtraHandleColumnInfo, err)
}
record = append(record, value)
alloc := kvcodec.Table.Allocators(kvcodec.SessionCtx).Get(autoid.RowIDAllocType)
alloc := kvcodec.Table.Allocators(kvcodec.SessionCtx.GetSessionVars()).Get(autoid.RowIDAllocType)
if err := alloc.Rebase(context.Background(), rowValue, false); err != nil {
return nil, errors.Trace(err)
}
Expand Down
15 changes: 7 additions & 8 deletions br/pkg/lightning/backend/kv/sql2kv_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,6 @@ import (
"github.com/pingcap/tidb/pkg/parser/model"
"github.com/pingcap/tidb/pkg/parser/mysql"
_ "github.com/pingcap/tidb/pkg/planner/core" // to setup expression.EvalAstExpr. Otherwise we cannot parse the default value
"github.com/pingcap/tidb/pkg/sessionctx"
"github.com/pingcap/tidb/pkg/table"
"github.com/pingcap/tidb/pkg/table/tables"
"github.com/pingcap/tidb/pkg/tablecodec"
Expand Down Expand Up @@ -72,7 +71,7 @@ type mockTable struct {
table.Table
}

func (mockTable) AddRecord(ctx sessionctx.Context, r []types.Datum, opts ...table.AddRecordOption) (recordID kv.Handle, err error) {
func (mockTable) AddRecord(ctx table.MutateContext, r []types.Datum, opts ...table.AddRecordOption) (recordID kv.Handle, err error) {
return kv.IntHandle(-1), errors.New("mock error")
}

Expand Down Expand Up @@ -385,7 +384,7 @@ func TestEncodeDoubleAutoIncrement(t *testing.T) {
require.NoError(t, err)

require.Equal(t, pairsExpect, pairs)
require.Equal(t, tbl.Allocators(lkv.GetEncoderSe(encoder)).Get(autoid.AutoIncrementType).Base(), int64(70))
require.Equal(t, tbl.Allocators(lkv.GetEncoderSe(encoder).GetSessionVars()).Get(autoid.AutoIncrementType).Base(), int64(70))
}

func TestEncodeMissingAutoValue(t *testing.T) {
Expand Down Expand Up @@ -446,13 +445,13 @@ func TestEncodeMissingAutoValue(t *testing.T) {
}, rowID, []int{0}, 1234)
require.NoError(t, err)
require.Equalf(t, pairsExpect, pairs, "test table info: %+v", testTblInfo)
require.Equalf(t, rowID, tbl.Allocators(lkv.GetEncoderSe(encoder)).Get(testTblInfo.AllocType).Base(), "test table info: %+v", testTblInfo)
require.Equalf(t, rowID, tbl.Allocators(lkv.GetEncoderSe(encoder).GetSessionVars()).Get(testTblInfo.AllocType).Base(), "test table info: %+v", testTblInfo)

// test insert a row without specifying the auto_xxxx column
pairs, err = encoder.Encode([]types.Datum{}, rowID, []int{0}, 1234)
require.NoError(t, err)
require.Equalf(t, pairsExpect, pairs, "test table info: %+v", testTblInfo)
require.Equalf(t, rowID, tbl.Allocators(lkv.GetEncoderSe(encoder)).Get(testTblInfo.AllocType).Base(), "test table info: %+v", testTblInfo)
require.Equalf(t, rowID, tbl.Allocators(lkv.GetEncoderSe(encoder).GetSessionVars()).Get(testTblInfo.AllocType).Base(), "test table info: %+v", testTblInfo)
}
}

Expand Down Expand Up @@ -525,7 +524,7 @@ func TestDefaultAutoRandoms(t *testing.T) {
RowID: common.EncodeIntRowID(70),
},
}))
require.Equal(t, tbl.Allocators(lkv.GetSession4test(encoder)).Get(autoid.AutoRandomType).Base(), int64(70))
require.Equal(t, tbl.Allocators(lkv.GetSession4test(encoder).GetSessionVars()).Get(autoid.AutoRandomType).Base(), int64(70))

pairs, err = encoder.Encode([]types.Datum{types.NewStringDatum("")}, 71, []int{-1, 0}, 1234)
require.NoError(t, err)
Expand All @@ -536,7 +535,7 @@ func TestDefaultAutoRandoms(t *testing.T) {
RowID: common.EncodeIntRowID(71),
},
}))
require.Equal(t, tbl.Allocators(lkv.GetSession4test(encoder)).Get(autoid.AutoRandomType).Base(), int64(71))
require.Equal(t, tbl.Allocators(lkv.GetSession4test(encoder).GetSessionVars()).Get(autoid.AutoRandomType).Base(), int64(71))
}

func TestShardRowId(t *testing.T) {
Expand Down Expand Up @@ -567,7 +566,7 @@ func TestShardRowId(t *testing.T) {
keyMap[rowID>>60] = struct{}{}
}
require.Len(t, keyMap, 8)
require.Equal(t, tbl.Allocators(lkv.GetSession4test(encoder)).Get(autoid.RowIDAllocType).Base(), int64(32))
require.Equal(t, tbl.Allocators(lkv.GetSession4test(encoder).GetSessionVars()).Get(autoid.RowIDAllocType).Base(), int64(32))
}

func TestSplitIntoChunks(t *testing.T) {
Expand Down
4 changes: 2 additions & 2 deletions pkg/ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -3994,7 +3994,7 @@ func (d *ddl) RebaseAutoID(ctx sessionctx.Context, ident ast.Ident, newBase int6
}

if !force {
newBaseTemp, err := adjustNewBaseToNextGlobalID(ctx, t, tp, newBase)
newBaseTemp, err := adjustNewBaseToNextGlobalID(ctx.GetSessionVars(), t, tp, newBase)
if err != nil {
return err
}
Expand All @@ -4021,7 +4021,7 @@ func (d *ddl) RebaseAutoID(ctx sessionctx.Context, ident ast.Ident, newBase int6
return errors.Trace(err)
}

func adjustNewBaseToNextGlobalID(ctx sessionctx.Context, t table.Table, tp autoid.AllocatorType, newBase int64) (int64, error) {
func adjustNewBaseToNextGlobalID(ctx table.AllocatorContext, t table.Table, tp autoid.AllocatorType, newBase int64) (int64, error) {
alloc := t.Allocators(ctx).Get(tp)
if alloc == nil {
return newBase, nil
Expand Down
2 changes: 1 addition & 1 deletion pkg/ddl/table.go
Original file line number Diff line number Diff line change
Expand Up @@ -1007,7 +1007,7 @@ func verifyNoOverflowShardBits(s *sess.Pool, tbl table.Table, shardRowIDBits uin
}
defer s.Put(ctx)
// Check next global max auto ID first.
autoIncID, err := tbl.Allocators(ctx).Get(autoid.RowIDAllocType).NextGlobalAutoID()
autoIncID, err := tbl.Allocators(ctx.GetSessionVars()).Get(autoid.RowIDAllocType).NextGlobalAutoID()
if err != nil {
return errors.Trace(err)
}
Expand Down
6 changes: 3 additions & 3 deletions pkg/errctx/context.go
Original file line number Diff line number Diff line change
Expand Up @@ -80,8 +80,8 @@ func (ctx *Context) WithErrGroupLevels(levels LevelMap) Context {
}
}

// appendWarning appends the error to warning. If the inner `warnHandler` is nil, do nothing.
func (ctx *Context) appendWarning(err error) {
// AppendWarning appends the error to warning. If the inner `warnHandler` is nil, do nothing.
func (ctx *Context) AppendWarning(err error) {
intest.Assert(ctx.warnHandler != nil)
if w := ctx.warnHandler; w != nil {
// warnHandler should always not be nil, check fn != nil here to just make code safe.
Expand Down Expand Up @@ -148,7 +148,7 @@ func (ctx *Context) HandleErrorWithAlias(internalErr error, err error, warnErr e
case LevelError:
return err
case LevelWarn:
ctx.appendWarning(warnErr)
ctx.AppendWarning(warnErr)
case LevelIgnore:
}

Expand Down
2 changes: 1 addition & 1 deletion pkg/executor/executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -298,7 +298,7 @@ func (e *ShowNextRowIDExec) Next(_ context.Context, req *chunk.Chunk) error {
}
tblMeta := tbl.Meta()

allocators := tbl.Allocators(e.Ctx())
allocators := tbl.Allocators(e.Ctx().GetSessionVars())
for _, alloc := range allocators.Allocs {
nextGlobalID, err := alloc.NextGlobalAutoID()
if err != nil {
Expand Down
2 changes: 1 addition & 1 deletion pkg/executor/importer/importer_testkit_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -256,7 +256,7 @@ func TestPostProcess(t *testing.T) {
require.NoError(t, importer.PostProcess(ctx, tk.Session(), map[autoid.AllocatorType]int64{
autoid.RowIDAllocType: 123,
}, plan, localChecksum, logger))
allocators := table.Allocators(tk.Session())
allocators := table.Allocators(tk.Session().GetSessionVars())
nextGlobalAutoID, err := allocators.Get(autoid.RowIDAllocType).NextGlobalAutoID()
require.NoError(t, err)
require.Equal(t, int64(124), nextGlobalAutoID)
Expand Down
2 changes: 1 addition & 1 deletion pkg/executor/importer/kv_encode.go
Original file line number Diff line number Diff line change
Expand Up @@ -202,7 +202,7 @@ func (en *tableKVEncoder) fillRow(row []types.Datum, hasValue []bool, rowID int6
newRowID := en.AutoIDFn(rowID)
value = types.NewIntDatum(newRowID)
record = append(record, value)
alloc := en.Table.Allocators(en.SessionCtx).Get(autoid.RowIDAllocType)
alloc := en.Table.Allocators(en.SessionCtx.GetSessionVars()).Get(autoid.RowIDAllocType)
if err := alloc.Rebase(context.Background(), rowValue, false); err != nil {
return nil, errors.Trace(err)
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/executor/infoschema_reader.go
Original file line number Diff line number Diff line change
Expand Up @@ -229,7 +229,7 @@ func getAutoIncrementID(ctx sessionctx.Context, schema *model.DBInfo, tblInfo *m
if err != nil {
return 0, err
}
return tbl.Allocators(ctx).Get(autoid.AutoIncrementType).Base() + 1, nil
return tbl.Allocators(ctx.GetSessionVars()).Get(autoid.AutoIncrementType).Base() + 1, nil
}

func hasPriv(ctx sessionctx.Context, priv mysql.PrivilegeType) bool {
Expand Down
16 changes: 9 additions & 7 deletions pkg/executor/insert_common.go
Original file line number Diff line number Diff line change
Expand Up @@ -820,7 +820,8 @@ func (e *InsertValues) lazyAdjustAutoIncrementDatum(ctx context.Context, rows []
if !found {
return rows, nil
}
retryInfo := e.Ctx().GetSessionVars().RetryInfo
sessVars := e.Ctx().GetSessionVars()
retryInfo := sessVars.RetryInfo
rowCount := len(rows)
for processedIdx := 0; processedIdx < rowCount; processedIdx++ {
autoDatum := rows[processedIdx][idx]
Expand All @@ -835,7 +836,7 @@ func (e *InsertValues) lazyAdjustAutoIncrementDatum(ctx context.Context, rows []
}
// Use the value if it's not null and not 0.
if recordID != 0 {
alloc := e.Table.Allocators(e.Ctx()).Get(autoid.AutoIncrementType)
alloc := e.Table.Allocators(sessVars).Get(autoid.AutoIncrementType)
err = alloc.Rebase(ctx, recordID, true)
if err != nil {
return nil, err
Expand Down Expand Up @@ -906,7 +907,8 @@ func (e *InsertValues) lazyAdjustAutoIncrementDatum(ctx context.Context, rows []
func (e *InsertValues) adjustAutoIncrementDatum(
ctx context.Context, d types.Datum, hasValue bool, c *table.Column,
) (types.Datum, error) {
retryInfo := e.Ctx().GetSessionVars().RetryInfo
sessVars := e.Ctx().GetSessionVars()
retryInfo := sessVars.RetryInfo
if retryInfo.Retrying {
id, ok := retryInfo.GetCurrAutoIncrementID()
if ok {
Expand All @@ -931,7 +933,7 @@ func (e *InsertValues) adjustAutoIncrementDatum(
}
// Use the value if it's not null and not 0.
if recordID != 0 {
err = e.Table.Allocators(e.Ctx()).Get(autoid.AutoIncrementType).Rebase(ctx, recordID, true)
err = e.Table.Allocators(sessVars).Get(autoid.AutoIncrementType).Rebase(ctx, recordID, true)
if err != nil {
return types.Datum{}, err
}
Expand Down Expand Up @@ -1049,7 +1051,7 @@ func (e *InsertValues) adjustAutoRandomDatum(

// allocAutoRandomID allocates a random id for primary key column. It assumes tableInfo.AutoRandomBits > 0.
func (e *InsertValues) allocAutoRandomID(ctx context.Context, fieldType *types.FieldType) (int64, error) {
alloc := e.Table.Allocators(e.Ctx()).Get(autoid.AutoRandomType)
alloc := e.Table.Allocators(e.Ctx().GetSessionVars()).Get(autoid.AutoRandomType)
tableInfo := e.Table.Meta()
increment := e.Ctx().GetSessionVars().AutoIncrementIncrement
offset := e.Ctx().GetSessionVars().AutoIncrementOffset
Expand All @@ -1073,7 +1075,7 @@ func (e *InsertValues) rebaseAutoRandomID(ctx context.Context, recordID int64, f
if recordID < 0 {
return nil
}
alloc := e.Table.Allocators(e.Ctx()).Get(autoid.AutoRandomType)
alloc := e.Table.Allocators(e.Ctx().GetSessionVars()).Get(autoid.AutoRandomType)
tableInfo := e.Table.Meta()

shardFmt := autoid.NewShardIDFormat(fieldType, tableInfo.AutoRandomBits, tableInfo.AutoRandomRangeBits)
Expand Down Expand Up @@ -1129,7 +1131,7 @@ func (e *InsertValues) rebaseImplicitRowID(ctx context.Context, recordID int64)
if recordID < 0 {
return nil
}
alloc := e.Table.Allocators(e.Ctx()).Get(autoid.RowIDAllocType)
alloc := e.Table.Allocators(e.Ctx().GetSessionVars()).Get(autoid.RowIDAllocType)
tableInfo := e.Table.Meta()

shardFmt := autoid.NewShardIDFormat(
Expand Down
2 changes: 1 addition & 1 deletion pkg/executor/show.go
Original file line number Diff line number Diff line change
Expand Up @@ -1424,7 +1424,7 @@ func (e *ShowExec) fetchShowCreateTable() error {
tableInfo := tb.Meta()
var buf bytes.Buffer
// TODO: let the result more like MySQL.
if err = constructResultOfShowCreateTable(e.Ctx(), &e.DBName, tableInfo, tb.Allocators(e.Ctx()), &buf); err != nil {
if err = constructResultOfShowCreateTable(e.Ctx(), &e.DBName, tableInfo, tb.Allocators(e.Ctx().GetSessionVars()), &buf); err != nil {
return err
}
if tableInfo.IsView() {
Expand Down
2 changes: 1 addition & 1 deletion pkg/executor/test/ddl/ddl_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -580,7 +580,7 @@ func TestShardRowIDBits(t *testing.T) {
tbl, err = dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t1"))
require.NoError(t, err)
maxID := 1<<(64-15-1) - 1
alloc := tbl.Allocators(tk.Session()).Get(autoid.RowIDAllocType)
alloc := tbl.Allocators(tk.Session().GetSessionVars()).Get(autoid.RowIDAllocType)
err = alloc.Rebase(context.Background(), int64(maxID)-1, false)
require.NoError(t, err)
tk.MustExec("insert into t1 values(1)")
Expand Down
6 changes: 3 additions & 3 deletions pkg/executor/write.go
Original file line number Diff line number Diff line change
Expand Up @@ -103,7 +103,7 @@ func updateRecord(
if err != nil {
return false, err
}
if err = t.Allocators(sctx).Get(autoid.AutoIncrementType).Rebase(ctx, recordID, true); err != nil {
if err = t.Allocators(sctx.GetSessionVars()).Get(autoid.AutoIncrementType).Rebase(ctx, recordID, true); err != nil {
return false, err
}
}
Expand Down Expand Up @@ -306,7 +306,7 @@ func rebaseAutoRandomValue(
shardFmt := autoid.NewShardIDFormat(&col.FieldType, tableInfo.AutoRandomBits, tableInfo.AutoRandomRangeBits)
// Set bits except incremental_bits to zero.
recordID = recordID & shardFmt.IncrementalMask()
return t.Allocators(sctx).Get(autoid.AutoRandomType).Rebase(ctx, recordID, true)
return t.Allocators(sctx.GetSessionVars()).Get(autoid.AutoRandomType).Rebase(ctx, recordID, true)
}

// resetErrDataTooLong reset ErrDataTooLong error msg.
Expand Down Expand Up @@ -341,7 +341,7 @@ func checkRowForExchangePartition(sctx sessionctx.Context, row []types.Datum, tb
}
if variable.EnableCheckConstraint.Load() {
type CheckConstraintTable interface {
CheckRowConstraint(sctx sessionctx.Context, rowToCheck []types.Datum) error
CheckRowConstraint(ctx expression.EvalContext, rowToCheck []types.Datum) error
}
cc, ok := pt.(CheckConstraintTable)
if !ok {
Expand Down
9 changes: 4 additions & 5 deletions pkg/expression/simple_rewriter.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,21 +24,20 @@ import (
"github.com/pingcap/tidb/pkg/sessionctx"
"github.com/pingcap/tidb/pkg/types"
"github.com/pingcap/tidb/pkg/util"
"github.com/pingcap/tidb/pkg/util/sqlexec"
)

// ParseSimpleExprWithTableInfo parses simple expression string to Expression.
// The expression string must only reference the column in table Info.
func ParseSimpleExprWithTableInfo(ctx sessionctx.Context, exprStr string, tableInfo *model.TableInfo) (Expression, error) {
func ParseSimpleExprWithTableInfo(ctx BuildContext, exprStr string, tableInfo *model.TableInfo) (Expression, error) {
if len(exprStr) == 0 {
return nil, nil
}
exprStr = "select " + exprStr
var stmts []ast.StmtNode
var err error
var warns []error
if p, ok := ctx.(interface {
ParseSQL(context.Context, string, ...parser.ParseParam) ([]ast.StmtNode, []error, error)
}); ok {
if p, ok := ctx.(sqlexec.SQLParser); ok {
stmts, warns, err = p.ParseSQL(context.Background(), exprStr)
} else {
stmts, warns, err = parser.New().ParseSQL(exprStr)
Expand All @@ -51,7 +50,7 @@ func ParseSimpleExprWithTableInfo(ctx sessionctx.Context, exprStr string, tableI
return nil, errors.Trace(err)
}
expr := stmts[0].(*ast.SelectStmt).Fields.Fields[0].Expr
return RewriteSimpleExprWithTableInfo(ctx, tableInfo, expr, false)
return BuildExprWithAst(ctx, expr, WithSourceTable(tableInfo))
}

// ParseSimpleExprCastWithTableInfo parses simple expression string to Expression.
Expand Down
16 changes: 8 additions & 8 deletions pkg/infoschema/tables.go
Original file line number Diff line number Diff line change
Expand Up @@ -2281,22 +2281,22 @@ func (it *infoschemaTable) IndexPrefix() kv.Key {
}

// AddRecord implements table.Table AddRecord interface.
func (it *infoschemaTable) AddRecord(ctx sessionctx.Context, r []types.Datum, opts ...table.AddRecordOption) (recordID kv.Handle, err error) {
func (it *infoschemaTable) AddRecord(ctx table.MutateContext, r []types.Datum, opts ...table.AddRecordOption) (recordID kv.Handle, err error) {
return nil, table.ErrUnsupportedOp
}

// RemoveRecord implements table.Table RemoveRecord interface.
func (it *infoschemaTable) RemoveRecord(ctx sessionctx.Context, h kv.Handle, r []types.Datum) error {
func (it *infoschemaTable) RemoveRecord(ctx table.MutateContext, h kv.Handle, r []types.Datum) error {
return table.ErrUnsupportedOp
}

// UpdateRecord implements table.Table UpdateRecord interface.
func (it *infoschemaTable) UpdateRecord(gctx context.Context, ctx sessionctx.Context, h kv.Handle, oldData, newData []types.Datum, touched []bool) error {
func (it *infoschemaTable) UpdateRecord(gctx context.Context, ctx table.MutateContext, h kv.Handle, oldData, newData []types.Datum, touched []bool) error {
return table.ErrUnsupportedOp
}

// Allocators implements table.Table Allocators interface.
func (it *infoschemaTable) Allocators(_ sessionctx.Context) autoid.Allocators {
func (it *infoschemaTable) Allocators(_ table.AllocatorContext) autoid.Allocators {
return autoid.Allocators{}
}

Expand Down Expand Up @@ -2369,22 +2369,22 @@ func (vt *VirtualTable) IndexPrefix() kv.Key {
}

// AddRecord implements table.Table AddRecord interface.
func (vt *VirtualTable) AddRecord(ctx sessionctx.Context, r []types.Datum, opts ...table.AddRecordOption) (recordID kv.Handle, err error) {
func (vt *VirtualTable) AddRecord(ctx table.MutateContext, r []types.Datum, opts ...table.AddRecordOption) (recordID kv.Handle, err error) {
return nil, table.ErrUnsupportedOp
}

// RemoveRecord implements table.Table RemoveRecord interface.
func (vt *VirtualTable) RemoveRecord(ctx sessionctx.Context, h kv.Handle, r []types.Datum) error {
func (vt *VirtualTable) RemoveRecord(ctx table.MutateContext, h kv.Handle, r []types.Datum) error {
return table.ErrUnsupportedOp
}

// UpdateRecord implements table.Table UpdateRecord interface.
func (vt *VirtualTable) UpdateRecord(ctx context.Context, sctx sessionctx.Context, h kv.Handle, oldData, newData []types.Datum, touched []bool) error {
func (vt *VirtualTable) UpdateRecord(ctx context.Context, sctx table.MutateContext, h kv.Handle, oldData, newData []types.Datum, touched []bool) error {
return table.ErrUnsupportedOp
}

// Allocators implements table.Table Allocators interface.
func (vt *VirtualTable) Allocators(_ sessionctx.Context) autoid.Allocators {
func (vt *VirtualTable) Allocators(_ table.AllocatorContext) autoid.Allocators {
return autoid.Allocators{}
}

Expand Down
3 changes: 2 additions & 1 deletion pkg/planner/cardinality/trace.go
Original file line number Diff line number Diff line change
Expand Up @@ -207,9 +207,10 @@ func recordUsedItemStatsStatus(sctx sessionctx.Context, stats any, tableID, id i
// ceTraceRange appends a list of ranges and related information into CE trace
func ceTraceRange(sctx sessionctx.Context, tableID int64, colNames []string, ranges []*ranger.Range, tp string, rowCount uint64) {
sc := sctx.GetSessionVars().StmtCtx
tc := sc.TypeCtx()
allPoint := true
for _, ran := range ranges {
if !ran.IsPointNullable(sctx) {
if !ran.IsPointNullable(tc) {
allPoint = false
break
}
Expand Down
Loading

0 comments on commit 6e022a5

Please sign in to comment.